From a4b036280d77172031936383791916600b681089 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Wed, 15 Jul 2020 23:28:35 +0300 Subject: [PATCH 01/29] statsd: remove tokio01 codec Signed-off-by: Kirill Fomichev --- Cargo.toml | 2 +- src/sinks/statsd.rs | 143 ++++++++++++++++++++------------------------ 2 files changed, 67 insertions(+), 78 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f0e75d8b6ea99..567c286c07cf8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -50,7 +50,7 @@ tokio-openssl = "0.3.0" tokio-retry = "0.2.0" tokio-signal = "0.2.7" tokio-compat = { version = "0.1", features = ["rt-full"] } -tokio-util = { version = "0.3.1", features = ["codec", "compat"] } +tokio-util = { version = "0.3.1", features = ["codec", "compat", "udp"] } async-trait = "0.1" # Tracing diff --git a/src/sinks/statsd.rs b/src/sinks/statsd.rs index f64df0f2a8d44..c4c351d779566 100644 --- a/src/sinks/statsd.rs +++ b/src/sinks/statsd.rs @@ -8,7 +8,7 @@ use crate::{ topology::config::{DataType, SinkConfig, SinkContext, SinkDescription}, }; use futures::{future, FutureExt, TryFutureExt}; -use futures01::{stream::iter_ok, Sink}; +use futures01::{stream, Sink}; use serde::{Deserialize, Serialize}; use snafu::{ResultExt, Snafu}; use std::collections::BTreeMap; @@ -106,7 +106,7 @@ impl StatsdSvc { acker, ) .sink_map_err(|e| error!("Fatal statsd sink error: {}", e)) - .with_flat_map(move |event| iter_ok(encode_event(event, &namespace))); + .with_flat_map(move |event| stream::iter_ok(encode_event(event, &namespace))); Ok(Box::new(sink)) } @@ -227,14 +227,13 @@ mod test { test_util::{collect_n, runtime}, Event, }; - use bytes::Bytes; - use futures01::{future, stream::Stream, sync::mpsc, Future, Sink}; - use std::time::{Duration, Instant}; - use tokio01::{ - self, - codec::BytesCodec, - net::{UdpFramed, UdpSocket}, - }; + use bytes05::Bytes; + use futures::compat::{Future01CompatExt, Sink01CompatExt}; + use futures::{SinkExt, StreamExt, TryStreamExt}; + use futures01::{sync::mpsc, Sink}; + use std::time::Duration; + use tokio::{net::UdpSocket, time::delay_for}; + use tokio_util::{codec::BytesCodec, udp::UdpFramed}; #[cfg(feature = "sources-statsd")] use {crate::sources::statsd::parser::parse, std::str::from_utf8}; @@ -329,75 +328,65 @@ mod test { fn test_send_to_statsd() { crate::test_util::trace_init(); - let config = StatsdSinkConfig { - namespace: "vector".into(), - address: default_address(), - batch: BatchConfig { - max_bytes: Some(512), - timeout_secs: Some(1), - ..Default::default() - }, - }; - let mut rt = runtime(); - let sink = StatsdSvc::new(config, Acker::Null).unwrap(); - - let mut events = Vec::new(); - let event = Event::Metric(Metric { - name: "counter".to_owned(), - timestamp: None, - tags: Some(tags()), - kind: MetricKind::Incremental, - value: MetricValue::Counter { value: 1.5 }, - }); - events.push(event); - - let event = Event::Metric(Metric { - name: "histogram".to_owned(), - timestamp: None, - tags: None, - kind: MetricKind::Incremental, - value: MetricValue::Distribution { - values: vec![2.0], - sample_rates: vec![100], - }, - }); - events.push(event); - - let stream = iter_ok(events.into_iter()); - let sender = sink.send_all(stream); - let deadline = Instant::now() + Duration::from_millis(100); - - // Add a delay to the write side to let the read side - // poll for read interest. Otherwise, this could cause - // a race condition in noisy environments. - let sender = tokio01::timer::Delay::new(deadline) - .map_err(drop) - .and_then(|_| sender); - - let (tx, rx) = mpsc::channel(1); - - let receiver = Box::new( - future::lazy(|| { - let socket = UdpSocket::bind(&default_address()).unwrap(); - future::ok(socket) - }) - .and_then(|socket| { + rt.block_on_std(async move { + let config = StatsdSinkConfig { + namespace: "vector".into(), + address: default_address(), + batch: BatchConfig { + max_bytes: Some(512), + timeout_secs: Some(1), + ..Default::default() + }, + }; + let sink = StatsdSvc::new(config, Acker::Null).unwrap(); + + let events = vec![ + Event::Metric(Metric { + name: "counter".to_owned(), + timestamp: None, + tags: Some(tags()), + kind: MetricKind::Incremental, + value: MetricValue::Counter { value: 1.5 }, + }), + Event::Metric(Metric { + name: "histogram".to_owned(), + timestamp: None, + tags: None, + kind: MetricKind::Incremental, + value: MetricValue::Distribution { + values: vec![2.0], + sample_rates: vec![100], + }, + }), + ]; + let (tx, rx) = mpsc::channel(1); + + tokio::spawn(async move { + let socket = UdpSocket::bind(default_address()).await.unwrap(); UdpFramed::new(socket, BytesCodec::new()) .map_err(|e| error!("error reading line: {:?}", e)) - .map(|(bytes, _addr)| bytes) - .forward(tx.sink_map_err(|e| error!("error sending event: {:?}", e))) - .map(|_| ()) - }), - ); - - rt.spawn(receiver); - let _ = rt.block_on(sender).unwrap(); - - let messages = rt.block_on(collect_n(rx, 1)).ok().unwrap(); - assert_eq!( - messages[0], - Bytes::from("vector.counter:1.5|c|#empty_tag:,normal_tag:value,true_tag\nvector.histogram:2|h|@0.01") - ); + .map_ok(|(bytes, _addr)| bytes.freeze()) + .forward( + tx.sink_compat() + .sink_map_err(|e| error!("error sending event: {:?}", e)), + ) + .await + .unwrap() + }); + + // Add a delay to the write side to let the read side + // poll for read interest. Otherwise, this could cause + // a race condition in noisy environments. + delay_for(Duration::from_millis(100)).await; + let stream = stream::iter_ok(events); + let _ = sink.send_all(stream).compat().await.unwrap(); + + let messages = collect_n(rx, 1).compat().await.ok().unwrap(); + assert_eq!( + messages[0], + Bytes::from("vector.counter:1.5|c|#empty_tag:,normal_tag:value,true_tag\nvector.histogram:2|h|@0.01"), + ); + }); } } From 060105c39b2a4994851fa8a419b83eed6e3cab2f Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Thu, 16 Jul 2020 16:28:52 +0300 Subject: [PATCH 02/29] tokio-openssl: update with deprecate Signed-off-by: Kirill Fomichev --- Cargo.lock | 1 + Cargo.toml | 3 +- benches/bench.rs | 23 ++++++---- benches/buffering.rs | 6 +-- benches/http.rs | 4 +- src/sources/socket/mod.rs | 11 ++--- src/test_util.rs | 93 +++++++++++++++++++-------------------- src/tls/incoming.rs | 2 +- src/tls/mod.rs | 2 +- src/tls/outgoing.rs | 2 +- tests/crash.rs | 9 ++-- tests/syslog.rs | 17 ++++--- tests/tcp.rs | 14 +++--- 13 files changed, 98 insertions(+), 89 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 27e6f83ee8bf0..b8f04d72d2449 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5529,6 +5529,7 @@ dependencies = [ "tokio 0.2.21", "tokio-compat", "tokio-openssl 0.3.0", + "tokio-openssl 0.4.0", "tokio-retry", "tokio-signal", "tokio-test", diff --git a/Cargo.toml b/Cargo.toml index 567c286c07cf8..a20511a7d4e1a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -46,7 +46,8 @@ futures01 = { package = "futures", version = "0.1.25" } futures = { version = "0.3", default-features = false, features = ["compat", "io-compat"] } tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "uds", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync", "fs"], default-features = false } tokio = { version = "0.2.13", features = ["blocking", "fs", "sync", "macros", "test-util", "rt-core", "io-std"] } -tokio-openssl = "0.3.0" +tokio-openssl03 = { package = "tokio-openssl", version = "0.3.0" } +tokio-openssl = "0.4.0" tokio-retry = "0.2.0" tokio-signal = "0.2.7" tokio-compat = { version = "0.1", features = ["rt-full"] } diff --git a/benches/bench.rs b/benches/bench.rs index 19bc6eb69615e..97a2e599fd496 100644 --- a/benches/bench.rs +++ b/benches/bench.rs @@ -1,6 +1,7 @@ use criterion::{criterion_group, criterion_main, Benchmark, Criterion, Throughput}; use approx::assert_relative_eq; +use futures::{FutureExt, TryFutureExt}; use futures01::future; use rand::distributions::{Alphanumeric, Uniform}; use rand::prelude::*; @@ -79,7 +80,7 @@ fn benchmark_simple_pipe(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -131,7 +132,7 @@ fn benchmark_simple_pipe_with_tiny_lines(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -183,7 +184,7 @@ fn benchmark_simple_pipe_with_huge_lines(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -238,7 +239,7 @@ fn benchmark_simple_pipe_with_many_writers(c: &mut Criterion) { let sends = (0..num_writers) .map(|_| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - futures01::sync::oneshot::spawn(send, &rt.executor()) + futures01::sync::oneshot::spawn(send.boxed().compat(), &rt.executor()) }) .collect::>(); @@ -314,8 +315,10 @@ fn benchmark_interconnected(c: &mut Criterion) { |(mut rt, topology, output_lines1, output_lines2)| { let send1 = send_lines(in_addr1, random_lines(line_size).take(num_lines)); let send2 = send_lines(in_addr2, random_lines(line_size).take(num_lines)); - let sends = vec![send1, send2]; - rt.block_on(future::join_all(sends)).unwrap(); + rt.block_on_std(async move { + send1.await.unwrap(); + send2.await.unwrap(); + }); block_on(topology.stop()).unwrap(); @@ -389,7 +392,7 @@ fn benchmark_transforms(c: &mut Criterion) { .map(|l| l + "status=404") .take(num_lines), ); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -575,7 +578,8 @@ fn benchmark_complex(c: &mut Criterion) { )| { // One sender generates pure random lines let send1 = send_lines(in_addr1, random_lines(100).take(num_lines)); - let send1 = futures01::sync::oneshot::spawn(send1, &rt.executor()); + let send1 = + futures01::sync::oneshot::spawn(send1.boxed().compat(), &rt.executor()); // The other includes either status=200 or status=404 let mut rng = SmallRng::from_rng(thread_rng()).unwrap(); @@ -590,7 +594,8 @@ fn benchmark_complex(c: &mut Criterion) { }) .take(num_lines), ); - let send2 = futures01::sync::oneshot::spawn(send2, &rt.executor()); + let send2 = + futures01::sync::oneshot::spawn(send2.boxed().compat(), &rt.executor()); let sends = vec![send1, send2]; rt.block_on(future::join_all(sends)).unwrap(); diff --git a/benches/buffering.rs b/benches/buffering.rs index f4102fa39caee..55d51d885878f 100644 --- a/benches/buffering.rs +++ b/benches/buffering.rs @@ -52,7 +52,7 @@ fn benchmark_buffers(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -94,7 +94,7 @@ fn benchmark_buffers(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); @@ -136,7 +136,7 @@ fn benchmark_buffers(c: &mut Criterion) { }, |(mut rt, topology, output_lines)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); block_on(topology.stop()).unwrap(); diff --git a/benches/http.rs b/benches/http.rs index f54d1a8a6a4d4..ffa6c6bc808d7 100644 --- a/benches/http.rs +++ b/benches/http.rs @@ -56,7 +56,7 @@ fn benchmark_http_no_compression(c: &mut Criterion) { }, |(mut rt, topology)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); rt.block_on(topology.stop()).unwrap(); @@ -114,7 +114,7 @@ fn benchmark_http_gzip(c: &mut Criterion) { }, |(mut rt, topology)| { let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); rt.block_on(topology.stop()).unwrap(); diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index ff8327d67b2e0..2e5c6217d6618 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -177,7 +177,7 @@ mod test { rt.spawn(server); wait_for_tcp(addr); - rt.block_on(send_lines(addr, vec!["test".to_owned()].into_iter())) + rt.block_on_std(send_lines(addr, vec!["test".to_owned()].into_iter())) .unwrap(); let event = rx.wait().next().unwrap().unwrap(); @@ -205,7 +205,7 @@ mod test { rt.spawn(server); wait_for_tcp(addr); - rt.block_on(send_lines(addr, vec!["test".to_owned()].into_iter())) + rt.block_on_std(send_lines(addr, vec!["test".to_owned()].into_iter())) .unwrap(); let event = rx.wait().next().unwrap().unwrap(); @@ -242,7 +242,8 @@ mod test { "more short".to_owned(), ]; - rt.block_on(send_lines(addr, lines.into_iter())).unwrap(); + rt.block_on_std(send_lines(addr, lines.into_iter())) + .unwrap(); let (event, rx) = block_on(rx.into_future()).unwrap(); assert_eq!( @@ -292,7 +293,7 @@ mod test { "more short".to_owned(), ]; - rt.block_on(send_lines_tls(addr, "localhost".into(), lines.into_iter())) + rt.block_on_std(send_lines_tls(addr, "localhost".into(), lines.into_iter())) .unwrap(); let (event, rx) = block_on(rx.into_future()).unwrap(); @@ -326,7 +327,7 @@ mod test { wait_for_tcp(addr); // Send data to Source. - rt.block_on(send_lines(addr, vec!["test".to_owned()].into_iter())) + rt.block_on_std(send_lines(addr, vec!["test".to_owned()].into_iter())) .unwrap(); let event = rx.wait().next().unwrap().unwrap(); diff --git a/src/test_util.rs b/src/test_util.rs index ed1bda3466376..dfba50ed20cba 100644 --- a/src/test_util.rs +++ b/src/test_util.rs @@ -1,11 +1,15 @@ use crate::runtime::Runtime; use crate::{event::LogEvent, Event}; -use futures01::{future, stream, sync::mpsc, try_ready, Async, Future, Poll, Sink, Stream}; +use futures::{stream, SinkExt, StreamExt}; +use futures01::{ + future, stream as stream01, sync::mpsc, try_ready, Async, Future, Poll, Sink, Stream, +}; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use rand::distributions::Alphanumeric; use rand::{thread_rng, Rng}; use std::collections::HashMap; +use std::convert::Infallible; use std::fs::File; use std::io::Read; use std::iter; @@ -14,11 +18,14 @@ use std::net::{Shutdown, SocketAddr}; use std::path::Path; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; -use stream_cancel::{StreamExt, Trigger, Tripwire}; -use tokio01::codec::{Encoder, FramedRead, FramedWrite, LinesCodec}; -use tokio01::net::{TcpListener, TcpStream}; +use stream_cancel::{StreamExt as _, Trigger, Tripwire}; +use tokio::net::TcpStream; +use tokio01::codec::{ + Encoder, FramedRead, FramedWrite as FramedWrite01, LinesCodec as LinesCodec01, +}; +use tokio01::net::{TcpListener, TcpStream as TcpStream01}; use tokio01::util::FutureExt; -use tokio_openssl::SslConnectorExt; +use tokio_util::codec::{FramedWrite, LinesCodec}; #[macro_export] macro_rules! assert_downcast_matches { @@ -50,24 +57,26 @@ pub fn trace_init() { let _ = tracing::dispatcher::set_global_default(tracing::Dispatch::new(subscriber)); } -pub fn send_lines( +use futures::compat::Future01CompatExt; + +pub async fn send_lines( addr: SocketAddr, lines: impl Iterator, -) -> impl Future { - send_encodable(addr, LinesCodec::new(), lines) +) -> Result<(), Infallible> { + send_encodable(addr, LinesCodec01::new(), lines).await } -pub fn send_encodable( +pub async fn send_encodable( addr: SocketAddr, encoder: impl Encoder, items: impl Iterator, -) -> impl Future { - let items = futures01::stream::iter_ok::<_, ()>(items); +) -> Result<(), Infallible> { + let items = stream01::iter_ok::<_, ()>(items); - TcpStream::connect(&addr) + TcpStream01::connect(&addr) .map_err(|e| panic!("{:}", e)) .and_then(|socket| { - let out = FramedWrite::new(socket, encoder).sink_map_err(|e| panic!("{:?}", e)); + let out = FramedWrite01::new(socket, encoder).sink_map_err(|e| panic!("{:?}", e)); items .forward(out) @@ -82,45 +91,33 @@ pub fn send_encodable( }) .map(|_| ()) }) + .compat() + .await + .unwrap(); + Ok(()) } -pub fn send_lines_tls( +pub async fn send_lines_tls( addr: SocketAddr, host: String, lines: impl Iterator, -) -> impl Future { - let lines = futures01::stream::iter_ok::<_, ()>(lines); +) -> Result<(), Infallible> { + let stream = TcpStream::connect(&addr).await.unwrap(); - let mut connector = - SslConnector::builder(SslMethod::tls()).expect("Failed to create TLS builder"); + let mut connector = SslConnector::builder(SslMethod::tls()).unwrap(); connector.set_verify(SslVerifyMode::NONE); - let connector = connector.build(); + let config = connector.build().configure().unwrap(); - TcpStream::connect(&addr) - .map_err(|e| panic!("{:}", e)) - .and_then(move |socket| { - connector - .connect_async(&host, socket) - .map_err(|e| panic!("{:}", e)) - .and_then(|stream| { - let out = FramedWrite::new(stream, LinesCodec::new()) - .sink_map_err(|e| panic!("{:?}", e)); - - lines - .forward(out) - .and_then(|(_source, sink)| { - let mut stream = sink.into_inner().into_inner(); - // We should catch TLS shutdown errors here, - // but doing so results in a repeatable - // "Resource temporarily available" error, - // and tests will be checking that contents - // are received anyways. - stream.get_mut().shutdown().ok(); - Ok(()) - }) - .map(|_| ()) - }) - }) + let stream = tokio_openssl::connect(config, &host, stream).await.unwrap(); + let mut sink = FramedWrite::new(stream, LinesCodec::new()); + + let mut lines = stream::iter(lines).map(|line| Ok(line)); + sink.send_all(&mut lines).await.unwrap(); + + let stream = sink.get_mut().get_mut(); + stream.shutdown(std::net::Shutdown::Both).unwrap(); + + Ok(()) } pub fn temp_file() -> std::path::PathBuf { @@ -140,7 +137,7 @@ pub fn random_lines_with_stream( count: usize, ) -> (Vec, impl Stream) { let lines = (0..count).map(|_| random_string(len)).collect::>(); - let stream = stream::iter_ok(lines.clone().into_iter().map(Event::from)); + let stream = stream01::iter_ok(lines.clone().into_iter().map(Event::from)); (lines, stream) } @@ -396,7 +393,7 @@ pub fn receive(addr: &SocketAddr) -> Receiver { let lines = listener .incoming() .take_until(tripwire) - .map(|socket| FramedRead::new(socket, LinesCodec::new())) + .map(|socket| FramedRead::new(socket, LinesCodec01::new())) .flatten() .inspect(move |_| { count_clone.fetch_add(1, Ordering::Relaxed); @@ -468,7 +465,7 @@ pub fn count_receive(addr: &SocketAddr) -> CountReceiver { let count = listener .incoming() .take_until(tripwire) - .map(|socket| FramedRead::new(socket, LinesCodec::new())) + .map(|socket| FramedRead::new(socket, LinesCodec01::new())) .flatten() .map_err(|e| panic!("{:?}", e)) .fold(0, |n, _| future::ok(n + 1)); @@ -489,7 +486,7 @@ where F: Fn() -> Event, { let events = (0..count).map(|_| generator()).collect::>(); - let stream = stream::iter_ok(events.clone().into_iter()); + let stream = stream01::iter_ok(events.clone().into_iter()); (events, stream) } diff --git a/src/tls/incoming.rs b/src/tls/incoming.rs index 880a3196d659a..b6347dfb4d79f 100644 --- a/src/tls/incoming.rs +++ b/src/tls/incoming.rs @@ -14,7 +14,7 @@ use tokio01::{ io::{AsyncRead, AsyncWrite}, net::{tcp::Incoming, TcpListener, TcpStream}, }; -use tokio_openssl::{AcceptAsync, SslAcceptorExt}; +use tokio_openssl03::{AcceptAsync, SslAcceptorExt}; pub(crate) struct MaybeTlsIncoming { incoming: I, diff --git a/src/tls/mod.rs b/src/tls/mod.rs index 8dd01b3667103..d0b07be96b9d1 100644 --- a/src/tls/mod.rs +++ b/src/tls/mod.rs @@ -8,7 +8,7 @@ use std::io::Error as IoError; use std::net::SocketAddr; use std::path::PathBuf; use tokio01::net::TcpStream; -use tokio_openssl::SslStream; +use tokio_openssl03::SslStream; #[cfg(feature = "sources-tls")] mod incoming; diff --git a/src/tls/outgoing.rs b/src/tls/outgoing.rs index ba009f5cfb29c..e5cfe4dec40e5 100644 --- a/src/tls/outgoing.rs +++ b/src/tls/outgoing.rs @@ -3,7 +3,7 @@ use futures01::{Async, Future}; use openssl::ssl::{ConnectConfiguration, HandshakeError}; use std::net::SocketAddr; use tokio01::net::tcp::{ConnectFuture, TcpStream}; -use tokio_openssl::{ConnectAsync, ConnectConfigurationExt}; +use tokio_openssl03::{ConnectAsync, ConnectConfigurationExt}; enum State { Connecting(ConnectFuture, Option), diff --git a/tests/crash.rs b/tests/crash.rs index af02e5eeb2905..f1d0f4aacd973 100644 --- a/tests/crash.rs +++ b/tests/crash.rs @@ -82,8 +82,7 @@ fn test_sink_panic() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); - let mut rt2 = runtime(); - rt2.block_on(send).unwrap(); + runtime().block_on_std(send).unwrap(); std::thread::sleep(std::time::Duration::from_millis(100)); let _ = std::panic::take_hook(); @@ -165,7 +164,7 @@ fn test_sink_error() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); let mut rt2 = runtime(); - rt2.block_on(send).unwrap(); + rt2.block_on_std(send).unwrap(); std::thread::sleep(std::time::Duration::from_millis(100)); assert!(crash.wait().next().is_some()); @@ -233,7 +232,7 @@ fn test_source_error() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); let mut rt2 = runtime(); - rt2.block_on(send).unwrap(); + rt2.block_on_std(send).unwrap(); std::thread::sleep(std::time::Duration::from_millis(100)); assert!(crash.wait().next().is_some()); @@ -304,7 +303,7 @@ fn test_source_panic() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); let mut rt2 = runtime(); - rt2.block_on(send).unwrap(); + rt2.block_on_std(send).unwrap(); std::thread::sleep(std::time::Duration::from_millis(100)); let _ = std::panic::take_hook(); diff --git a/tests/syslog.rs b/tests/syslog.rs index 69aec69b0e218..df030bb89c79c 100644 --- a/tests/syslog.rs +++ b/tests/syslog.rs @@ -55,7 +55,9 @@ fn test_tcp_syslog() { let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); - block_on(send_lines(in_addr, input_lines.into_iter())).unwrap(); + runtime() + .block_on_std(send_lines(in_addr, input_lines.into_iter())) + .unwrap(); // Shut down server block_on(topology.stop()).unwrap(); @@ -193,12 +195,13 @@ fn test_octet_counting_syslog() { }) .collect(); - block_on(send_encodable( - in_addr, - BytesCodec::new(), - input_lines.into_iter().map(Into::into), - )) - .unwrap(); + runtime() + .block_on_std(send_encodable( + in_addr, + BytesCodec::new(), + input_lines.into_iter().map(Into::into), + )) + .unwrap(); // Shut down server block_on(topology.stop()).unwrap(); diff --git a/tests/tcp.rs b/tests/tcp.rs index 751508653f16c..89991e4d8fd06 100644 --- a/tests/tcp.rs +++ b/tests/tcp.rs @@ -43,7 +43,7 @@ fn pipe() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); // Shut down server block_on(topology.stop()).unwrap(); @@ -91,7 +91,7 @@ fn sample() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); // Shut down server block_on(topology.stop()).unwrap(); @@ -146,7 +146,7 @@ fn fork() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); // Shut down server block_on(topology.stop()).unwrap(); @@ -205,8 +205,10 @@ fn merge_and_fork() { let input_lines2 = random_lines(100).take(num_lines).collect::>(); let send1 = send_lines(in_addr1, input_lines1.clone().into_iter()); let send2 = send_lines(in_addr2, input_lines2.clone().into_iter()); - let send = send1.join(send2); - rt.block_on(send).unwrap(); + rt.block_on_std(async move { + send1.await.unwrap(); + send2.await.unwrap(); + }); // Shut down server block_on(topology.stop()).unwrap(); @@ -274,7 +276,7 @@ fn reconnect() { let input_lines = random_lines(100).take(num_lines).collect::>(); let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on(send).unwrap(); + rt.block_on_std(send).unwrap(); // Shut down server and wait for it to fully flush block_on(topology.stop()).unwrap(); From 4c2e0f44597f323af76b3f93e30bee6bf0b366ba Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Thu, 16 Jul 2020 17:09:15 +0300 Subject: [PATCH 03/29] test_util: remove old codec Signed-off-by: Kirill Fomichev --- benches/bench.rs | 276 ++++++++++++++++++++++--------------------- benches/buffering.rs | 92 ++++++++------- src/sinks/socket.rs | 35 +++--- src/test_util.rs | 240 ++++++++++++++----------------------- tests/buffering.rs | 127 ++++++++++---------- tests/crash.rs | 174 ++++++++++++++------------- tests/support/mod.rs | 7 +- tests/syslog.rs | 271 +++++++++++++++++++++--------------------- tests/tcp.rs | 189 +++++++++++++++-------------- tests/topology.rs | 24 ++-- 10 files changed, 687 insertions(+), 748 deletions(-) diff --git a/benches/bench.rs b/benches/bench.rs index 97a2e599fd496..3578589db02e9 100644 --- a/benches/bench.rs +++ b/benches/bench.rs @@ -1,18 +1,19 @@ use criterion::{criterion_group, criterion_main, Benchmark, Criterion, Throughput}; use approx::assert_relative_eq; -use futures::{FutureExt, TryFutureExt}; -use futures01::future; -use rand::distributions::{Alphanumeric, Uniform}; -use rand::prelude::*; +use futures::{compat::Future01CompatExt, future, stream, StreamExt}; +use rand::{ + distributions::{Alphanumeric, Uniform}, + prelude::*, +}; use std::convert::TryFrom; use vector::event::Event; use vector::test_util::{ - block_on, count_receive, next_addr, send_lines, shutdown_on_idle, wait_for_tcp, + next_addr, runtime, send_lines, shutdown_on_idle, wait_for_tcp, CountReceiver, }; use vector::topology::config::{TransformConfig, TransformContext}; use vector::topology::{self, config}; -use vector::{runtime, sinks, sources, transforms}; +use vector::{sinks, sources, transforms}; mod batch; mod buffering; @@ -68,24 +69,24 @@ fn benchmark_simple_pipe(c: &mut Criterion) { ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -120,24 +121,24 @@ fn benchmark_simple_pipe_with_tiny_lines(c: &mut Criterion) { ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -172,24 +173,24 @@ fn benchmark_simple_pipe_with_huge_lines(c: &mut Criterion) { ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -225,32 +226,31 @@ fn benchmark_simple_pipe_with_many_writers(c: &mut Criterion) { ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let sends = (0..num_writers) - .map(|_| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - futures01::sync::oneshot::spawn(send.boxed().compat(), &rt.executor()) - }) - .collect::>(); - - rt.block_on(future::join_all(sends)).unwrap(); - - std::thread::sleep(std::time::Duration::from_millis(100)); - - block_on(topology.stop()).unwrap(); + rt.block_on_std(async move { + let sends = stream::iter(0..num_writers) + .map(|_| { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines) + }) + .collect::>() + .await; + future::try_join_all(sends).await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines * num_writers, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines * num_writers, output_lines.wait()); }, ); }) @@ -300,31 +300,29 @@ fn benchmark_interconnected(c: &mut Criterion) { ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines1 = count_receive(&out_addr1); - let output_lines2 = count_receive(&out_addr2); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines1, output_lines2, topology) = rt.block_on_std(async move { + let output_lines1 = CountReceiver::receive_lines(out_addr1); + let output_lines2 = CountReceiver::receive_lines(out_addr2); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines1, output_lines2, topology) + }); wait_for_tcp(in_addr1); wait_for_tcp(in_addr2); - (rt, topology, output_lines1, output_lines2) }, |(mut rt, topology, output_lines1, output_lines2)| { - let send1 = send_lines(in_addr1, random_lines(line_size).take(num_lines)); - let send2 = send_lines(in_addr2, random_lines(line_size).take(num_lines)); rt.block_on_std(async move { - send1.await.unwrap(); - send2.await.unwrap(); + let lines1 = random_lines(line_size).take(num_lines); + send_lines(in_addr1, lines1).await.unwrap(); + let lines2 = random_lines(line_size).take(num_lines); + send_lines(in_addr2, lines2).await.unwrap(); + + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines * 2, output_lines1.wait().await.len()); + assert_eq!(num_lines * 2, output_lines2.wait().await.len()); }); - - block_on(topology.stop()).unwrap(); - shutdown_on_idle(rt); - assert_eq!(num_lines * 2, output_lines1.wait()); - assert_eq!(num_lines * 2, output_lines2.wait()); }, ); }) @@ -375,29 +373,27 @@ fn benchmark_transforms(c: &mut Criterion) { out_addr.to_string(), ), ); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines( - in_addr, - random_lines(line_size) + rt.block_on_std(async move { + let lines = random_lines(line_size) .map(|l| l + "status=404") - .take(num_lines), - ); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); + .take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -547,18 +543,30 @@ fn benchmark_complex(c: &mut Criterion) { out_addr_500.to_string(), ), ); - let mut rt = runtime::Runtime::new().unwrap(); - let output_lines_all = count_receive(&out_addr_all); - let output_lines_sampled = count_receive(&out_addr_sampled); - let output_lines_200 = count_receive(&out_addr_200); - let output_lines_404 = count_receive(&out_addr_404); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let ( + output_lines_all, + output_lines_sampled, + output_lines_200, + output_lines_404, + topology, + ) = rt.block_on_std(async move { + let output_lines_all = CountReceiver::receive_lines(out_addr_all); + let output_lines_sampled = CountReceiver::receive_lines(out_addr_sampled); + let output_lines_200 = CountReceiver::receive_lines(out_addr_200); + let output_lines_404 = CountReceiver::receive_lines(out_addr_404); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + ( + output_lines_all, + output_lines_sampled, + output_lines_200, + output_lines_404, + topology, + ) + }); wait_for_tcp(in_addr1); wait_for_tcp(in_addr2); - ( rt, topology, @@ -576,47 +584,41 @@ fn benchmark_complex(c: &mut Criterion) { output_lines_200, output_lines_404, )| { - // One sender generates pure random lines - let send1 = send_lines(in_addr1, random_lines(100).take(num_lines)); - let send1 = - futures01::sync::oneshot::spawn(send1.boxed().compat(), &rt.executor()); - - // The other includes either status=200 or status=404 - let mut rng = SmallRng::from_rng(thread_rng()).unwrap(); - let send2 = send_lines( - in_addr2, - random_lines(100) + rt.block_on_std(async move { + // One sender generates pure random lines + let lines1 = random_lines(100).take(num_lines); + send_lines(in_addr1, lines1).await.unwrap(); + + // The other includes either status=200 or status=404 + let mut rng = SmallRng::from_rng(thread_rng()).unwrap(); + let lines2 = random_lines(100) .map(move |mut l| { let status = if rng.gen_bool(0.5) { "200" } else { "404" }; l += "status="; l += status; l }) - .take(num_lines), - ); - let send2 = - futures01::sync::oneshot::spawn(send2.boxed().compat(), &rt.executor()); - let sends = vec![send1, send2]; - rt.block_on(future::join_all(sends)).unwrap(); - - block_on(topology.stop()).unwrap(); - + .take(num_lines); + send_lines(in_addr2, lines2).await.unwrap(); + + topology.stop().compat().await.unwrap(); + + let output_lines_all = output_lines_all.wait().await.len(); + let output_lines_sampled = output_lines_sampled.wait().await.len(); + let output_lines_200 = output_lines_200.wait().await.len(); + let output_lines_404 = output_lines_404.wait().await.len(); + + assert_eq!(output_lines_all, num_lines * 2); + assert_relative_eq!( + output_lines_sampled as f32 / num_lines as f32, + 0.1, + epsilon = 0.01 + ); + assert!(output_lines_200 > 0); + assert!(output_lines_404 > 0); + assert_eq!(output_lines_200 + output_lines_404, num_lines); + }); shutdown_on_idle(rt); - - let output_lines_all = output_lines_all.wait(); - let output_lines_sampled = output_lines_sampled.wait(); - let output_lines_200 = output_lines_200.wait(); - let output_lines_404 = output_lines_404.wait(); - - assert_eq!(output_lines_all, num_lines * 2); - assert_relative_eq!( - output_lines_sampled as f32 / num_lines as f32, - 0.1, - epsilon = 0.01 - ); - assert!(output_lines_200 > 0); - assert!(output_lines_404 > 0); - assert_eq!(output_lines_200 + output_lines_404, num_lines); }, ); }) diff --git a/benches/buffering.rs b/benches/buffering.rs index 55d51d885878f..5db5a3a1fc137 100644 --- a/benches/buffering.rs +++ b/benches/buffering.rs @@ -1,11 +1,16 @@ use criterion::{criterion_group, criterion_main, Benchmark, Criterion, Throughput}; +use futures::compat::Future01CompatExt; +use rand::{distributions::Alphanumeric, rngs::SmallRng, thread_rng, Rng, SeedableRng}; use tempfile::tempdir; use vector::test_util::{ - block_on, count_receive, next_addr, send_lines, shutdown_on_idle, wait_for_tcp, + next_addr, runtime, send_lines, shutdown_on_idle, wait_for_tcp, CountReceiver, +}; +use vector::{ + buffers::BufferConfig, + sinks, sources, + topology::{self, config}, }; -use vector::topology::{self, config}; -use vector::{buffers::BufferConfig, runtime, sinks, sources}; fn benchmark_buffers(c: &mut Criterion) { let num_lines: usize = 100_000; @@ -40,24 +45,24 @@ fn benchmark_buffers(c: &mut Criterion) { when_full: Default::default(), }; - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -82,24 +87,25 @@ fn benchmark_buffers(c: &mut Criterion) { }; config.global.data_dir = Some(data_dir.clone()); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); - + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + tokio::time::delay_for(std::time::Duration::from_secs(100)).await; + + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -124,24 +130,25 @@ fn benchmark_buffers(c: &mut Criterion) { }; config.global.data_dir = Some(data_dir2.clone()); - let mut rt = runtime::Runtime::new().unwrap(); - - let output_lines = count_receive(&out_addr); - - let (topology, _crash) = - rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (output_lines, topology) = rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + (output_lines, topology) + }); wait_for_tcp(in_addr); - (rt, topology, output_lines) }, |(mut rt, topology, output_lines)| { - let send = send_lines(in_addr, random_lines(line_size).take(num_lines)); - rt.block_on_std(send).unwrap(); - - block_on(topology.stop()).unwrap(); - + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines); + send_lines(in_addr, lines).await.unwrap(); + tokio::time::delay_for(std::time::Duration::from_secs(100)).await; + + topology.stop().compat().await.unwrap(); + assert_eq!(num_lines, output_lines.wait().await.len()); + }); shutdown_on_idle(rt); - assert_eq!(num_lines, output_lines.wait()); }, ); }) @@ -155,9 +162,6 @@ criterion_group!(buffers, benchmark_buffers); criterion_main!(buffers); fn random_lines(size: usize) -> impl Iterator { - use rand::distributions::Alphanumeric; - use rand::{rngs::SmallRng, thread_rng, Rng, SeedableRng}; - let mut rng = SmallRng::from_rng(thread_rng()).unwrap(); std::iter::repeat(()).map(move |_| { diff --git a/src/sinks/socket.rs b/src/sinks/socket.rs index 8ac6a3eb9afd5..d79721437dd62 100644 --- a/src/sinks/socket.rs +++ b/src/sinks/socket.rs @@ -88,7 +88,7 @@ mod test { use super::*; use crate::{ event::Event, - test_util::{next_addr, random_lines_with_stream, receive, runtime}, + test_util::{next_addr, random_lines_with_stream, runtime, CountReceiver}, topology::config::SinkContext, }; use futures::compat::Future01CompatExt; @@ -141,25 +141,26 @@ mod test { }), }; let mut rt = runtime(); - let context = SinkContext::new_test(); - let (sink, _healthcheck) = config.build(context).unwrap(); + rt.block_on_std(async move { + let context = SinkContext::new_test(); + let (sink, _healthcheck) = config.build(context).unwrap(); - let receiver = receive(&addr); + let receiver = CountReceiver::receive_lines(addr); - let (lines, events) = random_lines_with_stream(10, 100); - let pump = sink.send_all(events); - let _ = rt.block_on(pump).unwrap(); - - // Some CI machines are very slow, be generous. - std::thread::sleep(std::time::Duration::from_secs(2)); + let (lines, events) = random_lines_with_stream(10, 100); + let _ = sink.send_all(events).compat().await.unwrap(); - let output = receiver.wait(); - assert_eq!(output.len(), lines.len()); - for (source, received) in lines.iter().zip(output) { - let json = serde_json::from_str::(&received).expect("Invalid JSON"); - let received = json.get("message").unwrap().as_str().unwrap(); - assert_eq!(source, received); - } + // Some CI machines are very slow, be generous. + std::thread::sleep(std::time::Duration::from_secs(2)); + + let output = receiver.wait().await; + assert_eq!(lines.len(), output.len()); + for (source, received) in lines.iter().zip(output) { + let json = serde_json::from_str::(&received).expect("Invalid JSON"); + let received = json.get("message").unwrap().as_str().unwrap(); + assert_eq!(source, received); + } + }); } // This is a test that checks that we properly receieve all events in the diff --git a/src/test_util.rs b/src/test_util.rs index dfba50ed20cba..22a23de38c186 100644 --- a/src/test_util.rs +++ b/src/test_util.rs @@ -1,31 +1,28 @@ use crate::runtime::Runtime; use crate::{event::LogEvent, Event}; -use futures::{stream, SinkExt, StreamExt}; -use futures01::{ - future, stream as stream01, sync::mpsc, try_ready, Async, Future, Poll, Sink, Stream, -}; +use futures::{compat::Stream01CompatExt, stream, SinkExt, StreamExt, TryStreamExt}; +use futures01::{future, stream as stream01, sync::mpsc, try_ready, Async, Future, Poll, Stream}; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; -use rand::distributions::Alphanumeric; -use rand::{thread_rng, Rng}; -use std::collections::HashMap; -use std::convert::Infallible; -use std::fs::File; -use std::io::Read; -use std::iter; -use std::mem; -use std::net::{Shutdown, SocketAddr}; -use std::path::Path; -use std::sync::atomic::{AtomicUsize, Ordering}; -use std::sync::Arc; -use stream_cancel::{StreamExt as _, Trigger, Tripwire}; -use tokio::net::TcpStream; -use tokio01::codec::{ - Encoder, FramedRead, FramedWrite as FramedWrite01, LinesCodec as LinesCodec01, +use rand::{distributions::Alphanumeric, thread_rng, Rng}; +use std::{ + collections::HashMap, + convert::Infallible, + fs::File, + io::Read, + iter, mem, + net::{Shutdown, SocketAddr}, + path::Path, + sync::atomic::{AtomicUsize, Ordering}, + sync::Arc, +}; +use tokio::{ + net::{TcpListener, TcpStream}, + sync::oneshot, + task::JoinHandle, }; -use tokio01::net::{TcpListener, TcpStream as TcpStream01}; use tokio01::util::FutureExt; -use tokio_util::codec::{FramedWrite, LinesCodec}; +use tokio_util::codec::{Encoder, FramedRead, FramedWrite, LinesCodec}; #[macro_export] macro_rules! assert_downcast_matches { @@ -57,43 +54,27 @@ pub fn trace_init() { let _ = tracing::dispatcher::set_global_default(tracing::Dispatch::new(subscriber)); } -use futures::compat::Future01CompatExt; - pub async fn send_lines( addr: SocketAddr, - lines: impl Iterator, + lines: impl IntoIterator, ) -> Result<(), Infallible> { - send_encodable(addr, LinesCodec01::new(), lines).await + send_encodable(addr, LinesCodec::new(), lines).await } -pub async fn send_encodable( +pub async fn send_encodable + std::fmt::Debug>( addr: SocketAddr, - encoder: impl Encoder, - items: impl Iterator, + encoder: impl Encoder, + lines: impl IntoIterator, ) -> Result<(), Infallible> { - let items = stream01::iter_ok::<_, ()>(items); - - TcpStream01::connect(&addr) - .map_err(|e| panic!("{:}", e)) - .and_then(|socket| { - let out = FramedWrite01::new(socket, encoder).sink_map_err(|e| panic!("{:?}", e)); - - items - .forward(out) - .and_then(|(_source, sink)| { - let socket = sink.into_inner().into_inner(); - // In tokio 0.1 `AsyncWrite::shutdown` for `TcpStream` is a noop. - // See https://docs.rs/tokio-tcp/0.1.4/src/tokio_tcp/stream.rs.html#917 - // Use `TcpStream::shutdown` instead - it actually does something. - socket - .shutdown(Shutdown::Both) - .map_err(|e| panic!("{:}", e)) - }) - .map(|_| ()) - }) - .compat() - .await - .unwrap(); + let stream = TcpStream::connect(&addr).await.unwrap(); + let mut sink = FramedWrite::new(stream, encoder); + + let mut lines = stream::iter(lines.into_iter()).map(Ok); + sink.send_all(&mut lines).await.unwrap(); + + let stream = sink.get_mut(); + stream.shutdown(Shutdown::Both).unwrap(); + Ok(()) } @@ -111,11 +92,11 @@ pub async fn send_lines_tls( let stream = tokio_openssl::connect(config, &host, stream).await.unwrap(); let mut sink = FramedWrite::new(stream, LinesCodec::new()); - let mut lines = stream::iter(lines).map(|line| Ok(line)); + let mut lines = stream::iter(lines).map(Ok); sink.send_all(&mut lines).await.unwrap(); let stream = sink.get_mut().get_mut(); - stream.shutdown(std::net::Shutdown::Both).unwrap(); + stream.shutdown(Shutdown::Both).unwrap(); Ok(()) } @@ -362,119 +343,76 @@ where } } -pub struct Receiver { - handle: futures01::sync::oneshot::SpawnHandle, ()>, +pub struct CountReceiver { count: Arc, - trigger: Trigger, - _runtime: Runtime, + trigger: oneshot::Sender<()>, + handle: JoinHandle>, } -impl Receiver { +impl CountReceiver { pub fn count(&self) -> usize { self.count.load(Ordering::Relaxed) } - pub fn wait(self) -> Vec { - self.trigger.cancel(); - self.handle.wait().unwrap() + pub async fn wait(self) -> Vec { + let _ = self.trigger.send(()); + self.handle.await.unwrap() } -} - -pub fn receive(addr: &SocketAddr) -> Receiver { - let runtime = runtime(); - - let listener = TcpListener::bind(addr).unwrap(); - - let count = Arc::new(AtomicUsize::new(0)); - let count_clone = Arc::clone(&count); - let (trigger, tripwire) = Tripwire::new(); - - let lines = listener - .incoming() - .take_until(tripwire) - .map(|socket| FramedRead::new(socket, LinesCodec01::new())) - .flatten() - .inspect(move |_| { - count_clone.fetch_add(1, Ordering::Relaxed); - }) - .map_err(|e| panic!("{:?}", e)) - .collect(); - - let handle = futures01::sync::oneshot::spawn(lines, &runtime.executor()); - Receiver { - handle, - count, - trigger, - _runtime: runtime, + fn new(make_fut: F) -> CountReceiver + where + F: FnOnce(Arc, oneshot::Receiver<()>) -> Fut, + Fut: std::future::Future> + Send + 'static, + { + let count = Arc::new(AtomicUsize::new(0)); + let (trigger, tripwire) = oneshot::channel(); + + CountReceiver { + count: Arc::clone(&count), + trigger, + handle: tokio::spawn(make_fut(count, tripwire)), + } } } -pub fn receive_events(stream: S) -> Receiver -where - S: Stream + Send + 'static, - ::Error: std::fmt::Debug, -{ - let runtime = runtime(); - - let count = Arc::new(AtomicUsize::new(0)); - let count_clone = Arc::clone(&count); - - let (trigger, tripwire) = Tripwire::new(); - - let events = stream - .take_until(tripwire) - .inspect(move |_| { - count_clone.fetch_add(1, Ordering::Relaxed); +impl CountReceiver { + pub fn receive_lines(addr: SocketAddr) -> CountReceiver { + CountReceiver::new(|count, tripwire| async move { + TcpListener::bind(addr) + .await + .unwrap() + .incoming() + .take_until(tripwire) + .map_ok(|socket| FramedRead::new(socket, LinesCodec::new())) + .map(|x| x.unwrap()) + .flatten() + .map(|x| x.unwrap()) + .inspect(move |_| { + count.fetch_add(1, Ordering::Relaxed); + }) + .collect::>() + .await }) - .map_err(|e| panic!("{:?}", e)) - .collect(); - - let handle = futures01::sync::oneshot::spawn(events, &runtime.executor()); - Receiver { - handle, - count, - trigger, - _runtime: runtime, - } -} - -pub struct CountReceiver { - handle: futures01::sync::oneshot::SpawnHandle, - trigger: Trigger, - _runtime: Runtime, -} - -impl CountReceiver { - pub fn cancel(self) { - self.trigger.cancel(); - } - - pub fn wait(self) -> usize { - self.handle.wait().unwrap() } } -pub fn count_receive(addr: &SocketAddr) -> CountReceiver { - let runtime = Runtime::new().unwrap(); - - let listener = TcpListener::bind(addr).unwrap(); - - let (trigger, tripwire) = Tripwire::new(); - - let count = listener - .incoming() - .take_until(tripwire) - .map(|socket| FramedRead::new(socket, LinesCodec01::new())) - .flatten() - .map_err(|e| panic!("{:?}", e)) - .fold(0, |n, _| future::ok(n + 1)); - - let handle = futures01::sync::oneshot::spawn(count, &runtime.executor()); - CountReceiver { - handle, - trigger, - _runtime: runtime, +impl CountReceiver { + pub fn receive_events(stream: S) -> CountReceiver + where + S: Stream + Send + 'static, + ::Error: std::fmt::Debug, + { + CountReceiver::new(|count, tripwire| async move { + stream + .compat() + .take_until(tripwire) + .map(|x| x.unwrap()) + .inspect(move |_| { + count.fetch_add(1, Ordering::Relaxed); + }) + .collect::>() + .await + }) } } diff --git a/tests/buffering.rs b/tests/buffering.rs index 8eb4a779eee7f..b7a095d1aa94d 100644 --- a/tests/buffering.rs +++ b/tests/buffering.rs @@ -1,22 +1,20 @@ -#![allow(clippy::match_bool)] #![cfg(feature = "leveldb")] +use futures::compat::Future01CompatExt; use futures01::{Future, Sink}; use prost::Message; use tempfile::tempdir; use tracing::trace; use vector::event; -use vector::test_util::{self, wait_for_atomic_usize}; +use vector::test_util::{ + random_events_with_stream, runtime, shutdown_on_idle, trace_init, wait_for_atomic_usize, + CountReceiver, +}; use vector::topology::{self, config}; use vector::{buffers::BufferConfig, runtime}; mod support; -fn terminate_gracefully(mut rt: runtime::Runtime, topology: topology::RunningTopology) { - rt.block_on(topology.stop()).unwrap(); - test_util::shutdown_on_idle(rt); -} - fn terminate_abruptly(rt: runtime::Runtime, topology: topology::RunningTopology) { rt.shutdown_now().wait().unwrap(); drop(topology); @@ -24,7 +22,7 @@ fn terminate_abruptly(rt: runtime::Runtime, topology: topology::RunningTopology) #[test] fn test_buffering() { - test_util::trace_init(); + trace_init(); let data_dir = tempdir().unwrap(); let data_dir = data_dir.path().to_path_buf(); @@ -57,12 +55,11 @@ fn test_buffering() { config }; - let mut rt = test_util::runtime(); + let mut rt = runtime(); let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - let (input_events, input_events_stream) = - test_util::random_events_with_stream(line_length, num_events); + let (input_events, input_events_stream) = random_events_with_stream(line_length, num_events); let send = in_tx .sink_map_err(|err| panic!(err)) .send_all(input_events_stream); @@ -100,31 +97,35 @@ fn test_buffering() { config }; - let mut rt = test_util::runtime(); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + rt.block_on_std(async move { + let (topology, _crash) = topology::start(config, false).await.unwrap(); - let (input_events2, input_events_stream) = - test_util::random_events_with_stream(line_length, num_events); + let (input_events2, input_events_stream) = + random_events_with_stream(line_length, num_events); - let send = in_tx - .sink_map_err(|err| panic!(err)) - .send_all(input_events_stream); - let _ = rt.block_on(send).unwrap(); + let _ = in_tx + .sink_map_err(|err| panic!(err)) + .send_all(input_events_stream) + .compat() + .await + .unwrap(); - let output_events = test_util::receive_events(out_rx); + let output_events = CountReceiver::receive_events(out_rx); - terminate_gracefully(rt, topology); + topology.stop().compat().await.unwrap(); - let output_events = output_events.wait(); - assert_eq!(expected_events_count, output_events.len()); - assert_eq!(input_events, &output_events[..num_events]); - assert_eq!(input_events2, &output_events[num_events..]); + let output_events = output_events.wait().await; + assert_eq!(expected_events_count, output_events.len()); + assert_eq!(input_events, &output_events[..num_events]); + assert_eq!(input_events2, &output_events[num_events..]); + }); + shutdown_on_idle(rt); } #[test] fn test_max_size() { - test_util::trace_init(); + trace_init(); let data_dir = tempdir().unwrap(); let data_dir = data_dir.path().to_path_buf(); @@ -132,8 +133,7 @@ fn test_max_size() { let num_events: usize = 1000; let line_length = 1000; - let (input_events, input_events_stream) = - test_util::random_events_with_stream(line_length, num_events); + let (input_events, input_events_stream) = random_events_with_stream(line_length, num_events); let max_size = input_events .clone() @@ -159,7 +159,7 @@ fn test_max_size() { config }; - let mut rt = test_util::runtime(); + let mut rt = runtime(); let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); @@ -201,22 +201,24 @@ fn test_max_size() { config }; - let mut rt = test_util::runtime(); + let mut rt = runtime(); + rt.block_on_std(async move { + let (topology, _crash) = topology::start(config, false).await.unwrap(); - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - - let output_events = test_util::receive_events(out_rx); + let output_events = CountReceiver::receive_events(out_rx); - terminate_gracefully(rt, topology); + topology.stop().compat().await.unwrap(); - let output_events = output_events.wait(); - assert_eq!(num_events / 2, output_events.len()); - assert_eq!(&input_events[..num_events / 2], &output_events[..]); + let output_events = output_events.wait().await; + assert_eq!(num_events / 2, output_events.len()); + assert_eq!(&input_events[..num_events / 2], &output_events[..]); + }); + shutdown_on_idle(rt); } #[test] fn test_reclaim_disk_space() { - test_util::trace_init(); + trace_init(); let data_dir = tempdir().unwrap(); let data_dir = data_dir.path().to_path_buf(); @@ -241,12 +243,11 @@ fn test_reclaim_disk_space() { config }; - let mut rt = test_util::runtime(); + let mut rt = runtime(); let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - let (input_events, input_events_stream) = - test_util::random_events_with_stream(line_length, num_events); + let (input_events, input_events_stream) = random_events_with_stream(line_length, num_events); let send = in_tx .sink_map_err(|err| panic!(err)) .send_all(input_events_stream); @@ -286,32 +287,36 @@ fn test_reclaim_disk_space() { config }; - let mut rt = test_util::runtime(); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + rt.block_on_std(async move { + let (topology, _crash) = topology::start(config, false).await.unwrap(); - let (input_events2, input_events_stream) = - test_util::random_events_with_stream(line_length, num_events); + let (input_events2, input_events_stream) = + random_events_with_stream(line_length, num_events); - let send = in_tx - .sink_map_err(|err| panic!(err)) - .send_all(input_events_stream); - let _ = rt.block_on(send).unwrap(); + let _ = in_tx + .sink_map_err(|err| panic!(err)) + .send_all(input_events_stream) + .compat() + .await + .unwrap(); - let output_events = test_util::receive_events(out_rx); + let output_events = CountReceiver::receive_events(out_rx); - terminate_gracefully(rt, topology); + topology.stop().compat().await.unwrap(); - let output_events = output_events.wait(); - assert_eq!(num_events * 2, output_events.len()); - assert_eq!(input_events, &output_events[..num_events]); - assert_eq!(input_events2, &output_events[num_events..]); + let output_events = output_events.wait().await; + assert_eq!(num_events * 2, output_events.len()); + assert_eq!(input_events, &output_events[..num_events]); + assert_eq!(input_events2, &output_events[num_events..]); - let after_disk_size: u64 = compute_disk_size(&data_dir); + let after_disk_size: u64 = compute_disk_size(&data_dir); - // Ensure that the disk space after is less than half of the size that it - // was before we reclaimed the space. - assert!(after_disk_size < before_disk_size / 2); + // Ensure that the disk space after is less than half of the size that it + // was before we reclaimed the space. + assert!(after_disk_size < before_disk_size / 2); + }); + shutdown_on_idle(rt); } fn compute_disk_size(dir: impl AsRef) -> u64 { diff --git a/tests/crash.rs b/tests/crash.rs index f1d0f4aacd973..a467a47363cb3 100644 --- a/tests/crash.rs +++ b/tests/crash.rs @@ -1,12 +1,12 @@ #![cfg(all(feature = "sources-socket", feature = "sinks-socket"))] +use futures::compat::Future01CompatExt; use futures01::{future, sync::mpsc, Async, AsyncSink, Sink, Stream}; use serde::{Deserialize, Serialize}; use vector::{ shutdown::ShutdownSignal, test_util::{ - block_on, next_addr, random_lines, receive, runtime, send_lines, shutdown_on_idle, - wait_for_tcp, + next_addr, random_lines, runtime, send_lines, shutdown_on_idle, wait_for_tcp, CountReceiver, }, topology::{ self, @@ -72,28 +72,29 @@ fn test_sink_panic() { config.add_sink("panic", &["in"], PanicSink); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - std::panic::set_hook(Box::new(|_| {})); // Suppress panic print on background thread - let (topology, crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - std::thread::sleep(std::time::Duration::from_millis(100)); - - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - runtime().block_on_std(send).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); - - let _ = std::panic::take_hook(); - assert!(crash.wait().next().is_some()); - block_on(topology.stop()).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + std::panic::set_hook(Box::new(|_| {})); // Suppress panic print on background thread + let (topology, crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let _ = std::panic::take_hook(); + assert!(crash.wait().next().is_some()); + topology.stop().compat().await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let output_lines = output_lines.wait().await; + assert_eq!(num_lines, output_lines.len()); + assert_eq!(input_lines, output_lines); + }); shutdown_on_idle(rt); - - let output_lines = output_lines.wait(); - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } #[derive(Debug, Serialize, Deserialize)] @@ -153,28 +154,27 @@ fn test_sink_error() { config.add_sink("error", &["in"], ErrorSink); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - let (topology, crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - std::thread::sleep(std::time::Duration::from_millis(100)); - - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - let mut rt2 = runtime(); - rt2.block_on_std(send).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); - - assert!(crash.wait().next().is_some()); - block_on(topology.stop()).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + let (topology, crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + assert!(crash.wait().next().is_some()); + topology.stop().compat().await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let output_lines = output_lines.wait().await; + assert_eq!(num_lines, output_lines.len()); + assert_eq!(input_lines, output_lines); + }); shutdown_on_idle(rt); - - let output_lines = output_lines.wait(); - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } #[derive(Deserialize, Serialize, Debug)] @@ -221,28 +221,27 @@ fn test_source_error() { ); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - let (topology, crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - std::thread::sleep(std::time::Duration::from_millis(100)); - - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - let mut rt2 = runtime(); - rt2.block_on_std(send).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); - - assert!(crash.wait().next().is_some()); - block_on(topology.stop()).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + let (topology, crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + assert!(crash.wait().next().is_some()); + topology.stop().compat().await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let output_lines = output_lines.wait().await; + assert_eq!(num_lines, output_lines.len()); + assert_eq!(input_lines, output_lines); + }); shutdown_on_idle(rt); - - let output_lines = output_lines.wait(); - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } #[derive(Deserialize, Serialize, Debug)] @@ -291,28 +290,27 @@ fn test_source_panic() { ); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - std::panic::set_hook(Box::new(|_| {})); // Suppress panic print on background thread - let (topology, crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - std::thread::sleep(std::time::Duration::from_millis(100)); - - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - let mut rt2 = runtime(); - rt2.block_on_std(send).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); - let _ = std::panic::take_hook(); - - assert!(crash.wait().next().is_some()); - block_on(topology.stop()).unwrap(); - std::thread::sleep(std::time::Duration::from_millis(100)); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + std::panic::set_hook(Box::new(|_| {})); // Suppress panic print on background thread + let (topology, crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + let _ = std::panic::take_hook(); + + assert!(crash.wait().next().is_some()); + topology.stop().compat().await.unwrap(); + std::thread::sleep(std::time::Duration::from_millis(100)); + + let output_lines = output_lines.wait().await; + assert_eq!(num_lines, output_lines.len()); + assert_eq!(input_lines, output_lines); + }); shutdown_on_idle(rt); - - let output_lines = output_lines.wait(); - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } diff --git a/tests/support/mod.rs b/tests/support/mod.rs index a194e3e6b1f13..170b157108ce2 100644 --- a/tests/support/mod.rs +++ b/tests/support/mod.rs @@ -281,9 +281,10 @@ where error!(message = "Ingesting an event failed at mock sink", ?error) }); let sink = StreamSink::new(sink, cx.acker()); - let healthcheck = match self.healthy { - true => future::ok(()), - false => future::err(HealthcheckError::Unhealthy.into()), + let healthcheck = if self.healthy { + future::ok(()) + } else { + future::err(HealthcheckError::Unhealthy.into()) }; Ok((Box::new(sink), Box::new(healthcheck))) } diff --git a/tests/syslog.rs b/tests/syslog.rs index df030bb89c79c..992fec4ba8916 100644 --- a/tests/syslog.rs +++ b/tests/syslog.rs @@ -1,5 +1,6 @@ #![cfg(all(feature = "sources-syslog", feature = "sinks-socket"))] +use futures::compat::Future01CompatExt; #[cfg(unix)] use futures01::{Future, Sink, Stream}; use rand::{thread_rng, Rng}; @@ -9,14 +10,14 @@ use sinks::socket::SocketSinkConfig; use sinks::util::{encoding::EncodingConfig, Encoding}; use std::fmt; use std::{collections::HashMap, str::FromStr}; -use tokio01::codec::BytesCodec; #[cfg(unix)] use tokio01::codec::{FramedWrite, LinesCodec}; #[cfg(unix)] use tokio_uds::UnixStream; +use tokio_util::codec::BytesCodec; use vector::test_util::{ - block_on, next_addr, random_maps, random_string, receive, runtime, send_encodable, send_lines, - shutdown_on_idle, trace_init, wait_for_tcp, + next_addr, random_maps, random_string, runtime, send_encodable, send_lines, shutdown_on_idle, + trace_init, wait_for_tcp, CountReceiver, }; use vector::topology::{self, config}; use vector::{ @@ -42,40 +43,39 @@ fn test_tcp_syslog() { config.add_sink("out", &["in"], tcp_json_sink(out_addr.to_string())); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - - let input_messages: Vec = (0..num_messages) - .map(|i| SyslogMessageRFC5424::random(i, 30, 4, 3, 3)) - .collect(); - - let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); - - runtime() - .block_on_std(send_lines(in_addr, input_lines.into_iter())) - .unwrap(); - - // Shut down server - block_on(topology.stop()).unwrap(); - + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + + let input_messages: Vec = (0..num_messages) + .map(|i| SyslogMessageRFC5424::random(i, 30, 4, 3, 3)) + .collect(); + + let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); + + send_lines(in_addr, input_lines).await.unwrap(); + + // Shut down server + topology.stop().compat().await.unwrap(); + + let output_lines = output_lines.wait().await; + assert_eq!(output_lines.len(), num_messages); + + let output_messages: Vec = output_lines + .iter() + .map(|s| { + let mut value = Value::from_str(s).unwrap(); + value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. + value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. + serde_json::from_value(value).unwrap() + }) + .collect(); + assert_eq!(output_messages, input_messages); + }); shutdown_on_idle(rt); - let output_lines = output_lines.wait(); - assert_eq!(output_lines.len(), num_messages); - - let output_messages: Vec = output_lines - .iter() - .map(|s| { - let mut value = Value::from_str(s).unwrap(); - value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. - value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. - serde_json::from_value(value).unwrap() - }) - .collect(); - assert_eq!(output_messages, input_messages); } #[cfg(unix)] @@ -96,60 +96,61 @@ fn test_unix_stream_syslog() { config.add_sink("out", &["in"], tcp_json_sink(out_addr.to_string())); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - while std::os::unix::net::UnixStream::connect(&in_path).is_err() {} - - let input_messages: Vec = (0..num_messages) - .map(|i| SyslogMessageRFC5424::random(i, 30, 4, 3, 3)) - .collect(); - - let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); - let input_stream = futures01::stream::iter_ok::<_, ()>(input_lines.into_iter()); - - UnixStream::connect(&in_path) - .map_err(|e| panic!("{:}", e)) - .and_then(|socket| { - let out = - FramedWrite::new(socket, LinesCodec::new()).sink_map_err(|e| panic!("{:?}", e)); - - input_stream - .forward(out) - .map(|(_source, sink)| sink) - .and_then(|sink| { - let socket = sink.into_inner().into_inner(); - // In tokio 0.1 `AsyncWrite::shutdown` for `TcpStream` is a noop. - // See https://docs.rs/tokio-tcp/0.1.4/src/tokio_tcp/stream.rs.html#917 - // Use `TcpStream::shutdown` instead - it actually does something. - socket - .shutdown(std::net::Shutdown::Both) - .map(|_| ()) - .map_err(|e| panic!("{:}", e)) - }) - }) - .wait() - .unwrap(); - - // Shut down server - block_on(topology.stop()).unwrap(); - + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + while std::os::unix::net::UnixStream::connect(&in_path).is_err() {} + + let input_messages: Vec = (0..num_messages) + .map(|i| SyslogMessageRFC5424::random(i, 30, 4, 3, 3)) + .collect(); + + let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); + let input_stream = futures01::stream::iter_ok::<_, ()>(input_lines.into_iter()); + + UnixStream::connect(&in_path) + .map_err(|e| panic!("{:}", e)) + .and_then(|socket| { + let out = + FramedWrite::new(socket, LinesCodec::new()).sink_map_err(|e| panic!("{:?}", e)); + + input_stream + .forward(out) + .map(|(_source, sink)| sink) + .and_then(|sink| { + let socket = sink.into_inner().into_inner(); + // In tokio 0.1 `AsyncWrite::shutdown` for `TcpStream` is a noop. + // See https://docs.rs/tokio-tcp/0.1.4/src/tokio_tcp/stream.rs.html#917 + // Use `TcpStream::shutdown` instead - it actually does something. + socket + .shutdown(std::net::Shutdown::Both) + .map(|_| ()) + .map_err(|e| panic!("{:}", e)) + }) + }) + .wait() + .unwrap(); + + // Shut down server + topology.stop().compat().await.unwrap(); + + let output_lines = output_lines.wait().await; + assert_eq!(output_lines.len(), num_messages); + + let output_messages: Vec = output_lines + .iter() + .map(|s| { + let mut value = Value::from_str(s).unwrap(); + value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. + value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. + serde_json::from_value(value).unwrap() + }) + .collect(); + assert_eq!(output_messages, input_messages); + }); shutdown_on_idle(rt); - let output_lines = output_lines.wait(); - assert_eq!(output_lines.len(), num_messages); - - let output_messages: Vec = output_lines - .iter() - .map(|s| { - let mut value = Value::from_str(s).unwrap(); - value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. - value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. - serde_json::from_value(value).unwrap() - }) - .collect(); - assert_eq!(output_messages, input_messages); } #[test] @@ -171,55 +172,51 @@ fn test_octet_counting_syslog() { config.add_sink("out", &["in"], tcp_json_sink(out_addr.to_string())); let mut rt = runtime(); - - let output_lines = receive(&out_addr); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - - let input_messages: Vec = (0..num_messages) - .map(|i| { - let mut msg = SyslogMessageRFC5424::random(i, 30, 4, 3, 3); - msg.message.push('\n'); - msg.message.push_str(&random_string(30)); - msg - }) - .collect(); - - let input_lines: Vec = input_messages - .iter() - .map(|msg| { - let s = msg.to_string(); - format!("{} {}", s.len(), s) - }) - .collect(); - - runtime() - .block_on_std(send_encodable( - in_addr, - BytesCodec::new(), - input_lines.into_iter().map(Into::into), - )) - .unwrap(); - - // Shut down server - block_on(topology.stop()).unwrap(); - + let output_lines = CountReceiver::receive_lines(out_addr); + + rt.block_on_std(async move { + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + + let input_messages: Vec = (0..num_messages) + .map(|i| { + let mut msg = SyslogMessageRFC5424::random(i, 30, 4, 3, 3); + msg.message.push('\n'); + msg.message.push_str(&random_string(30)); + msg + }) + .collect(); + + let codec = BytesCodec::new(); + let input_lines: Vec = input_messages + .iter() + .map(|msg| { + let s = msg.to_string(); + format!("{} {}", s.len(), s).into() + }) + .collect(); + + send_encodable(in_addr, codec, input_lines).await.unwrap(); + + // Shut down server + topology.stop().compat().await.unwrap(); + + let output_lines = output_lines.wait().await; + assert_eq!(output_lines.len(), num_messages); + + let output_messages: Vec = output_lines + .iter() + .map(|s| { + let mut value = Value::from_str(s).unwrap(); + value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. + value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. + serde_json::from_value(value).unwrap() + }) + .collect(); + assert_eq!(output_messages, input_messages); + }); shutdown_on_idle(rt); - let output_lines = output_lines.wait(); - assert_eq!(output_lines.len(), num_messages); - - let output_messages: Vec = output_lines - .iter() - .map(|s| { - let mut value = Value::from_str(s).unwrap(); - value.as_object_mut().unwrap().remove("hostname"); // Vector adds this field which will cause a parse error. - value.as_object_mut().unwrap().remove("source_ip"); // Vector adds this field which will cause a parse error. - serde_json::from_value(value).unwrap() - }) - .collect(); - assert_eq!(output_messages, input_messages); } #[derive(Deserialize, PartialEq, Clone, Debug)] diff --git a/tests/tcp.rs b/tests/tcp.rs index 89991e4d8fd06..2992f579746e1 100644 --- a/tests/tcp.rs +++ b/tests/tcp.rs @@ -1,16 +1,18 @@ #![cfg(all( - feature = "sources-socket", + feature = "sinks-socket", feature = "transforms-sampler", - feature = "sinks-socket" + feature = "sources-socket", ))] use approx::assert_relative_eq; +use futures::compat::Future01CompatExt; use futures01::{Future, Stream}; use stream_cancel::{StreamExt, Tripwire}; use tokio01::codec::{FramedRead, LinesCodec}; use tokio01::net::TcpListener; use vector::test_util::{ - block_on, next_addr, random_lines, receive, runtime, send_lines, shutdown_on_idle, wait_for_tcp, + block_on, next_addr, random_lines, runtime, send_lines, shutdown_on_idle, wait_for_tcp, + CountReceiver, }; use vector::topology::{self, config}; use vector::{sinks, sources, transforms}; @@ -34,24 +36,24 @@ fn pipe() { ); let mut rt = runtime(); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); - let output_lines = receive(&out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on_std(send).unwrap(); + // Shut down server + topology.stop().compat().await.unwrap(); - // Shut down server - block_on(topology.stop()).unwrap(); + let output_lines = output_lines.wait().await; + assert_eq!(num_lines, output_lines.len()); + assert_eq!(input_lines, output_lines); + }); shutdown_on_idle(rt); - - let output_lines = output_lines.wait(); - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } #[test] @@ -82,33 +84,33 @@ fn sample() { ); let mut rt = runtime(); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); - let output_lines = receive(&out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on_std(send).unwrap(); + // Shut down server + topology.stop().compat().await.unwrap(); - // Shut down server - block_on(topology.stop()).unwrap(); + let output_lines = output_lines.wait().await; + let num_output_lines = output_lines.len(); + let output_lines_ratio = num_output_lines as f32 / num_lines as f32; + assert_relative_eq!(output_lines_ratio, 0.1, epsilon = 0.01); + + let mut input_lines = input_lines.into_iter(); + // Assert that all of the output lines were present in the input and in the same order + for output_line in output_lines { + let next_line = input_lines.by_ref().find(|l| l == &output_line); + assert_eq!(Some(output_line), next_line); + } + }); shutdown_on_idle(rt); - let output_lines = output_lines.wait(); - let num_output_lines = output_lines.len(); - - let output_lines_ratio = num_output_lines as f32 / num_lines as f32; - assert_relative_eq!(output_lines_ratio, 0.1, epsilon = 0.01); - - let mut input_lines = input_lines.into_iter(); - // Assert that all of the output lines were present in the input and in the same order - for output_line in output_lines { - let next_line = input_lines.by_ref().find(|l| l == &output_line); - assert_eq!(Some(output_line), next_line); - } } #[test] @@ -136,28 +138,28 @@ fn fork() { ); let mut rt = runtime(); + rt.block_on_std(async move { + let output_lines1 = CountReceiver::receive_lines(out_addr1); + let output_lines2 = CountReceiver::receive_lines(out_addr2); - let output_lines1 = receive(&out_addr1); - let output_lines2 = receive(&out_addr2); + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on_std(send).unwrap(); - - // Shut down server - block_on(topology.stop()).unwrap(); + // Shut down server + topology.stop().compat().await.unwrap(); + let output_lines1 = output_lines1.wait().await; + let output_lines2 = output_lines2.wait().await; + assert_eq!(num_lines, output_lines1.len()); + assert_eq!(num_lines, output_lines2.len()); + assert_eq!(input_lines, output_lines1); + assert_eq!(input_lines, output_lines2); + }); shutdown_on_idle(rt); - let output_lines1 = output_lines1.wait(); - let output_lines2 = output_lines2.wait(); - assert_eq!(num_lines, output_lines1.len()); - assert_eq!(num_lines, output_lines2.len()); - assert_eq!(input_lines, output_lines1); - assert_eq!(input_lines, output_lines2); } #[test] @@ -192,50 +194,47 @@ fn merge_and_fork() { ); let mut rt = runtime(); - - let output_lines1 = receive(&out_addr1); - let output_lines2 = receive(&out_addr2); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr1); - wait_for_tcp(in_addr2); - - let input_lines1 = random_lines(100).take(num_lines).collect::>(); - let input_lines2 = random_lines(100).take(num_lines).collect::>(); - let send1 = send_lines(in_addr1, input_lines1.clone().into_iter()); - let send2 = send_lines(in_addr2, input_lines2.clone().into_iter()); rt.block_on_std(async move { - send1.await.unwrap(); - send2.await.unwrap(); + let output_lines1 = CountReceiver::receive_lines(out_addr1); + let output_lines2 = CountReceiver::receive_lines(out_addr2); + + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr1); + wait_for_tcp(in_addr2); + + let input_lines1 = random_lines(100).take(num_lines).collect::>(); + let input_lines2 = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr1, input_lines1.clone()).await.unwrap(); + send_lines(in_addr2, input_lines2.clone()).await.unwrap(); + + // Shut down server + topology.stop().compat().await.unwrap(); + + let output_lines1 = output_lines1.wait().await; + let output_lines2 = output_lines2.wait().await; + + assert_eq!(num_lines, output_lines2.len()); + + assert_eq!(input_lines2, output_lines2); + + assert_eq!(num_lines * 2, output_lines1.len()); + // Assert that all of the output lines were present in the input and in the same order + let mut input_lines1 = input_lines1.into_iter().peekable(); + let mut input_lines2 = input_lines2.into_iter().peekable(); + for output_line in &output_lines1 { + if Some(output_line) == input_lines1.peek() { + input_lines1.next(); + } else if Some(output_line) == input_lines2.peek() { + input_lines2.next(); + } else { + panic!("Got line in output that wasn't in input"); + } + } + assert_eq!(input_lines1.next(), None); + assert_eq!(input_lines2.next(), None); }); - - // Shut down server - block_on(topology.stop()).unwrap(); - shutdown_on_idle(rt); - let output_lines1 = output_lines1.wait(); - let output_lines2 = output_lines2.wait(); - - assert_eq!(num_lines, output_lines2.len()); - - assert_eq!(input_lines2, output_lines2); - - assert_eq!(num_lines * 2, output_lines1.len()); - // Assert that all of the output lines were present in the input and in the same order - let mut input_lines1 = input_lines1.into_iter().peekable(); - let mut input_lines2 = input_lines2.into_iter().peekable(); - for output_line in &output_lines1 { - if Some(output_line) == input_lines1.peek() { - input_lines1.next(); - } else if Some(output_line) == input_lines2.peek() { - input_lines2.next(); - } else { - panic!("Got line in output that wasn't in input"); - } - } - assert_eq!(input_lines1.next(), None); - assert_eq!(input_lines2.next(), None); } #[test] diff --git a/tests/topology.rs b/tests/topology.rs index 4d7f9db33d4a4..31a52c65d858b 100644 --- a/tests/topology.rs +++ b/tests/topology.rs @@ -1,7 +1,3 @@ -#![allow(clippy::bool_comparison)] -#![allow(clippy::match_bool)] -#![allow(clippy::redundant_clone)] - mod support; use crate::support::{sink, sink_failing_healthcheck, source, transform, MockSourceConfig}; @@ -202,7 +198,7 @@ fn topology_transform_chain() { let event = Event::from("this"); - in1.send(event.clone()).wait().unwrap(); + in1.send(event).wait().unwrap(); rt.block_on(topology.stop()).unwrap(); @@ -476,12 +472,13 @@ fn topology_swap_transform_is_atomic() { let send_total = send_counter.clone(); let recv_total = recv_counter.clone(); - let events = move || match running.load(Ordering::Acquire) { - true => { + let events = move || { + if running.load(Ordering::Acquire) { send_counter.fetch_add(1, Ordering::Release); Some(Event::from("this")) + } else { + None } - false => None, }; let input = iter_ok::<_, ()>(iter::from_fn(events)); let input = input @@ -595,12 +592,9 @@ fn topology_healthcheck_run_for_changes_on_reload() { config.add_source("in1", src); config.add_sink("out2", &["in1"], sink_failing_healthcheck(10).1); - assert!( - topology - .reload_config_and_respawn(config, true) - .await - .unwrap() - == false - ); + assert!(!topology + .reload_config_and_respawn(config, true) + .await + .unwrap()); }); } From 124d25ef6aa533e8ce69d9558fc79b0834346e0f Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 13:21:12 +0300 Subject: [PATCH 04/29] benches/files: update codec Signed-off-by: Kirill Fomichev --- Cargo.lock | 12 ------------ Cargo.toml | 2 +- benches/files.rs | 48 +++++++++++++++++++++++++----------------------- 3 files changed, 26 insertions(+), 36 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b8f04d72d2449..29a935d829d05 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4587,7 +4587,6 @@ dependencies = [ "tokio-codec", "tokio-current-thread", "tokio-executor", - "tokio-fs", "tokio-io", "tokio-reactor", "tokio-sync", @@ -4671,17 +4670,6 @@ dependencies = [ "futures 0.1.29", ] -[[package]] -name = "tokio-fs" -version = "0.1.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fe6dc22b08d6993916647d108a1a7d15b9cd29c4f4496c62b92c45b5041b7af" -dependencies = [ - "futures 0.1.29", - "tokio-io", - "tokio-threadpool", -] - [[package]] name = "tokio-io" version = "0.1.12" diff --git a/Cargo.toml b/Cargo.toml index a20511a7d4e1a..9fb714019ba47 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -44,7 +44,7 @@ tracing-limit = { path = "lib/tracing-limit" } # Tokio / Futures futures01 = { package = "futures", version = "0.1.25" } futures = { version = "0.3", default-features = false, features = ["compat", "io-compat"] } -tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "uds", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync", "fs"], default-features = false } +tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "uds", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync"], default-features = false } tokio = { version = "0.2.13", features = ["blocking", "fs", "sync", "macros", "test-util", "rt-core", "io-std"] } tokio-openssl03 = { package = "tokio-openssl", version = "0.3.0" } tokio-openssl = "0.4.0" diff --git a/benches/files.rs b/benches/files.rs index 9c74dbb053ce8..e37cfbe71c2e2 100644 --- a/benches/files.rs +++ b/benches/files.rs @@ -1,14 +1,15 @@ -use bytes::Bytes; +use bytes05::Bytes; use criterion::{criterion_group, Benchmark, Criterion, Throughput}; -use futures01::{sink::Sink, stream::Stream, Future}; +use futures::{compat::Future01CompatExt, stream, SinkExt, StreamExt}; +use futures01::Future; use std::convert::TryInto; use std::path::PathBuf; use tempfile::tempdir; -use tokio01::codec::{BytesCodec, FramedWrite}; -use tokio01::fs::OpenOptions; -use vector::test_util::random_lines; +use tokio::fs::OpenOptions; +use tokio_util::codec::{BytesCodec, FramedWrite}; use vector::{ - runtime, sinks, sources, + sinks, sources, + test_util::{random_lines, runtime}, topology::{self, config}, }; @@ -55,30 +56,31 @@ fn benchmark_files_without_partitions(c: &mut Criterion) { }, ); - let mut rt = runtime::Runtime::new().unwrap(); - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); + let mut rt = runtime(); + let (topology, input) = rt.block_on_std(async move { + let (topology, _crash) = topology::start(config, false).await.unwrap(); - let mut options = OpenOptions::new(); - options.create(true).write(true); + let mut options = OpenOptions::new(); + options.create(true).write(true); - let input = rt.block_on(options.open(input)).unwrap(); - let input = - FramedWrite::new(input, BytesCodec::new()).sink_map_err(|e| panic!("{:?}", e)); + let input = options.open(input).await.unwrap(); + let input = FramedWrite::new(input, BytesCodec::new()) + .sink_map_err(|e| panic!("{:?}", e)); + (topology, input) + }); (rt, topology, input) }, |(mut rt, topology, input)| { - let lines = random_lines(line_size).take(num_lines).map(|mut line| { - line.push('\n'); - Bytes::from(line) + rt.block_on_std(async move { + let lines = random_lines(line_size).take(num_lines).map(|mut line| { + line.push('\n'); + Ok(Bytes::from(line)) + }); + let _ = stream::iter(lines).forward(input).await.unwrap(); + + topology.stop().compat().await.unwrap(); }); - - let lines = futures01::stream::iter_ok::<_, ()>(lines); - - let pump = lines.forward(input); - let (_, _) = rt.block_on(pump).unwrap(); - - rt.block_on(topology.stop()).unwrap(); rt.shutdown_now().wait().unwrap(); }, ) From 1c6d85673952da7ac395b1488aa99a4d34b15655 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 13:55:50 +0300 Subject: [PATCH 05/29] tests/syslog: update codec Signed-off-by: Kirill Fomichev --- Cargo.toml | 2 +- tests/syslog.rs | 65 ++++++++++++++++++++----------------------------- 2 files changed, 27 insertions(+), 40 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 9fb714019ba47..b2cc187c21240 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -193,7 +193,7 @@ tower-test01 = { package = "tower-test", version = "0.1" } serde_yaml = "0.8.9" dirs = "2.0.2" tokio-test = "0.2" -tokio = { version = "0.2", features = ["test-util"] } +tokio = { version = "0.2", features = ["test-util", "uds"] } assert_cmd = "1.0" reqwest = { version = "0.10.6", features = ["json"] } diff --git a/tests/syslog.rs b/tests/syslog.rs index 992fec4ba8916..902b1090c7a9e 100644 --- a/tests/syslog.rs +++ b/tests/syslog.rs @@ -1,28 +1,31 @@ #![cfg(all(feature = "sources-syslog", feature = "sinks-socket"))] +use bytes05::Bytes; use futures::compat::Future01CompatExt; #[cfg(unix)] -use futures01::{Future, Sink, Stream}; +use futures::{stream, SinkExt, StreamExt}; use rand::{thread_rng, Rng}; use serde::Deserialize; use serde_json::Value; -use sinks::socket::SocketSinkConfig; -use sinks::util::{encoding::EncodingConfig, Encoding}; +use sinks::{ + socket::SocketSinkConfig, + util::{encoding::EncodingConfig, Encoding}, +}; use std::fmt; use std::{collections::HashMap, str::FromStr}; #[cfg(unix)] -use tokio01::codec::{FramedWrite, LinesCodec}; -#[cfg(unix)] -use tokio_uds::UnixStream; +use tokio::net::UnixStream; use tokio_util::codec::BytesCodec; -use vector::test_util::{ - next_addr, random_maps, random_string, runtime, send_encodable, send_lines, shutdown_on_idle, - trace_init, wait_for_tcp, CountReceiver, -}; -use vector::topology::{self, config}; +#[cfg(unix)] +use tokio_util::codec::{FramedWrite, LinesCodec}; use vector::{ sinks, sources::syslog::{Mode, SyslogConfig}, + test_util::{ + next_addr, random_maps, random_string, runtime, send_encodable, send_lines, + shutdown_on_idle, trace_init, wait_for_tcp, CountReceiver, + }, + topology::{self, config}, }; #[test] @@ -107,31 +110,15 @@ fn test_unix_stream_syslog() { .map(|i| SyslogMessageRFC5424::random(i, 30, 4, 3, 3)) .collect(); - let input_lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); - let input_stream = futures01::stream::iter_ok::<_, ()>(input_lines.into_iter()); - - UnixStream::connect(&in_path) - .map_err(|e| panic!("{:}", e)) - .and_then(|socket| { - let out = - FramedWrite::new(socket, LinesCodec::new()).sink_map_err(|e| panic!("{:?}", e)); - - input_stream - .forward(out) - .map(|(_source, sink)| sink) - .and_then(|sink| { - let socket = sink.into_inner().into_inner(); - // In tokio 0.1 `AsyncWrite::shutdown` for `TcpStream` is a noop. - // See https://docs.rs/tokio-tcp/0.1.4/src/tokio_tcp/stream.rs.html#917 - // Use `TcpStream::shutdown` instead - it actually does something. - socket - .shutdown(std::net::Shutdown::Both) - .map(|_| ()) - .map_err(|e| panic!("{:}", e)) - }) - }) - .wait() - .unwrap(); + let stream = UnixStream::connect(&in_path).await.unwrap(); + let mut sink = FramedWrite::new(stream, LinesCodec::new()); + + let lines: Vec = input_messages.iter().map(|msg| msg.to_string()).collect(); + let mut lines = stream::iter(lines).map(Ok); + sink.send_all(&mut lines).await.unwrap(); + + let stream = sink.get_mut(); + stream.shutdown(std::net::Shutdown::Both).unwrap(); // Shut down server topology.stop().compat().await.unwrap(); @@ -172,9 +159,9 @@ fn test_octet_counting_syslog() { config.add_sink("out", &["in"], tcp_json_sink(out_addr.to_string())); let mut rt = runtime(); - let output_lines = CountReceiver::receive_lines(out_addr); - rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + let (topology, _crash) = topology::start(config, false).await.unwrap(); // Wait for server to accept traffic wait_for_tcp(in_addr); @@ -189,7 +176,7 @@ fn test_octet_counting_syslog() { .collect(); let codec = BytesCodec::new(); - let input_lines: Vec = input_messages + let input_lines: Vec = input_messages .iter() .map(|msg| { let s = msg.to_string(); From 2aa9e4f48e0d15bd80c628a355610101eecc207c Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 14:15:41 +0300 Subject: [PATCH 06/29] tests/tcp: remove codec Signed-off-by: Kirill Fomichev --- tests/tcp.rs | 73 +++++++++++++++++++++------------------------------- 1 file changed, 29 insertions(+), 44 deletions(-) diff --git a/tests/tcp.rs b/tests/tcp.rs index 2992f579746e1..7d566497b1f55 100644 --- a/tests/tcp.rs +++ b/tests/tcp.rs @@ -6,16 +6,15 @@ use approx::assert_relative_eq; use futures::compat::Future01CompatExt; -use futures01::{Future, Stream}; -use stream_cancel::{StreamExt, Tripwire}; -use tokio01::codec::{FramedRead, LinesCodec}; -use tokio01::net::TcpListener; -use vector::test_util::{ - block_on, next_addr, random_lines, runtime, send_lines, shutdown_on_idle, wait_for_tcp, - CountReceiver, +use tokio::net::TcpListener; +use vector::{ + sinks, sources, + test_util::{ + next_addr, random_lines, runtime, send_lines, shutdown_on_idle, wait_for_tcp, CountReceiver, + }, + topology::{self, config}, + transforms, }; -use vector::topology::{self, config}; -use vector::{sinks, sources, transforms}; #[test] fn pipe() { @@ -256,46 +255,32 @@ fn reconnect() { ); let mut rt = runtime(); - let output_rt = runtime(); - - let (output_trigger, output_tripwire) = Tripwire::new(); - let output_listener = TcpListener::bind(&out_addr).unwrap(); - let output_lines = output_listener - .incoming() - .take_until(output_tripwire) - .map(|socket| FramedRead::new(socket, LinesCodec::new()).take(1)) - .flatten() - .map_err(|e| panic!("{:?}", e)) - .collect(); - let output_lines = futures01::sync::oneshot::spawn(output_lines, &output_rt.executor()); - - let (topology, _crash) = rt.block_on_std(topology::start(config, false)).unwrap(); - // Wait for server to accept traffic - wait_for_tcp(in_addr); - - let input_lines = random_lines(100).take(num_lines).collect::>(); - let send = send_lines(in_addr, input_lines.clone().into_iter()); - rt.block_on_std(send).unwrap(); - - // Shut down server and wait for it to fully flush - block_on(topology.stop()).unwrap(); - shutdown_on_idle(rt); + rt.block_on_std(async move { + let output_lines = CountReceiver::receive_lines(out_addr); + + let (topology, _crash) = topology::start(config, false).await.unwrap(); + // Wait for server to accept traffic + wait_for_tcp(in_addr); + + let input_lines = random_lines(100).take(num_lines).collect::>(); + send_lines(in_addr, input_lines.clone()).await.unwrap(); - drop(output_trigger); - shutdown_on_idle(output_rt); + // Shut down server and wait for it to fully flush + topology.stop().compat().await.unwrap(); - let output_lines = output_lines.wait().unwrap(); - assert!(num_lines >= 2); - assert!(output_lines.iter().all(|line| input_lines.contains(line))) + let output_lines = output_lines.wait().await; + assert!(num_lines >= 2); + assert!(output_lines.iter().all(|line| input_lines.contains(line))) + }); + shutdown_on_idle(rt); } -#[test] -fn healthcheck() { +#[tokio::test] +async fn healthcheck() { let addr = next_addr(); - let mut rt = runtime(); let resolver = vector::dns::Resolver; - let _listener = TcpListener::bind(&addr).unwrap(); + let _listener = TcpListener::bind(&addr).await.unwrap(); let healthcheck = vector::sinks::util::tcp::tcp_healthcheck( addr.ip().to_string(), @@ -304,7 +289,7 @@ fn healthcheck() { None.into(), ); - assert!(rt.block_on(healthcheck).is_ok()); + assert!(healthcheck.compat().await.is_ok()); let bad_addr = next_addr(); let bad_healthcheck = vector::sinks::util::tcp::tcp_healthcheck( @@ -314,5 +299,5 @@ fn healthcheck() { None.into(), ); - assert!(rt.block_on(bad_healthcheck).is_err()); + assert!(bad_healthcheck.compat().await.is_err()); } From 749dad727425cb13f1ca8ab8e2db74b8e4b68f09 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 18:36:51 +0300 Subject: [PATCH 07/29] sinks/util/unix: remove codec from test Signed-off-by: Kirill Fomichev --- src/sinks/util/unix.rs | 62 +++++++++++------------------- src/test_util.rs | 86 +++++++++++++++++++++++++++--------------- 2 files changed, 76 insertions(+), 72 deletions(-) diff --git a/src/sinks/util/unix.rs b/src/sinks/util/unix.rs index 94c52e102a24c..a83a2782e5707 100644 --- a/src/sinks/util/unix.rs +++ b/src/sinks/util/unix.rs @@ -197,18 +197,17 @@ impl Sink for UnixSink { #[cfg(test)] mod tests { use super::*; - use crate::test_util::{random_lines_with_stream, runtime, shutdown_on_idle}; - use futures01::{sync::mpsc, Sink, Stream}; - use stream_cancel::{StreamExt, Tripwire}; - use tokio01::codec::{FramedRead, LinesCodec}; - use tokio_uds::UnixListener; + use crate::test_util::{random_lines_with_stream, runtime, shutdown_on_idle, CountReceiver}; + use futures::compat::Future01CompatExt; + use futures01::Sink; + use tokio::net::UnixListener; fn temp_uds_path(name: &str) -> PathBuf { tempfile::tempdir().unwrap().into_path().join(name) } - #[test] - fn unix_sink_healthcheck() { + #[tokio::test] + async fn unix_sink_healthcheck() { let path = temp_uds_path("valid_uds"); let _listener = UnixListener::bind(&path).unwrap(); let healthcheck = unix_healthcheck(path); @@ -221,45 +220,26 @@ mod tests { #[test] fn basic_unix_sink() { - let num_lines = 1000; - let out_path = temp_uds_path("unix_test"); - - // Set up Sink - let config = UnixSinkConfig::new(out_path.clone(), Encoding::Text.into()); let mut rt = runtime(); - let cx = SinkContext::new_test(); - let (sink, _healthcheck) = config.build(cx).unwrap(); + rt.block_on_std(async move { + let num_lines = 1000; + let out_path = temp_uds_path("unix_test"); - // Set up server to receive events from the Sink. - let listener = UnixListener::bind(&out_path).expect("failed to bind to listener socket"); + // Set up server to receive events from the Sink. + let receiver = CountReceiver::receive_lines_unix(out_path.clone()); - let (tx, rx) = mpsc::channel(num_lines); - let (trigger, tripwire) = Tripwire::new(); + // Set up Sink + let config = UnixSinkConfig::new(out_path, Encoding::Text.into()); + let cx = SinkContext::new_test(); + let (sink, _healthcheck) = config.build(cx).unwrap(); - let receive_future = listener - .incoming() - .take_until(tripwire) - .map_err(|e| error!("failed to accept socket; error = {:?}", e)) - .for_each(move |socket| { - let tx = tx.clone(); - FramedRead::new(socket, LinesCodec::new()) - .map_err(|e| error!("error reading line: {:?}", e)) - .forward(tx.sink_map_err(|e| error!("error sending event: {:?}", e))) - .map(|_| ()) - }); - rt.spawn(receive_future); + // Send the test data + let (input_lines, events) = random_lines_with_stream(100, num_lines); + let _ = sink.send_all(events).compat().await.unwrap(); - // Send the test data - let (input_lines, events) = random_lines_with_stream(100, num_lines); - let pump = sink.send_all(events); - let _ = rt.block_on(pump).unwrap(); - drop(trigger); - - // Receive the data sent by the Sink to the receive_future - let output_lines = rx.wait().map(Result::unwrap).collect::>(); + // Receive the data sent by the Sink to the receiver + assert_eq!(input_lines, receiver.wait().await); + }); shutdown_on_idle(rt); - - assert_eq!(num_lines, output_lines.len()); - assert_eq!(input_lines, output_lines); } } diff --git a/src/test_util.rs b/src/test_util.rs index 22a23de38c186..f4e51f699e211 100644 --- a/src/test_util.rs +++ b/src/test_util.rs @@ -1,8 +1,10 @@ use crate::runtime::Runtime; use crate::{event::LogEvent, Event}; -use futures::{compat::Stream01CompatExt, stream, SinkExt, StreamExt, TryStreamExt}; -use futures01::{future, stream as stream01, sync::mpsc, try_ready, Async, Future, Poll, Stream}; +use futures::{compat::Stream01CompatExt, stream, SinkExt, Stream, StreamExt, TryStreamExt}; +use futures01::{ + future, stream as stream01, sync::mpsc, try_ready, Async, Future, Poll, Stream as Stream01, +}; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use rand::{distributions::Alphanumeric, thread_rng, Rng}; use std::{ @@ -12,11 +14,12 @@ use std::{ io::Read, iter, mem, net::{Shutdown, SocketAddr}, - path::Path, + path::{Path, PathBuf}, sync::atomic::{AtomicUsize, Ordering}, sync::Arc, }; use tokio::{ + io::{AsyncRead, AsyncWrite, Result as IoResult}, net::{TcpListener, TcpStream}, sync::oneshot, task::JoinHandle, @@ -101,13 +104,13 @@ pub async fn send_lines_tls( Ok(()) } -pub fn temp_file() -> std::path::PathBuf { +pub fn temp_file() -> PathBuf { let path = std::env::temp_dir(); let file_name = random_string(16); path.join(file_name + ".log") } -pub fn temp_dir() -> std::path::PathBuf { +pub fn temp_dir() -> PathBuf { let path = std::env::temp_dir(); let dir_name = random_string(16); path.join(dir_name) @@ -116,7 +119,7 @@ pub fn temp_dir() -> std::path::PathBuf { pub fn random_lines_with_stream( len: usize, count: usize, -) -> (Vec, impl Stream) { +) -> (Vec, impl Stream01) { let lines = (0..count).map(|_| random_string(len)).collect::>(); let stream = stream01::iter_ok(lines.clone().into_iter().map(Event::from)); (lines, stream) @@ -125,7 +128,7 @@ pub fn random_lines_with_stream( pub fn random_events_with_stream( len: usize, count: usize, -) -> (Vec, impl Stream) { +) -> (Vec, impl Stream01) { random_events_with_stream_generic(count, move || Event::from(random_string(len))) } @@ -134,7 +137,7 @@ pub fn random_nested_events_with_stream( breadth: usize, depth: usize, count: usize, -) -> (Vec, impl Stream) { +) -> (Vec, impl Stream01) { random_events_with_stream_generic(count, move || { let mut log = LogEvent::default(); @@ -246,14 +249,14 @@ pub fn shutdown_on_idle(runtime: Runtime) { #[derive(Debug)] pub struct CollectN where - S: Stream, + S: Stream01, { stream: Option, remaining: usize, items: Option>, } -impl CollectN { +impl CollectN { pub fn new(s: S, n: usize) -> Self { Self { stream: Some(s), @@ -265,7 +268,7 @@ impl CollectN { impl Future for CollectN where - S: Stream, + S: Stream01, { type Item = (S, Vec); type Error = S::Error; @@ -304,12 +307,12 @@ where #[derive(Debug)] pub struct CollectCurrent where - S: Stream, + S: Stream01, { stream: Option, } -impl CollectCurrent { +impl CollectCurrent { pub fn new(s: S) -> Self { Self { stream: Some(s) } } @@ -317,7 +320,7 @@ impl CollectCurrent { impl Future for CollectCurrent where - S: Stream, + S: Stream01, { type Item = (S, Vec); type Error = S::Error; @@ -378,29 +381,50 @@ impl CountReceiver { impl CountReceiver { pub fn receive_lines(addr: SocketAddr) -> CountReceiver { CountReceiver::new(|count, tripwire| async move { - TcpListener::bind(addr) - .await - .unwrap() - .incoming() - .take_until(tripwire) - .map_ok(|socket| FramedRead::new(socket, LinesCodec::new())) - .map(|x| x.unwrap()) - .flatten() - .map(|x| x.unwrap()) - .inspect(move |_| { - count.fetch_add(1, Ordering::Relaxed); - }) - .collect::>() - .await + let mut listener = TcpListener::bind(addr).await.unwrap(); + CountReceiver::receive_lines_stream(listener.incoming(), count, tripwire).await + }) + } + + #[cfg(unix)] + pub fn receive_lines_unix

(path: P) -> CountReceiver + where + P: AsRef + Send + 'static, + { + CountReceiver::new(|count, tripwire| async move { + let mut listener = tokio::net::UnixListener::bind(path).unwrap(); + CountReceiver::receive_lines_stream(listener.incoming(), count, tripwire).await }) } + + async fn receive_lines_stream( + stream: S, + count: Arc, + tripwire: oneshot::Receiver<()>, + ) -> Vec + where + S: Stream>, + T: AsyncWrite + AsyncRead, + { + stream + .take_until(tripwire) + .map_ok(|socket| FramedRead::new(socket, LinesCodec::new())) + .map(|x| x.unwrap()) + .flatten() + .map(|x| x.unwrap()) + .inspect(move |_| { + count.fetch_add(1, Ordering::Relaxed); + }) + .collect::>() + .await + } } impl CountReceiver { pub fn receive_events(stream: S) -> CountReceiver where - S: Stream + Send + 'static, - ::Error: std::fmt::Debug, + S: Stream01 + Send + 'static, + ::Error: std::fmt::Debug, { CountReceiver::new(|count, tripwire| async move { stream @@ -419,7 +443,7 @@ impl CountReceiver { fn random_events_with_stream_generic( count: usize, generator: F, -) -> (Vec, impl Stream) +) -> (Vec, impl Stream01) where F: Fn() -> Event, { From a1700c92ecd54cb5b681467376b5f0711fa37628 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 19:29:44 +0300 Subject: [PATCH 08/29] sinks/util/unix: remove codec Signed-off-by: Kirill Fomichev --- src/internal_events/unix.rs | 2 +- src/sinks/util/unix.rs | 109 +++++++++++++++++++----------------- 2 files changed, 58 insertions(+), 53 deletions(-) diff --git a/src/internal_events/unix.rs b/src/internal_events/unix.rs index 942159e11aadf..95ab0f8afefdc 100644 --- a/src/internal_events/unix.rs +++ b/src/internal_events/unix.rs @@ -20,7 +20,7 @@ impl InternalEvent for UnixSocketConnectionEstablished<'_> { #[derive(Debug)] pub struct UnixSocketConnectionFailure<'a> { - pub error: std::io::Error, + pub error: tokio::io::Error, pub path: &'a std::path::Path, } diff --git a/src/sinks/util/unix.rs b/src/sinks/util/unix.rs index a83a2782e5707..fe4bfb954e08b 100644 --- a/src/sinks/util/unix.rs +++ b/src/sinks/util/unix.rs @@ -7,19 +7,18 @@ use crate::{ sinks::{Healthcheck, RouterSink}, topology::config::SinkContext, }; -use bytes::Bytes; -use futures01::{ - future, stream::iter_ok, try_ready, Async, AsyncSink, Future, Poll, Sink, StartSend, -}; +use bytes05::Bytes; +use futures::{compat::CompatSink, FutureExt, TryFutureExt}; +use futures01::{stream::iter_ok, try_ready, Async, AsyncSink, Future, Poll, Sink, StartSend}; use serde::{Deserialize, Serialize}; use snafu::Snafu; -use std::io; -use std::path::PathBuf; -use std::time::{Duration, Instant}; -use tokio01::codec::{BytesCodec, FramedWrite}; -use tokio01::timer::Delay; +use std::{path::PathBuf, time::Duration}; +use tokio::{ + net::UnixStream, + time::{delay_for, Delay}, +}; use tokio_retry::strategy::ExponentialBackoff; -use tokio_uds::UnixStream; +use tokio_util::codec::{BytesCodec, FramedWrite}; use tracing::field; #[derive(Deserialize, Serialize, Debug, Clone)] @@ -41,27 +40,23 @@ impl UnixSinkConfig { let sink = Box::new(sink.with_flat_map(move |event| iter_ok(encode_event(event, &encoding)))); - let healthcheck = unix_healthcheck(self.path.clone()); + let healthcheck = healthcheck(self.path.clone()).boxed().compat(); - Ok((sink, healthcheck)) + Ok((sink, Box::new(healthcheck))) } } #[derive(Debug, Snafu)] enum HealthcheckError { #[snafu(display("Connect error: {}", source))] - ConnectError { source: std::io::Error }, + ConnectError { source: tokio::io::Error }, } -fn unix_healthcheck(path: PathBuf) -> Healthcheck { - // Lazy to avoid immediately connecting - let check = future::lazy(move || { - UnixStream::connect(&path) - .map(|_| ()) - .map_err(|source| HealthcheckError::ConnectError { source }.into()) - }); - - Box::new(check) +async fn healthcheck(path: PathBuf) -> crate::Result<()> { + match UnixStream::connect(&path).await { + Ok(_) => Ok(()), + Err(source) => Err(HealthcheckError::ConnectError { source }.into()), + } } pub struct UnixSink { @@ -72,9 +67,9 @@ pub struct UnixSink { enum UnixSinkState { Disconnected, - Creating(Box + Send + 'static>), - Open(FramedWrite), - Backoff(Delay), + Creating(Box + Send + 'static>), + Open(CompatSink, Bytes>), + Backoff(Box + Send>), } impl UnixSink { @@ -94,13 +89,15 @@ impl UnixSink { } fn next_delay(&mut self) -> Delay { - Delay::new(Instant::now() + self.backoff.next().unwrap()) + delay_for(self.backoff.next().unwrap()) } /** * Polls for whether the underlying UnixStream is connected and ready to receive writes. **/ - fn poll_connection(&mut self) -> Poll<&mut FramedWrite, ()> { + fn poll_connection( + &mut self, + ) -> Poll<&mut CompatSink, Bytes>, ()> { loop { self.state = match self.state { UnixSinkState::Open(ref mut stream) => { @@ -115,19 +112,20 @@ impl UnixSink { error, path: &self.path }); - UnixSinkState::Backoff(self.next_delay()) + let delay = self.next_delay(); + let delay = Box::new(async move { Ok(delay.await) }.boxed().compat()); + UnixSinkState::Backoff(delay) } Ok(Async::Ready(stream)) => { emit!(UnixSocketConnectionEstablished { path: &self.path }); self.backoff = Self::fresh_backoff(); let out = FramedWrite::new(stream, BytesCodec::new()); - UnixSinkState::Open(out) + UnixSinkState::Open(CompatSink::new(out)) } }, UnixSinkState::Backoff(ref mut delay) => match delay.poll() { Ok(Async::NotReady) => return Ok(Async::NotReady), - // Err can only occur if the tokio runtime has been shutdown or if more than 2^63 timers have been created - Err(err) => unreachable!(err), + Err(_) => unreachable!(), Ok(Async::Ready(())) => UnixSinkState::Disconnected, }, UnixSinkState::Disconnected => { @@ -135,7 +133,7 @@ impl UnixSink { message = "connecting", path = &field::display(self.path.to_str().unwrap()) ); - let connect_future = UnixStream::connect(&self.path); + let connect_future = UnixStream::connect(self.path.clone()).boxed().compat(); UnixSinkState::Creating(Box::new(connect_future)) } } @@ -144,7 +142,7 @@ impl UnixSink { } impl Sink for UnixSink { - type SinkItem = Bytes; + type SinkItem = bytes::Bytes; type SinkError = (); fn start_send(&mut self, line: Self::SinkItem) -> StartSend { @@ -154,20 +152,29 @@ impl Sink for UnixSink { Err(_) => { unreachable!(); // poll_ready() should never return an error } - Ok(Async::Ready(connection)) => match connection.start_send(line) { - Err(error) => { - emit!(UnixSocketError { - error, - path: &self.path - }); - self.state = UnixSinkState::Disconnected; - Ok(AsyncSink::Ready) - } - Ok(res) => { - emit!(UnixSocketEventSent { byte_size }); - Ok(res) + Ok(Async::Ready(connection)) => { + let line = Bytes::copy_from_slice(&line); + match connection.start_send(line) { + Err(error) => { + emit!(UnixSocketError { + error, + path: &self.path + }); + self.state = UnixSinkState::Disconnected; + Ok(AsyncSink::Ready) + } + Ok(res) => { + emit!(UnixSocketEventSent { byte_size }); + Ok(match res { + AsyncSink::Ready => AsyncSink::Ready, + AsyncSink::NotReady(bytes) => { + let bytes = bytes::Bytes::from(&bytes[..]); + AsyncSink::NotReady(bytes) + } + }) + } } - }, + } } } @@ -208,14 +215,12 @@ mod tests { #[tokio::test] async fn unix_sink_healthcheck() { - let path = temp_uds_path("valid_uds"); - let _listener = UnixListener::bind(&path).unwrap(); - let healthcheck = unix_healthcheck(path); - assert!(healthcheck.wait().is_ok()); + let good_path = temp_uds_path("valid_uds"); + let _listener = UnixListener::bind(&good_path).unwrap(); + assert!(healthcheck(good_path).await.is_ok()); let bad_path = temp_uds_path("no_one_listening"); - let bad_healthcheck = unix_healthcheck(bad_path); - assert!(bad_healthcheck.wait().is_err()); + assert!(healthcheck(bad_path).await.is_err()); } #[test] From 9ece8240bc3edcedb601e83c78afdb29efe42ffe Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Fri, 17 Jul 2020 23:06:45 +0300 Subject: [PATCH 09/29] sinks/util/tcp: remove codec Signed-off-by: Kirill Fomichev --- src/sinks/util/tcp.rs | 63 +++++++++++++++++++++++++++++-------------- 1 file changed, 43 insertions(+), 20 deletions(-) diff --git a/src/sinks/util/tcp.rs b/src/sinks/util/tcp.rs index 5dad7fe47a6a3..9be321c905397 100644 --- a/src/sinks/util/tcp.rs +++ b/src/sinks/util/tcp.rs @@ -10,21 +10,28 @@ use crate::{ tls::{MaybeTlsConnector, MaybeTlsSettings, MaybeTlsStream, TlsConfig}, topology::config::SinkContext, }; -use bytes::Bytes; +use bytes05::Bytes; +use futures::{ + compat::{Compat01As03, CompatSink}, + FutureExt, TryFutureExt, +}; use futures01::{ future, stream::iter_ok, try_ready, Async, AsyncSink, Future, Poll, Sink, StartSend, }; use serde::{Deserialize, Serialize}; use snafu::Snafu; -use std::io::{ErrorKind, Read}; -use std::net::SocketAddr; -use std::time::{Duration, Instant}; -use tokio01::{ - codec::{BytesCodec, FramedWrite}, - net::tcp::TcpStream, - timer::Delay, +use std::{ + io::{ErrorKind, Read}, + net::SocketAddr, + time::Duration, }; +use tokio::time::{delay_for, Delay}; +use tokio01::net::tcp::TcpStream; use tokio_retry::strategy::ExponentialBackoff; +use tokio_util::{ + codec::{BytesCodec, FramedWrite}, + compat::{Compat, FuturesAsyncWriteCompatExt}, +}; #[derive(Deserialize, Serialize, Debug, Clone)] #[serde(deny_unknown_fields)] @@ -79,10 +86,11 @@ enum TcpSinkState { ResolvingDns(crate::dns::ResolverFuture), Connecting(MaybeTlsConnector), Connected(TcpOrTlsStream), - Backoff(Delay), + Backoff(Box + Send>), } -type TcpOrTlsStream = FramedWrite, BytesCodec>; +type TcpOrTlsStream = + CompatSink>>, BytesCodec>, Bytes>; impl TcpSink { pub fn new(host: String, port: u16, resolver: Resolver, tls: MaybeTlsSettings) -> Self { @@ -115,7 +123,12 @@ impl TcpSink { } fn next_delay(&mut self) -> Delay { - Delay::new(Instant::now() + self.backoff.next().unwrap()) + delay_for(self.backoff.next().unwrap()) + } + + fn next_delay01(&mut self) -> Box + Send> { + let delay = self.next_delay(); + Box::new(async move { Ok(delay.await) }.boxed().compat()) } fn poll_connection(&mut self) -> Poll<&mut TcpOrTlsStream, ()> { @@ -137,24 +150,24 @@ impl TcpSink { Ok(connector) => TcpSinkState::Connecting(connector), Err(error) => { error!(message = "unable to connect", %error); - TcpSinkState::Backoff(self.next_delay()) + TcpSinkState::Backoff(self.next_delay01()) } } } else { error!("DNS resolved but there were no IP addresses."); - TcpSinkState::Backoff(self.next_delay()) + TcpSinkState::Backoff(self.next_delay01()) } } Ok(Async::NotReady) => return Ok(Async::NotReady), Err(error) => { error!(message = "unable to resolve dns.", %error); - TcpSinkState::Backoff(self.next_delay()) + TcpSinkState::Backoff(self.next_delay01()) } }, TcpSinkState::Backoff(ref mut delay) => match delay.poll() { Ok(Async::NotReady) => return Ok(Async::NotReady), // Err can only occur if the tokio runtime has been shutdown or if more than 2^63 timers have been created - Err(err) => unreachable!(err), + Err(_) => unreachable!(), Ok(Async::Ready(())) => { debug!(message = "disconnected."); TcpSinkState::Disconnected @@ -166,12 +179,14 @@ impl TcpSink { peer_addr: stream.peer_addr().ok(), }); self.backoff = Self::fresh_backoff(); - TcpSinkState::Connected(FramedWrite::new(stream, BytesCodec::new())) + let stream = Compat01As03::new(stream).compat_write(); + let out = FramedWrite::new(stream, BytesCodec::new()); + TcpSinkState::Connected(CompatSink::new(out)) } Ok(Async::NotReady) => return Ok(Async::NotReady), Err(error) => { emit!(TcpConnectionFailed { error }); - TcpSinkState::Backoff(self.next_delay()) + TcpSinkState::Backoff(self.next_delay01()) } }, TcpSinkState::Connected(ref mut connection) => return Ok(Async::Ready(connection)), @@ -181,7 +196,7 @@ impl TcpSink { } impl Sink for TcpSink { - type SinkItem = Bytes; + type SinkItem = bytes::Bytes; type SinkError = (); fn start_send(&mut self, line: Self::SinkItem) -> StartSend { @@ -198,7 +213,8 @@ impl Sink for TcpSink { // // If this returns `WouldBlock` we know the connection is still // valid and the write will most likely succeed. - match connection.get_mut().read(&mut [0u8; 1]) { + let stream = connection.get_mut().get_mut().get_mut().get_mut(); + match stream.read(&mut [0u8; 1]) { Err(error) if error.kind() != ErrorKind::WouldBlock => { emit!(TcpConnectionDisconnected { error }); self.state = TcpSinkState::Disconnected; @@ -225,13 +241,20 @@ impl Sink for TcpSink { emit!(TcpEventSent { byte_size: line.len() }); + let line = Bytes::copy_from_slice(&line); match connection.start_send(line) { Err(error) => { error!(message = "connection disconnected.", %error); self.state = TcpSinkState::Disconnected; Ok(AsyncSink::Ready) } - Ok(ok) => Ok(ok), + Ok(res) => Ok(match res { + AsyncSink::Ready => AsyncSink::Ready, + AsyncSink::NotReady(bytes) => { + let bytes = bytes::Bytes::from(&bytes[..]); + AsyncSink::NotReady(bytes) + } + }), } } } From db9ab6d9e58fca86ee4d478fcd34399ac4c51d73 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 01:15:13 +0300 Subject: [PATCH 10/29] sources/socket/mod: remove tokio-uds Signed-off-by: Kirill Fomichev --- src/sources/socket/mod.rs | 136 +++++++++++++++++--------------------- 1 file changed, 62 insertions(+), 74 deletions(-) diff --git a/src/sources/socket/mod.rs b/src/sources/socket/mod.rs index 2e5c6217d6618..094bead15d41e 100644 --- a/src/sources/socket/mod.rs +++ b/src/sources/socket/mod.rs @@ -141,22 +141,26 @@ mod test { use crate::topology::config::{GlobalOptions, SourceConfig}; use bytes::Bytes; #[cfg(unix)] - use futures01::Sink; + use futures::{compat::Future01CompatExt, stream, SinkExt}; use futures01::{ - stream, sync::{mpsc, oneshot}, Future, Stream, }; - use std::net::UdpSocket; #[cfg(unix)] use std::path::PathBuf; - use std::sync::atomic::{AtomicBool, Ordering}; - use std::sync::Arc; - use std::{net::SocketAddr, thread, time::Duration, time::Instant}; + use std::{ + net::{SocketAddr, UdpSocket}, + sync::{ + atomic::{AtomicBool, Ordering}, + Arc, + }, + thread, + time::{Duration, Instant}, + }; #[cfg(unix)] - use tokio01::codec::{FramedWrite, LinesCodec}; + use tokio::net::UnixStream; #[cfg(unix)] - use tokio_uds::UnixStream; + use tokio_util::codec::{FramedWrite, LinesCodec}; //////// TCP TESTS //////// #[test] @@ -378,7 +382,7 @@ mod test { MaybeTlsSettings::Raw(()), ); rt.spawn( - stream::iter_ok::<_, ()>(std::iter::repeat(())) + futures01::stream::iter_ok::<_, ()>(std::iter::repeat(())) .map(|_| Bytes::from("test\n")) .map_err(|_| ()) .forward(sink) @@ -444,7 +448,7 @@ mod test { MaybeTlsSettings::Raw(()), ); rt.spawn( - stream::iter_ok::<_, ()>(0..n) + futures01::stream::iter_ok::<_, ()>(0..n) .map(|i| Bytes::from(format!("{}\n", i))) .forward(sink) .map(|_| ()), @@ -760,95 +764,79 @@ mod test { } #[cfg(unix)] - fn send_lines_unix(path: PathBuf, lines: Vec<&str>) { - let input_stream = - futures01::stream::iter_ok::<_, ()>(lines.clone().into_iter().map(|s| s.to_string())); + async fn send_lines_unix(path: PathBuf, lines: Vec<&str>) { + let socket = UnixStream::connect(path).await.unwrap(); + let mut sink = FramedWrite::new(socket, LinesCodec::new()); - UnixStream::connect(&path) - .map_err(|e| panic!("{:}", e)) - .and_then(|socket| { - let out = - FramedWrite::new(socket, LinesCodec::new()).sink_map_err(|e| panic!("{:?}", e)); - - input_stream - .forward(out) - .map(|(_source, sink)| sink) - .and_then(|sink| { - let socket = sink.into_inner().into_inner(); - // In tokio 0.1 `AsyncWrite::shutdown` for `TcpStream` is a noop. - // See https://docs.rs/tokio-tcp/0.1.4/src/tokio_tcp/stream.rs.html#917 - // Use `TcpStream::shutdown` instead - it actually does something. - socket - .shutdown(std::net::Shutdown::Both) - .map(|_| ()) - .map_err(|e| panic!("{:}", e)) - }) - }) - .wait() - .unwrap(); + let lines = lines.into_iter().map(|s| Ok(s.to_string())); + let lines = lines.collect::>(); + sink.send_all(&mut stream::iter(lines)).await.unwrap(); + + let socket = sink.into_inner(); + socket.shutdown(std::net::Shutdown::Both).unwrap(); } #[cfg(unix)] #[test] fn unix_message() { let (tx, rx) = mpsc::channel(2); - let (path, mut rt) = init_unix(tx); + rt.block_on_std(async move { + send_lines_unix(path, vec!["test"]).await; - send_lines_unix(path, vec!["test"]); - - let events = rt.block_on(collect_n(rx, 1)).ok().unwrap(); + let events = collect_n(rx, 1).compat().await.ok().unwrap(); - assert_eq!(1, events.len()); - assert_eq!( - events[0].as_log()[&event::log_schema().message_key()], - "test".into() - ); - assert_eq!( - events[0].as_log()[event::log_schema().source_type_key()], - "socket".into() - ); + assert_eq!(1, events.len()); + assert_eq!( + events[0].as_log()[&event::log_schema().message_key()], + "test".into() + ); + assert_eq!( + events[0].as_log()[event::log_schema().source_type_key()], + "socket".into() + ); + }); } #[cfg(unix)] #[test] fn unix_multiple_messages() { let (tx, rx) = mpsc::channel(10); - let (path, mut rt) = init_unix(tx); + rt.block_on_std(async move { + send_lines_unix(path, vec!["test\ntest2"]).await; + let events = collect_n(rx, 2).compat().await.ok().unwrap(); - send_lines_unix(path, vec!["test\ntest2"]); - let events = rt.block_on(collect_n(rx, 2)).ok().unwrap(); - - assert_eq!(2, events.len()); - assert_eq!( - events[0].as_log()[&event::log_schema().message_key()], - "test".into() - ); - assert_eq!( - events[1].as_log()[&event::log_schema().message_key()], - "test2".into() - ); + assert_eq!(2, events.len()); + assert_eq!( + events[0].as_log()[&event::log_schema().message_key()], + "test".into() + ); + assert_eq!( + events[1].as_log()[&event::log_schema().message_key()], + "test2".into() + ); + }); } #[cfg(unix)] #[test] fn unix_multiple_packets() { let (tx, rx) = mpsc::channel(10); - let (path, mut rt) = init_unix(tx); + rt.block_on_std(async move { + send_lines_unix(path, vec!["test", "test2"]).await; + let events = collect_n(rx, 2).compat().await.ok().unwrap(); - send_lines_unix(path, vec!["test", "test2"]); - let events = rt.block_on(collect_n(rx, 2)).ok().unwrap(); - - assert_eq!(2, events.len()); - assert_eq!( - events[0].as_log()[&event::log_schema().message_key()], - "test".into() - ); - assert_eq!( - events[1].as_log()[&event::log_schema().message_key()], - "test2".into() - ); + assert_eq!(2, events.len()); + assert_eq!( + events[0].as_log()[&event::log_schema().message_key()], + "test".into() + ); + assert_eq!( + events[1].as_log()[&event::log_schema().message_key()], + "test2".into() + ); + }); } } From edc626f830f30546d4bbdb9bc748e95705d766eb Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 11:14:39 +0300 Subject: [PATCH 11/29] sources/socket/unix: update codec Signed-off-by: Kirill Fomichev --- Cargo.toml | 4 +- src/async_read.rs | 91 ++++++++++---------------- src/internal_events/unix.rs | 8 ++- src/sources/socket/unix.rs | 2 +- src/sources/util/unix.rs | 126 ++++++++++++++++++++---------------- 5 files changed, 111 insertions(+), 120 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index b2cc187c21240..7753377c93a18 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -257,7 +257,7 @@ sources-journald = [] sources-kafka = [] sources-logplex = ["warp", "sources-tls"] sources-prometheus = [] -sources-socket = ["bytesize", "listenfd", "tokio-uds", "sources-tls"] +sources-socket = ["bytesize", "listenfd", "tokio/uds", "tokio-uds", "sources-tls"] sources-splunk_hec = ["bytesize", "warp", "sources-tls"] sources-statsd = [] sources-stdin = ["bytesize"] @@ -375,7 +375,7 @@ sinks-loki = ["bytesize"] sinks-new_relic_logs = ["bytesize", "sinks-http"] sinks-prometheus = [] sinks-sematext_logs = ["sinks-elasticsearch"] -sinks-socket = ["tokio-uds"] +sinks-socket = ["tokio/uds", "tokio-uds"] sinks-papertrail = ["sinks-socket"] sinks-splunk_hec = ["bytesize"] sinks-statsd = [] diff --git a/src/async_read.rs b/src/async_read.rs index 47a623218dab6..e3d655e0cf5dc 100644 --- a/src/async_read.rs +++ b/src/async_read.rs @@ -1,80 +1,55 @@ -use bytes::BufMut; -use futures01::{Async, Future, IntoFuture, Poll}; -use std::io::Read; -use tokio01::io::AsyncRead; +use pin_project::pin_project; +use std::{ + future::Future, + mem::MaybeUninit, + pin::Pin, + task::{Context, Poll}, +}; +use tokio::io::{AsyncRead, Result as IoResult}; -/// A AsyncRead combinator which reads from a reader until a future resolves. -/// -/// This structure is produced by the [`AsyncReadExt::read_until`] method. -#[derive(Clone, Debug)] -pub struct AllowReadUntil { - reader: S, - until: F, - until_res: Option, - free: bool, -} - -/// This `AsyncRead` extension trait provides a `read_until` method that terminates the reader once -/// the given future resolves. -pub trait AsyncAllowReadExt: AsyncRead { +pub trait AsyncReadExt: AsyncRead { /// Read data from this reader until the given future resolves. - /// - /// If the future produces an error, the read will be allowed to continue indefinitely. - fn allow_read_until(self, until: U) -> AllowReadUntil + fn allow_read_until(self, until: F) -> AllowReadUntil where - U: IntoFuture, Self: Sized, + F: Future, { AllowReadUntil { reader: self, - until: until.into_future(), - until_res: None, - free: false, + until, } } } -impl AsyncAllowReadExt for S where S: AsyncRead {} +impl AsyncReadExt for S where S: AsyncRead {} -impl Read for AllowReadUntil -where - S: Read, -{ - fn read(&mut self, buf: &mut [u8]) -> Result { - self.reader.read(buf) - } +/// A AsyncRead combinator which reads from a reader until a future resolves. +#[pin_project] +#[derive(Clone, Debug)] +pub struct AllowReadUntil { + #[pin] + reader: S, + #[pin] + until: F, } -impl AsyncRead for AllowReadUntil +impl AsyncRead for AllowReadUntil where S: AsyncRead, - F: Future, + F: Future, { - unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool { - self.reader.prepare_uninitialized_buffer(buf) - } - - fn read_buf(&mut self, buf: &mut B) -> Poll - where - Self: Sized, - { - if !self.free { - match self.until.poll() { - Ok(Async::Ready(res)) => { - // future resolved -- terminate reader - self.until_res = Some(res); - return Ok(Async::Ready(0)); - } - Err(_) => { - // future failed -- unclear whether we should stop or continue? - // to provide a mechanism for the creator to let the stream run forever, - // we interpret this as "run forever". - self.free = true; - } - Ok(Async::NotReady) => {} + fn poll_read(self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { + let this = self.project(); + match this.until.poll(cx) { + Poll::Ready(_) => { + // TODO: Need proper shutdown + Poll::Ready(Ok(0)) } + Poll::Pending => this.reader.poll_read(cx, buf), } + } - self.reader.read_buf(buf) + unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [MaybeUninit]) -> bool { + self.reader.prepare_uninitialized_buffer(buf) } } diff --git a/src/internal_events/unix.rs b/src/internal_events/unix.rs index 95ab0f8afefdc..6d68ec3c1a35d 100644 --- a/src/internal_events/unix.rs +++ b/src/internal_events/unix.rs @@ -41,12 +41,14 @@ impl InternalEvent for UnixSocketConnectionFailure<'_> { } #[derive(Debug)] -pub struct UnixSocketError<'a> { - pub error: std::io::Error, +pub struct UnixSocketError<'a, E> { + pub error: E, pub path: &'a std::path::Path, } -impl InternalEvent for UnixSocketError<'_> { +impl + std::fmt::Debug + std::fmt::Display> InternalEvent + for UnixSocketError<'_, E> +{ fn emit_logs(&self) { debug!( message = "unix socket error.", diff --git a/src/sources/socket/unix.rs b/src/sources/socket/unix.rs index 8c62d7b6de81e..23f4842c8e7f9 100644 --- a/src/sources/socket/unix.rs +++ b/src/sources/socket/unix.rs @@ -8,7 +8,7 @@ use bytes::Bytes; use futures01::sync::mpsc; use serde::{Deserialize, Serialize}; use std::path::PathBuf; -use tokio01::codec::LinesCodec; +use tokio_util::codec::LinesCodec; #[derive(Deserialize, Serialize, Debug, Clone)] #[serde(deny_unknown_fields)] diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index 7215fa28966a5..43d7e1c51d42a 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -1,15 +1,16 @@ use crate::{ - async_read::AsyncAllowReadExt, emit, event::Event, internal_events::UnixSocketError, - shutdown::ShutdownSignal, sources::Source, stream::StreamExt01, + async_read::AsyncReadExt, emit, event::Event, internal_events::UnixSocketError, + shutdown::ShutdownSignal, sources::Source, }; use bytes::Bytes; -use futures01::{future, sync::mpsc, Future, Sink, Stream}; -use std::path::PathBuf; -use tokio01::{ - self, - codec::{Decoder, FramedRead}, +use futures::{ + compat::{Future01CompatExt, Sink01CompatExt}, + future, FutureExt, StreamExt, TryFutureExt, }; -use tokio_uds::UnixListener; +use futures01::{sync::mpsc, Sink}; +use std::path::PathBuf; +use tokio::net::UnixListener; +use tokio_util::codec::{Decoder, FramedRead}; use tracing::field; use tracing_futures::Instrument; @@ -18,66 +19,79 @@ use tracing_futures::Instrument; * for build_event can allow for different source-specific logic (such as decoding syslog messages * in the syslog source). **/ -pub fn build_unix_source< - D: Decoder + Clone + Send + 'static, ->( - path: PathBuf, +pub fn build_unix_source( + listen_path: PathBuf, decoder: D, host_key: String, shutdown: ShutdownSignal, out: mpsc::Sender, - build_event: impl Fn(&str, Option, &str) -> Option - + std::marker::Send - + std::marker::Sync - + std::clone::Clone - + 'static, -) -> Source { + build_event: impl Fn(&str, Option, &str) -> Option + Clone + Send + Sync + 'static, +) -> Source +where + D: Decoder + Clone + Send + 'static, + E: From + std::fmt::Debug + std::fmt::Display, +{ let out = out.sink_map_err(|e| error!("error sending line: {:?}", e)); - Box::new(future::lazy(move || { - let listener = UnixListener::bind(&path).expect("failed to bind to listener socket"); + let fut = async move { + let mut listener = + UnixListener::bind(&listen_path).expect("failed to bind to listener socket"); + info!(message = "listening.", ?listen_path, r#type = "unix"); - info!(message = "listening.", ?path, r#type = "unix"); + let mut stream = listener.incoming().take_until(shutdown.clone().compat()); + while let Some(socket) = stream.next().await { + let socket = match socket { + Err(error) => { + error!("failed to accept socket; error = {:?}", error); + continue; + } + Ok(socket) => socket, + }; - listener - .incoming() - .take_until(shutdown.clone()) - .map_err(|e| error!("failed to accept socket; error = {:?}", e)) - .for_each(move |socket| { - let out = out.clone(); - let peer_addr = socket.peer_addr().ok(); - let host_key = host_key.clone(); - let listen_path = path.clone(); + let listen_path = listen_path.clone(); + let host_key = host_key.clone(); + let shutdown = shutdown.clone(); - let span = info_span!("connection"); - let path = if let Some(addr) = peer_addr { - if let Some(path) = addr.as_pathname().map(|e| e.to_owned()) { - span.record("peer_path", &field::debug(&path)); - Some(path) - } else { - None - } + let span = info_span!("connection"); + let path = if let Some(addr) = socket.peer_addr().ok() { + if let Some(path) = addr.as_pathname().map(|e| e.to_owned()) { + span.record("peer_path", &field::debug(&path)); + Some(path) } else { None - }; + } + } else { + None + }; + + let build_event = build_event.clone(); + let received_from: Option = + path.map(|p| p.to_string_lossy().into_owned().into()); - let build_event = build_event.clone(); - let received_from: Option = - path.map(|p| p.to_string_lossy().into_owned().into()); - let lines_in = - FramedRead::new(socket.allow_read_until(shutdown.clone()), decoder.clone()) - .filter_map(move |line| { - build_event(&host_key, received_from.clone(), &line) - }) - .map_err(move |error| { - emit!(UnixSocketError { - error, - path: &listen_path, - }); + let stream = socket.allow_read_until(shutdown.compat().map(|_| ())); + let stream = FramedRead::new(stream, decoder.clone()).filter_map(move |line| { + future::ready(match line { + Ok(line) => build_event(&host_key, received_from.clone(), &line).map(Ok), + Err(error) => { + emit!(UnixSocketError { + error, + path: &listen_path }); + None + } + }) + }); + + let fut = stream + .forward(out.clone().sink_compat()) + .map(|_| info!("finished sending")) + .instrument(span); + + tokio::spawn(fut); + } + + Ok(()) + }; - let handler = lines_in.forward(out).map(|_| info!("finished sending")); - tokio01::spawn(handler.instrument(span)) - }) - })) + Box::new(fut.boxed().compat()) } From 3d9a874df29496138237eb7c275a154766ee5d2b Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 19:07:12 +0300 Subject: [PATCH 12/29] sources/util/tcp: update codec Signed-off-by: Kirill Fomichev --- Cargo.lock | 20 -------------------- Cargo.toml | 7 +++---- src/sources/socket/tcp.rs | 4 ++-- src/sources/util/tcp.rs | 22 ++++++++++++---------- src/sources/util/unix.rs | 2 +- 5 files changed, 18 insertions(+), 37 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 29a935d829d05..c0620d591fcc4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4594,7 +4594,6 @@ dependencies = [ "tokio-threadpool", "tokio-timer", "tokio-udp", - "tokio-uds", "tracing-core", ] @@ -4859,24 +4858,6 @@ dependencies = [ "tokio-reactor", ] -[[package]] -name = "tokio-uds" -version = "0.2.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab57a4ac4111c8c9dbcf70779f6fc8bc35ae4b2454809febac840ad19bd7e4e0" -dependencies = [ - "bytes 0.4.12", - "futures 0.1.29", - "iovec", - "libc", - "log 0.4.8", - "mio", - "mio-uds", - "tokio-codec", - "tokio-io", - "tokio-reactor", -] - [[package]] name = "tokio-util" version = "0.3.1" @@ -5521,7 +5502,6 @@ dependencies = [ "tokio-retry", "tokio-signal", "tokio-test", - "tokio-uds", "tokio-util", "tokio01-test", "toml 0.4.10", diff --git a/Cargo.toml b/Cargo.toml index 7753377c93a18..01341615c7bfb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -44,7 +44,7 @@ tracing-limit = { path = "lib/tracing-limit" } # Tokio / Futures futures01 = { package = "futures", version = "0.1.25" } futures = { version = "0.3", default-features = false, features = ["compat", "io-compat"] } -tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "uds", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync"], default-features = false } +tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync"], default-features = false } tokio = { version = "0.2.13", features = ["blocking", "fs", "sync", "macros", "test-util", "rt-core", "io-std"] } tokio-openssl03 = { package = "tokio-openssl", version = "0.3.0" } tokio-openssl = "0.4.0" @@ -114,7 +114,6 @@ typetag = "0.1" toml = "0.4" syslog = "5" syslog_loose = { version = "0.3.0", optional = true } -tokio-uds = { version = "0.2.5", optional = true } derive_is_enum_variant = "0.1.1" leveldb = { git = "https://github.com/timberio/leveldb", optional = true, default-features = false } db-key = "0.0.5" @@ -257,7 +256,7 @@ sources-journald = [] sources-kafka = [] sources-logplex = ["warp", "sources-tls"] sources-prometheus = [] -sources-socket = ["bytesize", "listenfd", "tokio/uds", "tokio-uds", "sources-tls"] +sources-socket = ["bytesize", "listenfd", "tokio/uds", "sources-tls"] sources-splunk_hec = ["bytesize", "warp", "sources-tls"] sources-statsd = [] sources-stdin = ["bytesize"] @@ -375,7 +374,7 @@ sinks-loki = ["bytesize"] sinks-new_relic_logs = ["bytesize", "sinks-http"] sinks-prometheus = [] sinks-sematext_logs = ["sinks-elasticsearch"] -sinks-socket = ["tokio/uds", "tokio-uds"] +sinks-socket = ["tokio/uds"] sinks-papertrail = ["sinks-socket"] sinks-splunk_hec = ["bytesize"] sinks-statsd = [] diff --git a/src/sources/socket/tcp.rs b/src/sources/socket/tcp.rs index b61abc02a26a0..4613819e9d2a2 100644 --- a/src/sources/socket/tcp.rs +++ b/src/sources/socket/tcp.rs @@ -4,8 +4,8 @@ use crate::{ sources::util::{SocketListenAddr, TcpSource}, tls::TlsConfig, }; -use bytes::Bytes; -use codec01::BytesDelimitedCodec; +use bytes05::Bytes; +use codec::BytesDelimitedCodec; use serde::{Deserialize, Serialize}; use string_cache::DefaultAtom as Atom; use tracing::field; diff --git a/src/sources/util/tcp.rs b/src/sources/util/tcp.rs index f935e3eea0148..0ccd8ae0b6bb9 100644 --- a/src/sources/util/tcp.rs +++ b/src/sources/util/tcp.rs @@ -5,7 +5,8 @@ use crate::{ tls::{MaybeTlsIncomingStream, MaybeTlsListener, MaybeTlsSettings}, Event, }; -use bytes::Bytes; +use bytes05::Bytes; +use futures::compat::{Compat, Compat01As03}; use futures01::{future, stream, sync::mpsc, Async, Future, Sink, Stream}; use listenfd::ListenFd; use serde::{de, Deserialize, Deserializer, Serialize}; @@ -15,11 +16,15 @@ use std::{ time::{Duration, Instant}, }; use tokio01::{ - codec::{Decoder, FramedRead}, + // codec::Decoder, net::{TcpListener, TcpStream}, reactor::Handle, timer, }; +use tokio_util::{ + codec::{Decoder, FramedRead}, + compat::FuturesAsyncReadCompatExt, +}; use tracing::{field, Span}; use tracing_futures::Instrument; @@ -61,11 +66,7 @@ pub trait TcpSource: Clone + Send + 'static { fn decoder(&self) -> Self::Decoder; - fn build_event( - &self, - frame: ::Item, - host: Bytes, - ) -> Option; + fn build_event(&self, frame: ::Item, host: Bytes) -> Option; fn run( self, @@ -163,7 +164,8 @@ fn handle_stream( ) { let mut shutdown = Some(shutdown); let mut token = None; - let mut reader = FramedRead::new(socket, source.decoder()); + let socket = Compat01As03::new(socket).compat(); + let mut reader = Compat::new(FramedRead::new(socket, source.decoder())); let handler = stream::poll_fn(move || { // Gracefull shutdown procedure if let Some(future) = shutdown.as_mut() { @@ -172,8 +174,8 @@ fn handle_stream( debug!("Start gracefull shutdown"); // Close our write part of TCP socket to signal the other side // that it should stop writing and close the channel. - if let Some(socket) = reader.get_ref().get_ref() { - if let Err(error)=socket.shutdown(Shutdown::Write){ + if let Some(socket) = reader.get_ref().get_ref().get_ref().get_ref().get_ref() { + if let Err(error) = socket.shutdown(Shutdown::Write) { warn!(message = "Failed in signalling to the other side to close the TCP channel.",%error); } } else { diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index 43d7e1c51d42a..f646ff334225c 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -29,7 +29,7 @@ pub fn build_unix_source( ) -> Source where D: Decoder + Clone + Send + 'static, - E: From + std::fmt::Debug + std::fmt::Display, + E: From + std::fmt::Debug + std::fmt::Display, // change to std::io::Error ? internal_events too! { let out = out.sink_map_err(|e| error!("error sending line: {:?}", e)); From b874167886f579b241b7911fbff9cfc66dabf0f3 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 19:14:16 +0300 Subject: [PATCH 13/29] sources/syslog: fix new codec Signed-off-by: Kirill Fomichev --- src/internal_events/tcp.rs | 6 +++--- src/sources/socket/tcp.rs | 1 + src/sources/socket/unix.rs | 2 +- src/sources/syslog.rs | 38 +++++++++++++++++++++----------------- src/sources/util/tcp.rs | 6 ++++-- src/sources/util/unix.rs | 4 ++-- src/sources/vector.rs | 7 +++---- 7 files changed, 35 insertions(+), 29 deletions(-) diff --git a/src/internal_events/tcp.rs b/src/internal_events/tcp.rs index f54d45b6bf3ff..821b1dfc614a9 100644 --- a/src/internal_events/tcp.rs +++ b/src/internal_events/tcp.rs @@ -74,11 +74,11 @@ impl InternalEvent for TcpConnectionShutdown { } #[derive(Debug)] -pub struct TcpConnectionError { - pub error: std::io::Error, +pub struct TcpConnectionError { + pub error: T, } -impl InternalEvent for TcpConnectionError { +impl InternalEvent for TcpConnectionError { fn emit_logs(&self) { warn!(message = "connection error.", error = %self.error, rate_limit_secs = 10); } diff --git a/src/sources/socket/tcp.rs b/src/sources/socket/tcp.rs index 4613819e9d2a2..e10d9ba27c3a9 100644 --- a/src/sources/socket/tcp.rs +++ b/src/sources/socket/tcp.rs @@ -48,6 +48,7 @@ pub struct RawTcpSource { } impl TcpSource for RawTcpSource { + type Error = std::io::Error; type Decoder = BytesDelimitedCodec; fn decoder(&self) -> Self::Decoder { diff --git a/src/sources/socket/unix.rs b/src/sources/socket/unix.rs index 23f4842c8e7f9..b981ec0510b7b 100644 --- a/src/sources/socket/unix.rs +++ b/src/sources/socket/unix.rs @@ -4,7 +4,7 @@ use crate::{ shutdown::ShutdownSignal, sources::{util::build_unix_source, Source}, }; -use bytes::Bytes; +use bytes05::Bytes; use futures01::sync::mpsc; use serde::{Deserialize, Serialize}; use std::path::PathBuf; diff --git a/src/sources/syslog.rs b/src/sources/syslog.rs index fd6c2489cd107..1f43e18bbda14 100644 --- a/src/sources/syslog.rs +++ b/src/sources/syslog.rs @@ -9,7 +9,7 @@ use crate::{ tls::{MaybeTlsSettings, TlsConfig}, topology::config::{DataType, GlobalOptions, SourceConfig, SourceDescription}, }; -use bytes::{Bytes, BytesMut}; +use bytes05::{Buf, Bytes, BytesMut}; use chrono::{Datelike, Utc}; use derive_is_enum_variant::is_enum_variant; use futures01::{future, sync::mpsc, Future, Sink, Stream}; @@ -18,12 +18,12 @@ use std::io; use std::net::SocketAddr; #[cfg(unix)] use std::path::PathBuf; -use syslog_loose::{self, IncompleteDate, Message, ProcId, Protocol}; +use syslog_loose::{IncompleteDate, Message, ProcId, Protocol}; use tokio01::{ - self, - codec::{BytesCodec, Decoder, LinesCodec}, + codec::BytesCodec, net::{UdpFramed, UdpSocket}, }; +use tokio_util::codec::{Decoder, LinesCodec, LinesCodecError}; use tracing::field; #[derive(Deserialize, Serialize, Debug)] @@ -125,6 +125,7 @@ struct SyslogTcpSource { } impl TcpSource for SyslogTcpSource { + type Error = LinesCodecError; type Decoder = SyslogDecoder; fn decoder(&self) -> Self::Decoder { @@ -156,7 +157,7 @@ impl SyslogDecoder { } } - fn octet_decode(&self, src: &mut BytesMut) -> Result, io::Error> { + fn octet_decode(&self, src: &mut BytesMut) -> Result, LinesCodecError> { // Encoding scheme: // // len ' ' data @@ -171,10 +172,10 @@ impl SyslogDecoder { .map_err(|_| ()) .and_then(|num| num.parse().map_err(|_| ())) .map_err(|_| { - io::Error::new( + LinesCodecError::Io(io::Error::new( io::ErrorKind::InvalidData, "Unable to decode message len as number", - ) + )) })?; let from = i + 1; @@ -183,10 +184,10 @@ impl SyslogDecoder { if let Some(msg) = src.get(from..to) { let s = std::str::from_utf8(msg) .map_err(|_| { - io::Error::new( + LinesCodecError::Io(io::Error::new( io::ErrorKind::InvalidData, "Unable to decode message as UTF8", - ) + )) })? .to_string(); src.advance(to); @@ -198,15 +199,18 @@ impl SyslogDecoder { Ok(None) } else { // This is certainly mallformed, and there is no recovering from this. - Err(io::Error::new( + Err(LinesCodecError::Io(io::Error::new( io::ErrorKind::Other, "frame length limit exceeded", - )) + ))) } } /// None if this is not octet counting encoded - fn checked_decode(&self, src: &mut BytesMut) -> Option, io::Error>> { + fn checked_decode( + &self, + src: &mut BytesMut, + ) -> Option, LinesCodecError>> { if let Some(&first_byte) = src.get(0) { if 49 <= first_byte && first_byte <= 57 { // First character is non zero number so we can assume that @@ -221,10 +225,9 @@ impl SyslogDecoder { impl Decoder for SyslogDecoder { type Item = String; + type Error = LinesCodecError; - type Error = io::Error; - - fn decode(&mut self, src: &mut BytesMut) -> Result, io::Error> { + fn decode(&mut self, src: &mut BytesMut) -> Result, Self::Error> { if let Some(ret) = self.checked_decode(src) { ret } else { @@ -233,7 +236,7 @@ impl Decoder for SyslogDecoder { } } - fn decode_eof(&mut self, buf: &mut BytesMut) -> Result, io::Error> { + fn decode_eof(&mut self, buf: &mut BytesMut) -> Result, Self::Error> { if let Some(ret) = self.checked_decode(buf) { ret } else { @@ -321,7 +324,8 @@ fn event_from_str(host_key: &str, default_host: Option, line: &str) -> Op event.as_mut_log().insert("source_ip", default_host); } - if let Some(parsed_host) = parsed.hostname.map(Bytes::from).or(default_host) { + let parsed_hostname = parsed.hostname.map(|x| Bytes::from(x.to_owned())); + if let Some(parsed_host) = parsed_hostname.or(default_host) { event.as_mut_log().insert(host_key, parsed_host); } diff --git a/src/sources/util/tcp.rs b/src/sources/util/tcp.rs index 0ccd8ae0b6bb9..900c415dcc857 100644 --- a/src/sources/util/tcp.rs +++ b/src/sources/util/tcp.rs @@ -16,7 +16,6 @@ use std::{ time::{Duration, Instant}, }; use tokio01::{ - // codec::Decoder, net::{TcpListener, TcpStream}, reactor::Handle, timer, @@ -62,7 +61,10 @@ fn make_listener( } pub trait TcpSource: Clone + Send + 'static { - type Decoder: Decoder + Send + 'static; + // Should be default: `std::io::Error`. + // Right now this is unstable: https://github.com/rust-lang/rust/issues/29661 + type Error: From + std::fmt::Debug + std::fmt::Display; + type Decoder: Decoder + Send + 'static; fn decoder(&self) -> Self::Decoder; diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index f646ff334225c..fe9575da8d36e 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -2,7 +2,7 @@ use crate::{ async_read::AsyncReadExt, emit, event::Event, internal_events::UnixSocketError, shutdown::ShutdownSignal, sources::Source, }; -use bytes::Bytes; +use bytes05::Bytes; use futures::{ compat::{Future01CompatExt, Sink01CompatExt}, future, FutureExt, StreamExt, TryFutureExt, @@ -53,7 +53,7 @@ where let shutdown = shutdown.clone(); let span = info_span!("connection"); - let path = if let Some(addr) = socket.peer_addr().ok() { + let path = if let Ok(addr) = socket.peer_addr() { if let Some(path) = addr.as_pathname().map(|e| e.to_owned()) { span.record("peer_path", &field::debug(&path)); Some(path) diff --git a/src/sources/vector.rs b/src/sources/vector.rs index 6decd80460a41..fd49921cdc79e 100644 --- a/src/sources/vector.rs +++ b/src/sources/vector.rs @@ -7,11 +7,11 @@ use crate::{ topology::config::{DataType, GlobalOptions, SourceConfig, SourceDescription}, Event, }; -use bytes::{Bytes, BytesMut}; +use bytes05::{Bytes, BytesMut}; use futures01::sync::mpsc; use prost::Message; use serde::{Deserialize, Serialize}; -use tokio01::codec::LengthDelimitedCodec; +use tokio_util::codec::LengthDelimitedCodec; #[derive(Deserialize, Serialize, Debug, Clone)] #[serde(deny_unknown_fields)] @@ -68,6 +68,7 @@ impl SourceConfig for VectorConfig { struct VectorSource; impl TcpSource for VectorSource { + type Error = std::io::Error; type Decoder = LengthDelimitedCodec; fn decoder(&self) -> Self::Decoder { @@ -75,8 +76,6 @@ impl TcpSource for VectorSource { } fn build_event(&self, frame: BytesMut, _host: Bytes) -> Option { - let frame = bytes05::Bytes::copy_from_slice(&frame); - let byte_size = frame.len(); match proto::EventWrapper::decode(frame).map(Event::from) { Ok(event) => { From 9ec79359e59ba298d4a62c49ce20582d8234a596 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:27:24 +0300 Subject: [PATCH 14/29] sources/syslog: update codec Signed-off-by: Kirill Fomichev --- src/sources/syslog.rs | 66 +++++++++++++++++++++++++------------------ 1 file changed, 39 insertions(+), 27 deletions(-) diff --git a/src/sources/syslog.rs b/src/sources/syslog.rs index 1f43e18bbda14..89623cc5b92d9 100644 --- a/src/sources/syslog.rs +++ b/src/sources/syslog.rs @@ -5,25 +5,28 @@ use crate::{ event::{self, Event, Value}, internal_events::{SyslogEventReceived, SyslogUdpReadError}, shutdown::ShutdownSignal, - stream::StreamExt01, tls::{MaybeTlsSettings, TlsConfig}, topology::config::{DataType, GlobalOptions, SourceConfig, SourceDescription}, }; use bytes05::{Buf, Bytes, BytesMut}; use chrono::{Datelike, Utc}; use derive_is_enum_variant::is_enum_variant; -use futures01::{future, sync::mpsc, Future, Sink, Stream}; +use futures::{ + compat::{Future01CompatExt, Sink01CompatExt}, + FutureExt, StreamExt, TryFutureExt, +}; +use futures01::{sync::mpsc, Sink}; use serde::{Deserialize, Serialize}; use std::io; use std::net::SocketAddr; #[cfg(unix)] use std::path::PathBuf; use syslog_loose::{IncompleteDate, Message, ProcId, Protocol}; -use tokio01::{ - codec::BytesCodec, - net::{UdpFramed, UdpSocket}, +use tokio::net::UdpSocket; +use tokio_util::{ + codec::{BytesCodec, Decoder, LinesCodec, LinesCodecError}, + udp::UdpFramed, }; -use tokio_util::codec::{Decoder, LinesCodec, LinesCodecError}; use tracing::field; #[derive(Deserialize, Serialize, Debug)] @@ -256,34 +259,43 @@ pub fn udp( let out = out.sink_map_err(|e| error!("error sending line: {:?}", e)); Box::new( - future::lazy(move || { - let socket = UdpSocket::bind(&addr).expect("failed to bind to udp listener socket"); - + async move { + let socket = UdpSocket::bind(&addr) + .await + .expect("failed to bind to udp listener socket"); info!( message = "listening.", addr = &field::display(addr), r#type = "udp" ); - future::ok(socket) - }) - .and_then(move |socket| { - let host_key = host_key.clone(); - - let lines_in = UdpFramed::new(socket, BytesCodec::new()) - .take_until(shutdown) - .filter_map(move |(bytes, received_from)| { + let stream = UdpFramed::new(socket, BytesCodec::new()) + .take_until(shutdown.compat()) + .filter_map(|frame| { let host_key = host_key.clone(); - let received_from = received_from.to_string().into(); - - std::str::from_utf8(&bytes) - .ok() - .and_then(|s| event_from_str(&host_key, Some(received_from), s)) - }) - .map_err(|error| emit!(SyslogUdpReadError { error })); - - lines_in.forward(out).map(|_| info!("finished sending")) - }), + async move { + match frame { + Ok((bytes, received_from)) => { + let received_from = received_from.to_string().into(); + + std::str::from_utf8(&bytes).ok().and_then(|s| { + event_from_str(&host_key, Some(received_from), s).map(Ok) + }) + } + Err(error) => { + emit!(SyslogUdpReadError { error }); + None + } + } + } + }); + + let _ = stream.forward(out.sink_compat()).await; + info!("finished sending"); + Ok(()) + } + .boxed() + .compat(), ) } From 73490855bc0f3c3ef7d05a40942fcd4b2c097fad Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:40:04 +0300 Subject: [PATCH 15/29] sources/socket/udp: update codec Signed-off-by: Kirill Fomichev --- src/sources/socket/udp.rs | 80 +++++++++++++++++++++------------------ src/sources/syslog.rs | 7 ++-- 2 files changed, 48 insertions(+), 39 deletions(-) diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index 6f88be54910cb..1348b14669744 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -3,15 +3,18 @@ use crate::{ internal_events::{UdpEventReceived, UdpSocketError}, shutdown::ShutdownSignal, sources::Source, - stream::StreamExt01, }; -use bytes::Bytes; -use codec01::BytesDelimitedCodec; -use futures01::{future, sync::mpsc, Future, Sink, Stream}; +use codec::BytesDelimitedCodec; +use futures::{ + compat::{Future01CompatExt, Sink01CompatExt}, + FutureExt, StreamExt, TryFutureExt, +}; +use futures01::{sync::mpsc, Sink}; use serde::{Deserialize, Serialize}; -use std::{io, net::SocketAddr}; +use std::{net::SocketAddr}; use string_cache::DefaultAtom as Atom; -use tokio01::net::udp::{UdpFramed, UdpSocket}; +use tokio::net::UdpSocket; +use tokio_util::udp::UdpFramed; /// UDP processes messages per packet, where messages are separated by newline. #[derive(Deserialize, Serialize, Debug, Clone)] @@ -39,41 +42,46 @@ pub fn udp( let out = out.sink_map_err(|e| error!("error sending event: {:?}", e)); Box::new( - future::lazy(move || { - let socket = UdpSocket::bind(&address).expect("failed to bind to udp listener socket"); - + async move { + let socket = UdpSocket::bind(&address) + .await + .expect("failed to bind to udp listener socket"); info!(message = "listening.", %address); - Ok(socket) - }) - .and_then(move |socket| { - let host_key = host_key.clone(); - // UDP processes messages per packet, where messages are separated by newline. - // And stretch to end of packet. - UdpFramed::with_decode(socket, BytesDelimitedCodec::new(b'\n'), true) - .take_until(shutdown) - .map(move |(line, addr): (Bytes, _)| { - let byte_size = line.len(); - let mut event = Event::from(line); + let _ = UdpFramed::new(socket, BytesDelimitedCodec::new(b'\n')) + .take_until(shutdown.compat()) + .filter_map(|frame| { + let host_key = host_key.clone(); + async move { + match frame { + Ok((line, addr)) => { + let byte_size = line.len(); + let mut event = Event::from(line); - event - .as_mut_log() - .insert(event::log_schema().source_type_key(), "socket"); + event + .as_mut_log() + .insert(event::log_schema().source_type_key(), "socket"); - event - .as_mut_log() - .insert(host_key.clone(), addr.to_string()); + event + .as_mut_log() + .insert(host_key, addr.to_string()); - emit!(UdpEventReceived { byte_size }); - event - }) - // Error from Decoder or UdpSocket - .map_err(|error: io::Error| { - emit!(UdpSocketError { error }); + emit!(UdpEventReceived { byte_size }); + Some(Ok(event)) + } + Err(error) => { + emit!(UdpSocketError { error }); + None + } + } + } }) - .forward(out) - // Done with listening and sending - .map(|_| ()) - }), + .forward(out.sink_compat()) + .await; + + Ok(()) + } + .boxed() + .compat(), ) } diff --git a/src/sources/syslog.rs b/src/sources/syslog.rs index 89623cc5b92d9..0f81cb1e980a2 100644 --- a/src/sources/syslog.rs +++ b/src/sources/syslog.rs @@ -269,7 +269,7 @@ pub fn udp( r#type = "udp" ); - let stream = UdpFramed::new(socket, BytesCodec::new()) + let _ = UdpFramed::new(socket, BytesCodec::new()) .take_until(shutdown.compat()) .filter_map(|frame| { let host_key = host_key.clone(); @@ -288,9 +288,10 @@ pub fn udp( } } } - }); + }) + .forward(out.sink_compat()) + .await; - let _ = stream.forward(out.sink_compat()).await; info!("finished sending"); Ok(()) } From f916a55e35315cebbaa5653ba325d77adeb7dec3 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:40:54 +0300 Subject: [PATCH 16/29] remove crate codec01 Signed-off-by: Kirill Fomichev --- Cargo.lock | 11 --- Cargo.toml | 2 - lib/codec01/Cargo.toml | 13 --- lib/codec01/src/lib.rs | 139 ----------------------------- lib/codec01/tests/bytes_delim.rs | 145 ------------------------------- 5 files changed, 310 deletions(-) delete mode 100644 lib/codec01/Cargo.toml delete mode 100644 lib/codec01/src/lib.rs delete mode 100644 lib/codec01/tests/bytes_delim.rs diff --git a/Cargo.lock b/Cargo.lock index c0620d591fcc4..3269b7f8735ef 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -545,16 +545,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "codec01" -version = "0.1.0" -dependencies = [ - "bytes 0.4.12", - "serde_json", - "tokio-codec", - "tracing", -] - [[package]] name = "colored" version = "1.9.2" @@ -5410,7 +5400,6 @@ dependencies = [ "chrono", "cidr-utils", "codec", - "codec01", "colored", "criterion", "db-key", diff --git a/Cargo.toml b/Cargo.toml index 01341615c7bfb..a5ebaea8faf85 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,7 +28,6 @@ extended-description-file = "target/debian-extended-description.txt" [workspace] members = [ ".", - "lib/codec01", "lib/file-source", "lib/tracing-limit", "lib/vector-wasm", @@ -37,7 +36,6 @@ members = [ [dependencies] # Internal libs codec = { path = "lib/codec" } -codec01 = { path = "lib/codec01" } file-source = { path = "lib/file-source" } tracing-limit = { path = "lib/tracing-limit" } diff --git a/lib/codec01/Cargo.toml b/lib/codec01/Cargo.toml deleted file mode 100644 index 8b36c6e82d7c0..0000000000000 --- a/lib/codec01/Cargo.toml +++ /dev/null @@ -1,13 +0,0 @@ -[package] -name = "codec01" -version = "0.1.0" -authors = ["Vector Contributors "] -edition = "2018" - -[dependencies] -bytes = { version = "0.4.10", features = ["serde"] } -tokio-codec = "0.1" -tracing = "0.1.15" - -[dev-dependencies] -serde_json = "1.0.33" diff --git a/lib/codec01/src/lib.rs b/lib/codec01/src/lib.rs deleted file mode 100644 index 39bf0b243c151..0000000000000 --- a/lib/codec01/src/lib.rs +++ /dev/null @@ -1,139 +0,0 @@ -#[macro_use] -extern crate tracing; - -use bytes::{BufMut, Bytes, BytesMut}; -use std::{cmp, io, usize}; -use tokio_codec::{Decoder, Encoder}; - -#[derive(Copy, Clone, Debug, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub struct BytesDelimitedCodec { - delim: u8, - max_length: usize, - is_discarding: bool, - next_index: usize, -} - -impl BytesDelimitedCodec { - /// Returns a `BytesDelimitedCodec` with the specified delimiter. - pub fn new(delim: u8) -> Self { - BytesDelimitedCodec { - delim, - max_length: usize::MAX, - is_discarding: false, - next_index: 0, - } - } - - /// Returns a `BytesDelimitedCodec` with a maximum frame length limit. - pub fn new_with_max_length(delim: u8, max_length: usize) -> Self { - BytesDelimitedCodec { - max_length, - ..BytesDelimitedCodec::new(delim) - } - } - - /// Returns the maximum frame length when decoding. - pub fn max_length(&self) -> usize { - self.max_length - } -} - -impl Decoder for BytesDelimitedCodec { - type Item = Bytes; - type Error = io::Error; - - fn decode(&mut self, buf: &mut BytesMut) -> Result, io::Error> { - loop { - // Determine how far into the buffer we'll search for a newline. If - // there's no max_length set, we'll read to the end of the buffer. - let read_to = cmp::min(self.max_length.saturating_add(1), buf.len()); - - let newline_pos = buf[self.next_index..read_to] - .iter() - .position(|b| *b == self.delim); - - match (self.is_discarding, newline_pos) { - (true, Some(offset)) => { - // If we found a newline, discard up to that offset and - // then stop discarding. On the next iteration, we'll try - // to read a line normally. - buf.advance(offset + self.next_index + 1); - self.is_discarding = false; - self.next_index = 0; - } - (true, None) => { - // Otherwise, we didn't find a newline, so we'll discard - // everything we read. On the next iteration, we'll continue - // discarding up to max_len bytes unless we find a newline. - buf.advance(read_to); - self.next_index = 0; - if buf.is_empty() { - return Ok(None); - } - } - (false, Some(pos)) => { - // We found a correct frame - - let newpos_index = pos + self.next_index; - self.next_index = 0; - let frame = buf.split_to(newpos_index + 1); - - trace!( - message = "decoding the frame.", - bytes_proccesed = frame.len() - ); - - let frame = &frame[..frame.len() - 1]; - - return Ok(Some(frame.into())); - } - (false, None) if buf.len() > self.max_length => { - // We reached the max length without finding the - // delimiter so must discard the rest until we - // reach the next delimiter - self.is_discarding = true; - warn!( - message = "discarding frame larger than max_length", - buf_len = buf.len(), - max_length = self.max_length, - rate_limit_secs = 30 - ); - return Ok(None); - } - (false, None) => { - // We didn't find the delimiter and didn't - // reach the max frame length. - self.next_index = read_to; - return Ok(None); - } - } - } - } - - fn decode_eof(&mut self, buf: &mut BytesMut) -> Result, io::Error> { - let frame = match self.decode(buf)? { - Some(frame) => Some(frame), - None if !buf.is_empty() && !self.is_discarding => { - let frame = buf.take(); - self.next_index = 0; - - Some(frame.into()) - } - _ => None, - }; - - Ok(frame) - } -} - -impl Encoder for BytesDelimitedCodec { - type Item = Bytes; - type Error = io::Error; - - fn encode(&mut self, item: Bytes, buf: &mut BytesMut) -> Result<(), io::Error> { - buf.reserve(item.len() + 1); - buf.put(item); - buf.put_u8(self.delim); - Ok(()) - } -} diff --git a/lib/codec01/tests/bytes_delim.rs b/lib/codec01/tests/bytes_delim.rs deleted file mode 100644 index f38506e923019..0000000000000 --- a/lib/codec01/tests/bytes_delim.rs +++ /dev/null @@ -1,145 +0,0 @@ -use bytes::{BufMut, BytesMut}; -use codec01::BytesDelimitedCodec; -use std::collections::HashMap; -use tokio_codec::{Decoder, Encoder}; - -#[test] -fn bytes_delim_decod() { - let mut codec = BytesDelimitedCodec::new(b'\n'); - let buf = &mut BytesMut::new(); - buf.put_slice(b"abc\n"); - assert_eq!(Some("abc".into()), codec.decode(buf).unwrap()); -} - -#[test] -fn bytes_delim_encode() { - let mut codec = BytesDelimitedCodec::new(b'\n'); - - let mut buf = BytesMut::new(); - codec.encode("abc".into(), &mut buf).unwrap(); - - assert_eq!(b"abc\n", &buf[..]); -} - -#[test] -fn bytes_decode_max_length() { - const MAX_LENGTH: usize = 6; - - let mut codec = BytesDelimitedCodec::new_with_max_length(b'\n', MAX_LENGTH); - let buf = &mut BytesMut::new(); - - buf.reserve(200); - // limit is 6 so this should fail - buf.put_slice(b"1234567\n123456\n123412314\n123"); - - assert!(codec.decode(buf).unwrap().is_none()); - assert!(codec.decode(buf).unwrap().is_some()); - assert!(codec.decode_eof(buf).unwrap().is_none()); - assert!(codec.decode_eof(buf).unwrap().is_some()); -} - -// Regression test for [infinite loop bug](https://github.com/timberio/vector/issues/2564) -// Derived from https://github.com/tokio-rs/tokio/issues/1483 -#[test] -fn bytes_decoder_discard_repeat() { - const MAX_LENGTH: usize = 1; - - let mut codec = BytesDelimitedCodec::new_with_max_length(b'\n', MAX_LENGTH); - let buf = &mut BytesMut::new(); - - buf.reserve(200); - buf.put("aa"); - assert!(codec.decode(buf).unwrap().is_none()); - buf.put("a"); - assert!(codec.decode(buf).unwrap().is_none()); -} - -#[test] -fn bytes_decode_json_escaped() { - let mut input = HashMap::new(); - input.insert("key", "value"); - input.insert("new", "li\nne"); - - let mut bytes = serde_json::to_vec(&input).unwrap(); - bytes.push(b'\n'); - - let mut codec = BytesDelimitedCodec::new(b'\n'); - let buf = &mut BytesMut::new(); - - buf.reserve(bytes.len()); - buf.extend(bytes); - - let result = codec.decode(buf).unwrap(); - - assert!(result.is_some()); - assert!(buf.is_empty()); -} - -#[test] -fn bytes_decode_json_multiline() { - let events = r#" -{"log":"\u0009at org.springframework.security.web.context.SecurityContextPersistenceFilter.doFilter(SecurityContextPersistenceFilter.java:105)\n","stream":"stdout","time":"2019-01-18T07:49:27.374616758Z"} -{"log":"\u0009at org.springframework.security.web.FilterChainProxy$VirtualFilterChain.doFilter(FilterChainProxy.java:334)\n","stream":"stdout","time":"2019-01-18T07:49:27.374640288Z"} -{"log":"\u0009at org.springframework.security.web.context.request.async.WebAsyncManagerIntegrationFilter.doFilterInternal(WebAsyncManagerIntegrationFilter.java:56)\n","stream":"stdout","time":"2019-01-18T07:49:27.374655505Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.374671955Z"} -{"log":"\u0009at org.springframework.security.web.FilterChainProxy$VirtualFilterChain.doFilter(FilterChainProxy.java:334)\n","stream":"stdout","time":"2019-01-18T07:49:27.374690312Z"} -{"log":"\u0009at org.springframework.security.web.FilterChainProxy.doFilterInternal(FilterChainProxy.java:215)\n","stream":"stdout","time":"2019-01-18T07:49:27.374704522Z"} -{"log":"\u0009at org.springframework.security.web.FilterChainProxy.doFilter(FilterChainProxy.java:178)\n","stream":"stdout","time":"2019-01-18T07:49:27.374718459Z"} -{"log":"\u0009at org.springframework.web.filter.DelegatingFilterProxy.invokeDelegate(DelegatingFilterProxy.java:357)\n","stream":"stdout","time":"2019-01-18T07:49:27.374732919Z"} -{"log":"\u0009at org.springframework.web.filter.DelegatingFilterProxy.doFilter(DelegatingFilterProxy.java:270)\n","stream":"stdout","time":"2019-01-18T07:49:27.374750799Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.374764819Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.374778682Z"} -{"log":"\u0009at org.springframework.web.filter.RequestContextFilter.doFilterInternal(RequestContextFilter.java:99)\n","stream":"stdout","time":"2019-01-18T07:49:27.374792429Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.374805985Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.374819625Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.374833335Z"} -{"log":"\u0009at org.springframework.web.filter.HttpPutFormContentFilter.doFilterInternal(HttpPutFormContentFilter.java:109)\n","stream":"stdout","time":"2019-01-18T07:49:27.374847845Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.374861925Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.37487589Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.374890043Z"} -{"log":"\u0009at org.springframework.web.filter.HiddenHttpMethodFilter.doFilterInternal(HiddenHttpMethodFilter.java:93)\n","stream":"stdout","time":"2019-01-18T07:49:27.374903813Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.374917793Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.374931586Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.374946006Z"} -{"log":"\u0009at org.springframework.boot.actuate.metrics.web.servlet.WebMvcMetricsFilter.filterAndRecordMetrics(WebMvcMetricsFilter.java:117)\n","stream":"stdout","time":"2019-01-18T07:49:27.37496104Z"} -{"log":"\u0009at org.springframework.boot.actuate.metrics.web.servlet.WebMvcMetricsFilter.doFilterInternal(WebMvcMetricsFilter.java:106)\n","stream":"stdout","time":"2019-01-18T07:49:27.37498773Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.375003113Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.375017063Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.37503086Z"} -{"log":"\u0009at org.springframework.web.filter.CharacterEncodingFilter.doFilterInternal(CharacterEncodingFilter.java:200)\n","stream":"stdout","time":"2019-01-18T07:49:27.3750454Z"} -{"log":"\u0009at org.springframework.web.filter.OncePerRequestFilter.doFilter(OncePerRequestFilter.java:107)\n","stream":"stdout","time":"2019-01-18T07:49:27.37505928Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:193)\n","stream":"stdout","time":"2019-01-18T07:49:27.37507306Z"} -{"log":"\u0009at org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:166)\n","stream":"stdout","time":"2019-01-18T07:49:27.375086726Z"} -{"log":"\u0009at org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:198)\n","stream":"stdout","time":"2019-01-18T07:49:27.375100817Z"} -{"log":"\u0009at org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:96)\n","stream":"stdout","time":"2019-01-18T07:49:27.375115354Z"} -{"log":"\u0009at org.apache.catalina.authenticator.AuthenticatorBase.invoke(AuthenticatorBase.java:493)\n","stream":"stdout","time":"2019-01-18T07:49:27.375129454Z"} -{"log":"\u0009at org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:140)\n","stream":"stdout","time":"2019-01-18T07:49:27.375144001Z"} -{"log":"\u0009at org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:81)\n","stream":"stdout","time":"2019-01-18T07:49:27.375157464Z"} -{"log":"\u0009at org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:87)\n","stream":"stdout","time":"2019-01-18T07:49:27.375170981Z"} -{"log":"\u0009at org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:342)\n","stream":"stdout","time":"2019-01-18T07:49:27.375184417Z"} -{"log":"\u0009at org.apache.coyote.http11.Http11Processor.service(Http11Processor.java:800)\n","stream":"stdout","time":"2019-01-18T07:49:27.375198024Z"} -{"log":"\u0009at org.apache.coyote.AbstractProcessorLight.process(AbstractProcessorLight.java:66)\n","stream":"stdout","time":"2019-01-18T07:49:27.375211594Z"} -{"log":"\u0009at org.apache.coyote.AbstractProtocol$ConnectionHandler.process(AbstractProtocol.java:806)\n","stream":"stdout","time":"2019-01-18T07:49:27.375225237Z"} -{"log":"\u0009at org.apache.tomcat.util.net.NioEndpoint$SocketProcessor.doRun(NioEndpoint.java:1498)\n","stream":"stdout","time":"2019-01-18T07:49:27.375239487Z"} -{"log":"\u0009at org.apache.tomcat.util.net.SocketProcessorBase.run(SocketProcessorBase.java:49)\n","stream":"stdout","time":"2019-01-18T07:49:27.375253464Z"} -{"log":"\u0009at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n","stream":"stdout","time":"2019-01-18T07:49:27.375323255Z"} -{"log":"\u0009at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n","stream":"stdout","time":"2019-01-18T07:49:27.375345642Z"} -{"log":"\u0009at org.apache.tomcat.util.threads.TaskThread$WrappingRunnable.run(TaskThread.java:61)\n","stream":"stdout","time":"2019-01-18T07:49:27.375363208Z"} -{"log":"\u0009at java.lang.Thread.run(Thread.java:748)\n","stream":"stdout","time":"2019-01-18T07:49:27.375377695Z"} -{"log":"\n","stream":"stdout","time":"2019-01-18T07:49:27.375391335Z"} -{"log":"\n","stream":"stdout","time":"2019-01-18T07:49:27.375416915Z"} -{"log":"2019-01-18 07:53:06.419 [ ] INFO 1 --- [vent-bus.prod-1] c.t.listener.CommonListener : warehousing Dailywarehousing.daily\n","stream":"stdout","time":"2019-01-18T07:53:06.420527437Z"} -"#; - - let mut codec = BytesDelimitedCodec::new(b'\n'); - let buf = &mut BytesMut::new(); - - buf.extend(events.to_string().as_bytes()); - - let mut i = 0; - while codec.decode(buf).unwrap().is_some() { - i += 1; - } - - assert_eq!(i, 52); -} From b709dc70827ee017d246b9917bfa151367700dd4 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:43:18 +0300 Subject: [PATCH 17/29] add tokio-util/udp to features Signed-off-by: Kirill Fomichev --- Cargo.toml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index a5ebaea8faf85..aa30f22c41281 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,7 +49,7 @@ tokio-openssl = "0.4.0" tokio-retry = "0.2.0" tokio-signal = "0.2.7" tokio-compat = { version = "0.1", features = ["rt-full"] } -tokio-util = { version = "0.3.1", features = ["codec", "compat", "udp"] } +tokio-util = { version = "0.3.1", features = ["codec", "compat"] } async-trait = "0.1" # Tracing @@ -254,9 +254,9 @@ sources-journald = [] sources-kafka = [] sources-logplex = ["warp", "sources-tls"] sources-prometheus = [] -sources-socket = ["bytesize", "listenfd", "tokio/uds", "sources-tls"] +sources-socket = ["bytesize", "listenfd", "tokio/uds", "tokio-util/udp", "sources-tls"] sources-splunk_hec = ["bytesize", "warp", "sources-tls"] -sources-statsd = [] +sources-statsd = ["tokio-util/udp"] sources-stdin = ["bytesize"] sources-syslog = ["sources-socket", "syslog_loose"] sources-tls = ["sources-http", "sources-logplex", "sources-socket", "sources-splunk_hec"] @@ -375,7 +375,7 @@ sinks-sematext_logs = ["sinks-elasticsearch"] sinks-socket = ["tokio/uds"] sinks-papertrail = ["sinks-socket"] sinks-splunk_hec = ["bytesize"] -sinks-statsd = [] +sinks-statsd = ["tokio-util/udp"] sinks-vector = [] sinks-pulsar = ["pulsar"] From 3917d9594acea0e2820b51cd94fcc3f1ffff54da Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:54:48 +0300 Subject: [PATCH 18/29] sources/statsd: update codec Signed-off-by: Kirill Fomichev --- src/sources/socket/udp.rs | 6 ++-- src/sources/statsd/mod.rs | 66 +++++++++++++++++++++++---------------- 2 files changed, 41 insertions(+), 31 deletions(-) diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index 1348b14669744..0099cc87ecd98 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -11,7 +11,7 @@ use futures::{ }; use futures01::{sync::mpsc, Sink}; use serde::{Deserialize, Serialize}; -use std::{net::SocketAddr}; +use std::net::SocketAddr; use string_cache::DefaultAtom as Atom; use tokio::net::UdpSocket; use tokio_util::udp::UdpFramed; @@ -62,9 +62,7 @@ pub fn udp( .as_mut_log() .insert(event::log_schema().source_type_key(), "socket"); - event - .as_mut_log() - .insert(host_key, addr.to_string()); + event.as_mut_log().insert(host_key, addr.to_string()); emit!(UdpEventReceived { byte_size }); Some(Ok(event)) diff --git a/src/sources/statsd/mod.rs b/src/sources/statsd/mod.rs index e10868391bde9..111cae6377283 100644 --- a/src/sources/statsd/mod.rs +++ b/src/sources/statsd/mod.rs @@ -1,15 +1,16 @@ use crate::{ - shutdown::ShutdownSignal, stream::StreamExt01, topology::config::GlobalOptions, Event, + shutdown::ShutdownSignal, topology::config::GlobalOptions, Event, }; -use futures01::{future, sync::mpsc, Future, Sink, Stream}; +use futures::{ + compat::{Future01CompatExt, Sink01CompatExt}, stream, + FutureExt, StreamExt, TryFutureExt, +}; +use futures01::{sync::mpsc, Sink}; use parser::parse; use serde::{Deserialize, Serialize}; use std::net::SocketAddr; -use tokio01::{ - self, - codec::BytesCodec, - net::{UdpFramed, UdpSocket}, -}; +use tokio::net::UdpSocket; +use tokio_util::{codec::BytesCodec, udp::UdpFramed}; use tracing::field; pub mod parser; @@ -44,35 +45,46 @@ fn statsd(addr: SocketAddr, shutdown: ShutdownSignal, out: mpsc::Sender) let out = out.sink_map_err(|e| error!("error sending metric: {:?}", e)); Box::new( - future::lazy(move || { - let socket = UdpSocket::bind(&addr).expect("failed to bind to udp listener socket"); - + async move { + let socket = UdpSocket::bind(&addr) + .await + .expect("failed to bind to udp listener socket"); info!( message = "listening.", addr = &field::display(addr), r#type = "udp" ); - future::ok(socket) - }) - .and_then(move |socket| { - let metrics_in = UdpFramed::new(socket, BytesCodec::new()) - .take_until(shutdown) - .map(|(bytes, _sock)| { - let packet = String::from_utf8_lossy(bytes.as_ref()); - let metrics = packet - .lines() - .map(parse) - .filter_map(|res| res.map_err(|e| error!("{}", e)).ok()) - .map(Event::Metric) - .collect::>(); - futures01::stream::iter_ok::<_, std::io::Error>(metrics) + let _ = UdpFramed::new(socket, BytesCodec::new()) + .take_until(shutdown.compat()) + .filter_map(|frame| async move { + match frame { + Ok((bytes, _sock)) => { + let packet = String::from_utf8_lossy(bytes.as_ref()); + let metrics = packet + .lines() + .map(parse) + .filter_map(|res| res.map_err(|e| error!("{}", e)).ok()) + .map(Event::Metric) + .map(Ok) + .collect::>(); + Some(stream::iter(metrics)) + } + Err(error) => { + error!("error reading datagram: {:?}", error); + None + } + } }) .flatten() - .map_err(|e| error!("error reading datagram: {:?}", e)); + .forward(out.sink_compat()) + .await; - metrics_in.forward(out).map(|_| info!("finished sending")) - }), + info!("finished sending"); + Ok(()) + } + .boxed() + .compat(), ) } From 61071b11fb9b84eef82c6d369de7d73c86fc9c4e Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 21:57:26 +0300 Subject: [PATCH 19/29] remove unused tokio01 features: udp, codec Signed-off-by: Kirill Fomichev --- Cargo.lock | 28 ---------------------------- Cargo.toml | 2 +- src/sources/util/unix.rs | 2 +- 3 files changed, 2 insertions(+), 30 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3269b7f8735ef..8f4bc676054d3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4574,7 +4574,6 @@ dependencies = [ "futures 0.1.29", "mio", "num_cpus", - "tokio-codec", "tokio-current-thread", "tokio-executor", "tokio-io", @@ -4583,7 +4582,6 @@ dependencies = [ "tokio-tcp", "tokio-threadpool", "tokio-timer", - "tokio-udp", "tracing-core", ] @@ -4611,17 +4609,6 @@ dependencies = [ "winapi 0.3.8", ] -[[package]] -name = "tokio-codec" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "25b2998660ba0e70d18684de5d06b70b70a3a747469af9dea7618cc59e75976b" -dependencies = [ - "bytes 0.4.12", - "futures 0.1.29", - "tokio-io", -] - [[package]] name = "tokio-compat" version = "0.1.5" @@ -4833,21 +4820,6 @@ dependencies = [ "tokio 0.2.21", ] -[[package]] -name = "tokio-udp" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f02298505547f73e60f568359ef0d016d5acd6e830ab9bc7c4a5b3403440121b" -dependencies = [ - "bytes 0.4.12", - "futures 0.1.29", - "log 0.4.8", - "mio", - "tokio-codec", - "tokio-io", - "tokio-reactor", -] - [[package]] name = "tokio-util" version = "0.3.1" diff --git a/Cargo.toml b/Cargo.toml index aa30f22c41281..e7415b7cd6e72 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -42,7 +42,7 @@ tracing-limit = { path = "lib/tracing-limit" } # Tokio / Futures futures01 = { package = "futures", version = "0.1.25" } futures = { version = "0.3", default-features = false, features = ["compat", "io-compat"] } -tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "tcp", "rt-full", "experimental-tracing", "codec", "udp", "sync"], default-features = false } +tokio01 = { package = "tokio", version = "0.1.22", features = ["io", "tcp", "rt-full", "experimental-tracing", "sync"], default-features = false } tokio = { version = "0.2.13", features = ["blocking", "fs", "sync", "macros", "test-util", "rt-core", "io-std"] } tokio-openssl03 = { package = "tokio-openssl", version = "0.3.0" } tokio-openssl = "0.4.0" diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index fe9575da8d36e..e73ab7443a056 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -29,7 +29,7 @@ pub fn build_unix_source( ) -> Source where D: Decoder + Clone + Send + 'static, - E: From + std::fmt::Debug + std::fmt::Display, // change to std::io::Error ? internal_events too! + E: From + std::fmt::Debug + std::fmt::Display, { let out = out.sink_map_err(|e| error!("error sending line: {:?}", e)); From 4e3bff0137d8c3bd93af6392692e7df88d29ba86 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sat, 18 Jul 2020 22:09:16 +0300 Subject: [PATCH 20/29] cargo-fmt Signed-off-by: Kirill Fomichev --- src/sources/statsd/mod.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/sources/statsd/mod.rs b/src/sources/statsd/mod.rs index 111cae6377283..0d05160892da3 100644 --- a/src/sources/statsd/mod.rs +++ b/src/sources/statsd/mod.rs @@ -1,9 +1,7 @@ -use crate::{ - shutdown::ShutdownSignal, topology::config::GlobalOptions, Event, -}; +use crate::{shutdown::ShutdownSignal, topology::config::GlobalOptions, Event}; use futures::{ - compat::{Future01CompatExt, Sink01CompatExt}, stream, - FutureExt, StreamExt, TryFutureExt, + compat::{Future01CompatExt, Sink01CompatExt}, + stream, FutureExt, StreamExt, TryFutureExt, }; use futures01::{sync::mpsc, Sink}; use parser::parse; From 880f7d6d3117d20d148c4c644e00b3af07c66573 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sun, 19 Jul 2020 08:25:05 +0300 Subject: [PATCH 21/29] remove already enabled feature Signed-off-by: Kirill Fomichev --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index e7415b7cd6e72..3a1c0c8fe156f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -190,7 +190,7 @@ tower-test01 = { package = "tower-test", version = "0.1" } serde_yaml = "0.8.9" dirs = "2.0.2" tokio-test = "0.2" -tokio = { version = "0.2", features = ["test-util", "uds"] } +tokio = { version = "0.2", features = ["test-util"] } assert_cmd = "1.0" reqwest = { version = "0.10.6", features = ["json"] } From d9dc85360505e7e9182df82c712cebd837124e65 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sun, 19 Jul 2020 08:59:19 +0300 Subject: [PATCH 22/29] add shutdown to sources/util/unix Signed-off-by: Kirill Fomichev --- src/async_read.rs | 15 +++++++++------ src/sources/util/unix.rs | 24 ++++++++++++++---------- 2 files changed, 23 insertions(+), 16 deletions(-) diff --git a/src/async_read.rs b/src/async_read.rs index e3d655e0cf5dc..d072d8fd0fdeb 100644 --- a/src/async_read.rs +++ b/src/async_read.rs @@ -7,7 +7,7 @@ use std::{ }; use tokio::io::{AsyncRead, Result as IoResult}; -pub trait AsyncReadExt: AsyncRead { +pub trait VecAsyncReadExt: AsyncRead { /// Read data from this reader until the given future resolves. fn allow_read_until(self, until: F) -> AllowReadUntil where @@ -21,7 +21,7 @@ pub trait AsyncReadExt: AsyncRead { } } -impl AsyncReadExt for S where S: AsyncRead {} +impl VecAsyncReadExt for S where S: AsyncRead {} /// A AsyncRead combinator which reads from a reader until a future resolves. #[pin_project] @@ -33,6 +33,12 @@ pub struct AllowReadUntil { until: F, } +impl AllowReadUntil { + pub fn get_ref(&self) -> &S { + &self.reader + } +} + impl AsyncRead for AllowReadUntil where S: AsyncRead, @@ -41,10 +47,7 @@ where fn poll_read(self: Pin<&mut Self>, cx: &mut Context, buf: &mut [u8]) -> Poll> { let this = self.project(); match this.until.poll(cx) { - Poll::Ready(_) => { - // TODO: Need proper shutdown - Poll::Ready(Ok(0)) - } + Poll::Ready(_) => Poll::Ready(Ok(0)), Poll::Pending => this.reader.poll_read(cx, buf), } } diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index e73ab7443a056..0efb0f17261e6 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -1,11 +1,11 @@ use crate::{ - async_read::AsyncReadExt, emit, event::Event, internal_events::UnixSocketError, + async_read::VecAsyncReadExt, emit, event::Event, internal_events::UnixSocketError, shutdown::ShutdownSignal, sources::Source, }; use bytes05::Bytes; use futures::{ compat::{Future01CompatExt, Sink01CompatExt}, - future, FutureExt, StreamExt, TryFutureExt, + future, FutureExt, SinkExt, StreamExt, TryFutureExt, }; use futures01::{sync::mpsc, Sink}; use std::path::PathBuf; @@ -50,7 +50,6 @@ where let listen_path = listen_path.clone(); let host_key = host_key.clone(); - let shutdown = shutdown.clone(); let span = info_span!("connection"); let path = if let Ok(addr) = socket.peer_addr() { @@ -68,8 +67,8 @@ where let received_from: Option = path.map(|p| p.to_string_lossy().into_owned().into()); - let stream = socket.allow_read_until(shutdown.compat().map(|_| ())); - let stream = FramedRead::new(stream, decoder.clone()).filter_map(move |line| { + let stream = socket.allow_read_until(shutdown.clone().compat().map(|_| ())); + let mut stream = FramedRead::new(stream, decoder.clone()).filter_map(move |line| { future::ready(match line { Ok(line) => build_event(&host_key, received_from.clone(), &line).map(Ok), Err(error) => { @@ -82,12 +81,17 @@ where }) }); - let fut = stream - .forward(out.clone().sink_compat()) - .map(|_| info!("finished sending")) - .instrument(span); + let mut out = out.clone().sink_compat(); + tokio::spawn( + async move { + let _ = out.send_all(&mut stream).await; + info!("finished sending"); - tokio::spawn(fut); + let socket = stream.get_ref().get_ref().get_ref(); + let _ = socket.shutdown(std::net::Shutdown::Both); + } + .instrument(span), + ); } Ok(()) From d7ca86b1ae1e25a30f8bc2328b68290261e2866e Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sun, 19 Jul 2020 11:31:02 +0300 Subject: [PATCH 23/29] fix uds feature for tests Signed-off-by: Kirill Fomichev --- src/test_util.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test_util.rs b/src/test_util.rs index f4e51f699e211..5ca2a2e93bb61 100644 --- a/src/test_util.rs +++ b/src/test_util.rs @@ -386,7 +386,7 @@ impl CountReceiver { }) } - #[cfg(unix)] + #[cfg(all(feature = "tokio/uds", unix))] pub fn receive_lines_unix

(path: P) -> CountReceiver where P: AsRef + Send + 'static, From 99b88094cfbfe9075f601c8a2eecf2009de764d5 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Sun, 19 Jul 2020 13:05:40 +0300 Subject: [PATCH 24/29] fix uds feature again Signed-off-by: Kirill Fomichev --- src/sinks/util/unix.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sinks/util/unix.rs b/src/sinks/util/unix.rs index fe4bfb954e08b..0468d030e9e50 100644 --- a/src/sinks/util/unix.rs +++ b/src/sinks/util/unix.rs @@ -201,7 +201,7 @@ impl Sink for UnixSink { } } -#[cfg(test)] +#[cfg(all(feature = "tokio/uds", test))] mod tests { use super::*; use crate::test_util::{random_lines_with_stream, runtime, shutdown_on_idle, CountReceiver}; From 3cf9fb5fe6f05158b71cd5a6bc8a9128b48e9e4e Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Mon, 20 Jul 2020 15:24:38 +0300 Subject: [PATCH 25/29] fix udp socket source Signed-off-by: Kirill Fomichev --- src/sources/socket/udp.rs | 72 ++++++++++++++++++++++----------------- 1 file changed, 41 insertions(+), 31 deletions(-) diff --git a/src/sources/socket/udp.rs b/src/sources/socket/udp.rs index 0099cc87ecd98..3d33bbd4b7dff 100644 --- a/src/sources/socket/udp.rs +++ b/src/sources/socket/udp.rs @@ -4,17 +4,15 @@ use crate::{ shutdown::ShutdownSignal, sources::Source, }; +use bytes05::BytesMut; use codec::BytesDelimitedCodec; -use futures::{ - compat::{Future01CompatExt, Sink01CompatExt}, - FutureExt, StreamExt, TryFutureExt, -}; +use futures::{compat::Future01CompatExt, FutureExt, TryFutureExt}; use futures01::{sync::mpsc, Sink}; use serde::{Deserialize, Serialize}; use std::net::SocketAddr; use string_cache::DefaultAtom as Atom; use tokio::net::UdpSocket; -use tokio_util::udp::UdpFramed; +use tokio_util::codec::Decoder; /// UDP processes messages per packet, where messages are separated by newline. #[derive(Deserialize, Serialize, Debug, Clone)] @@ -39,45 +37,57 @@ pub fn udp( shutdown: ShutdownSignal, out: mpsc::Sender, ) -> Source { - let out = out.sink_map_err(|e| error!("error sending event: {:?}", e)); + let mut out = out.sink_map_err(|e| error!("error sending event: {:?}", e)); Box::new( async move { - let socket = UdpSocket::bind(&address) + let mut socket = UdpSocket::bind(&address) .await .expect("failed to bind to udp listener socket"); info!(message = "listening.", %address); - let _ = UdpFramed::new(socket, BytesDelimitedCodec::new(b'\n')) - .take_until(shutdown.compat()) - .filter_map(|frame| { - let host_key = host_key.clone(); - async move { - match frame { - Ok((line, addr)) => { - let byte_size = line.len(); - let mut event = Event::from(line); + let mut shutdown = shutdown.compat(); - event - .as_mut_log() - .insert(event::log_schema().source_type_key(), "socket"); + // TODO: need to be changed to max_length from config + // Issue: https://github.com/timberio/vector/issues/3117 + let max_length = 100 * 1024; + let mut buf = BytesMut::with_capacity(max_length); + loop { + buf.resize(max_length, 0); + tokio::select! { + recv = socket.recv_from(&mut buf) => { + let (byte_size, address) = recv.map_err(|error| { + emit!(UdpSocketError { error }); + })?; - event.as_mut_log().insert(host_key, addr.to_string()); + let mut payload = buf.split_to(byte_size); - emit!(UdpEventReceived { byte_size }); - Some(Ok(event)) - } - Err(error) => { - emit!(UdpSocketError { error }); - None + // UDP processes messages per payload, where messages are separated by newline + // and stretch to end of payload. + let mut decoder = BytesDelimitedCodec::new(b'\n'); + while let Ok(Some(line)) = decoder.decode_eof(&mut payload) { + let mut event = Event::from(line); + + event + .as_mut_log() + .insert(event::log_schema().source_type_key(), "socket"); + event + .as_mut_log() + .insert(host_key.clone(), address.to_string()); + + emit!(UdpEventReceived { byte_size }); + + tokio::select!{ + result = out.send(event).compat() => { + out = result?; + } + _ = &mut shutdown => return Ok(()), } } } - }) - .forward(out.sink_compat()) - .await; - - Ok(()) + _ = &mut shutdown => return Ok(()), + } + } } .boxed() .compat(), From 56d8122fb1f5294bff4d827c266928208bc72c72 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Mon, 20 Jul 2020 20:29:07 +0300 Subject: [PATCH 26/29] fix syslog test for unix socket Signed-off-by: Kirill Fomichev --- src/sinks/file/mod.rs | 2 +- tests/syslog.rs | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/sinks/file/mod.rs b/src/sinks/file/mod.rs index 4a7ce592ab87e..e46793b1f9d9f 100644 --- a/src/sinks/file/mod.rs +++ b/src/sinks/file/mod.rs @@ -120,7 +120,7 @@ impl FileSink { match event { None => { // If we got `None` - terminate the processing. - debug!(message = "Receiver exausted, terminating the processing loop."); + debug!(message = "Receiver exhausted, terminating the processing loop."); break; } Some(event) => self.process_event(event).await, diff --git a/tests/syslog.rs b/tests/syslog.rs index 902b1090c7a9e..bfa2bc45a397d 100644 --- a/tests/syslog.rs +++ b/tests/syslog.rs @@ -120,6 +120,9 @@ fn test_unix_stream_syslog() { let stream = sink.get_mut(); stream.shutdown(std::net::Shutdown::Both).unwrap(); + // Otherwise some lines will be lost + tokio::time::delay_for(std::time::Duration::from_millis(100)).await; + // Shut down server topology.stop().compat().await.unwrap(); From fc4c810af973769571875bb18ffc932322787d37 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Tue, 21 Jul 2020 13:00:11 +0300 Subject: [PATCH 27/29] remove not required delay_for in statsd Signed-off-by: Kirill Fomichev --- src/sinks/statsd.rs | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/sinks/statsd.rs b/src/sinks/statsd.rs index c4c351d779566..812f7dea86a5e 100644 --- a/src/sinks/statsd.rs +++ b/src/sinks/statsd.rs @@ -231,8 +231,7 @@ mod test { use futures::compat::{Future01CompatExt, Sink01CompatExt}; use futures::{SinkExt, StreamExt, TryStreamExt}; use futures01::{sync::mpsc, Sink}; - use std::time::Duration; - use tokio::{net::UdpSocket, time::delay_for}; + use tokio::{net::UdpSocket}; use tokio_util::{codec::BytesCodec, udp::UdpFramed}; #[cfg(feature = "sources-statsd")] use {crate::sources::statsd::parser::parse, std::str::from_utf8}; @@ -362,8 +361,8 @@ mod test { ]; let (tx, rx) = mpsc::channel(1); + let socket = UdpSocket::bind(default_address()).await.unwrap(); tokio::spawn(async move { - let socket = UdpSocket::bind(default_address()).await.unwrap(); UdpFramed::new(socket, BytesCodec::new()) .map_err(|e| error!("error reading line: {:?}", e)) .map_ok(|(bytes, _addr)| bytes.freeze()) @@ -375,10 +374,6 @@ mod test { .unwrap() }); - // Add a delay to the write side to let the read side - // poll for read interest. Otherwise, this could cause - // a race condition in noisy environments. - delay_for(Duration::from_millis(100)).await; let stream = stream::iter_ok(events); let _ = sink.send_all(stream).compat().await.unwrap(); From 4e5c05c8487e540c37761522c7f7f4e763d1ea58 Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Tue, 21 Jul 2020 13:12:10 +0300 Subject: [PATCH 28/29] add type annotations for long get_ref/get_mut chains Signed-off-by: Kirill Fomichev --- src/sinks/statsd.rs | 2 +- src/sinks/util/tcp.rs | 3 ++- src/sources/util/tcp.rs | 3 ++- src/sources/util/unix.rs | 4 ++-- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/sinks/statsd.rs b/src/sinks/statsd.rs index 812f7dea86a5e..2d28b0dfbced2 100644 --- a/src/sinks/statsd.rs +++ b/src/sinks/statsd.rs @@ -231,7 +231,7 @@ mod test { use futures::compat::{Future01CompatExt, Sink01CompatExt}; use futures::{SinkExt, StreamExt, TryStreamExt}; use futures01::{sync::mpsc, Sink}; - use tokio::{net::UdpSocket}; + use tokio::net::UdpSocket; use tokio_util::{codec::BytesCodec, udp::UdpFramed}; #[cfg(feature = "sources-statsd")] use {crate::sources::statsd::parser::parse, std::str::from_utf8}; diff --git a/src/sinks/util/tcp.rs b/src/sinks/util/tcp.rs index 9be321c905397..386bc451741d7 100644 --- a/src/sinks/util/tcp.rs +++ b/src/sinks/util/tcp.rs @@ -213,7 +213,8 @@ impl Sink for TcpSink { // // If this returns `WouldBlock` we know the connection is still // valid and the write will most likely succeed. - let stream = connection.get_mut().get_mut().get_mut().get_mut(); + let stream: &mut MaybeTlsStream = + connection.get_mut().get_mut().get_mut().get_mut(); match stream.read(&mut [0u8; 1]) { Err(error) if error.kind() != ErrorKind::WouldBlock => { emit!(TcpConnectionDisconnected { error }); diff --git a/src/sources/util/tcp.rs b/src/sources/util/tcp.rs index 5bb2a18d88474..3ccd7005a992c 100644 --- a/src/sources/util/tcp.rs +++ b/src/sources/util/tcp.rs @@ -176,7 +176,8 @@ fn handle_stream( debug!("Start gracefull shutdown"); // Close our write part of TCP socket to signal the other side // that it should stop writing and close the channel. - if let Some(socket) = reader.get_ref().get_ref().get_ref().get_ref().get_ref() { + let socket: Option<&TcpStream> = reader.get_ref().get_ref().get_ref().get_ref().get_ref(); + if let Some(socket) = socket { if let Err(error) = socket.shutdown(Shutdown::Write) { warn!(message = "Failed in signalling to the other side to close the TCP channel.",%error); } diff --git a/src/sources/util/unix.rs b/src/sources/util/unix.rs index 0efb0f17261e6..71aba9b46e9f6 100644 --- a/src/sources/util/unix.rs +++ b/src/sources/util/unix.rs @@ -9,7 +9,7 @@ use futures::{ }; use futures01::{sync::mpsc, Sink}; use std::path::PathBuf; -use tokio::net::UnixListener; +use tokio::net::{UnixListener, UnixStream}; use tokio_util::codec::{Decoder, FramedRead}; use tracing::field; use tracing_futures::Instrument; @@ -87,7 +87,7 @@ where let _ = out.send_all(&mut stream).await; info!("finished sending"); - let socket = stream.get_ref().get_ref().get_ref(); + let socket: &UnixStream = stream.get_ref().get_ref().get_ref(); let _ = socket.shutdown(std::net::Shutdown::Both); } .instrument(span), From 9c5708450da833faa66ce260b7e786c4b2e648dd Mon Sep 17 00:00:00 2001 From: Kirill Fomichev Date: Wed, 22 Jul 2020 10:56:38 +0300 Subject: [PATCH 29/29] add explicit type to sinks/util Signed-off-by: Kirill Fomichev --- src/sinks/util/tcp.rs | 2 +- src/sinks/util/unix.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sinks/util/tcp.rs b/src/sinks/util/tcp.rs index 386bc451741d7..9c659b550c604 100644 --- a/src/sinks/util/tcp.rs +++ b/src/sinks/util/tcp.rs @@ -167,7 +167,7 @@ impl TcpSink { TcpSinkState::Backoff(ref mut delay) => match delay.poll() { Ok(Async::NotReady) => return Ok(Async::NotReady), // Err can only occur if the tokio runtime has been shutdown or if more than 2^63 timers have been created - Err(_) => unreachable!(), + Err(()) => unreachable!(), Ok(Async::Ready(())) => { debug!(message = "disconnected."); TcpSinkState::Disconnected diff --git a/src/sinks/util/unix.rs b/src/sinks/util/unix.rs index 0468d030e9e50..6c6daa2b08c6e 100644 --- a/src/sinks/util/unix.rs +++ b/src/sinks/util/unix.rs @@ -125,7 +125,7 @@ impl UnixSink { }, UnixSinkState::Backoff(ref mut delay) => match delay.poll() { Ok(Async::NotReady) => return Ok(Async::NotReady), - Err(_) => unreachable!(), + Err(()) => unreachable!(), Ok(Async::Ready(())) => UnixSinkState::Disconnected, }, UnixSinkState::Disconnected => {