Add timeout to Outbound::bind_service (#436)

Closes #403.

When the Destination service does not return a result for a service, the proxy connection for that service will hang indefinitely waiting for a result from Destination. If, for example, the requested name doesn't exist, this means that the proxy will wait forever, rather than responding with an error.

I've added a timeout wrapping the service returned from `<Outbound as Recognize>::bind_service`. The timeout can be configured by setting the `CONDUIT_PROXY_BIND_TIMEOUT` environment variable, and defaults to 10 seconds (because that's the default value for [a similar configuration in Linkerd](https://linkerd.io/config/1.3.5/linkerd/index.html#router-parameters)).

Testing with @klingerf's reproduction from #403:
```
curl -sIH 'Host: httpbin.org' $(minikube service proxy-http --url)/get | head -n1
HTTP/1.1 500 Internal Server Error
```
proxy logs:
```rust
proxy-5698f79b66-8rczl conduit-proxy INFO conduit_proxy using controller at HostAndPort { host: Domain("proxy-api.conduit.svc.cluster.local"), port: 8086 }
proxy-5698f79b66-8rczl conduit-proxy INFO conduit_proxy routing on V4(127.0.0.1:4140)
proxy-5698f79b66-8rczl conduit-proxy INFO conduit_proxy proxying on V4(0.0.0.0:4143) to None
proxy-5698f79b66-8rczl conduit-proxy INFO conduit_proxy::transport::connect "controller-client", DNS resolved proxy-api.conduit.svc.cluster.local to 10.0.0.240
proxy-5698f79b66-8rczl conduit-proxy ERR! conduit_proxy::map_err turning service error into 500: Inner(Timeout(Duration { secs: 10, nanos: 0 }))
```
This commit is contained in:
Eliza Weisman 2018-02-26 10:18:35 -08:00 committed by GitHub
parent 6309741ae7
commit 694f691b71
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 97 additions and 31 deletions

View File

@ -49,6 +49,9 @@ pub struct Config {
/// Timeout after which to cancel telemetry reports. /// Timeout after which to cancel telemetry reports.
pub report_timeout: Duration, pub report_timeout: Duration,
/// Timeout after which to cancel binding a request.
pub bind_timeout: Duration,
pub pod_name: Option<String>, pub pod_name: Option<String>,
pub pod_namespace: Option<String>, pub pod_namespace: Option<String>,
pub pod_zone: Option<String>, pub pod_zone: Option<String>,
@ -139,6 +142,7 @@ pub const ENV_PUBLIC_LISTENER: &str = "CONDUIT_PROXY_PUBLIC_LISTENER";
pub const ENV_CONTROL_LISTENER: &str = "CONDUIT_PROXY_CONTROL_LISTENER"; pub const ENV_CONTROL_LISTENER: &str = "CONDUIT_PROXY_CONTROL_LISTENER";
const ENV_PRIVATE_CONNECT_TIMEOUT: &str = "CONDUIT_PROXY_PRIVATE_CONNECT_TIMEOUT"; const ENV_PRIVATE_CONNECT_TIMEOUT: &str = "CONDUIT_PROXY_PRIVATE_CONNECT_TIMEOUT";
const ENV_PUBLIC_CONNECT_TIMEOUT: &str = "CONDUIT_PROXY_PUBLIC_CONNECT_TIMEOUT"; const ENV_PUBLIC_CONNECT_TIMEOUT: &str = "CONDUIT_PROXY_PUBLIC_CONNECT_TIMEOUT";
pub const ENV_BIND_TIMEOUT: &str = "CONDUIT_PROXY_BIND_TIMEOUT";
const ENV_NODE_NAME: &str = "CONDUIT_PROXY_NODE_NAME"; const ENV_NODE_NAME: &str = "CONDUIT_PROXY_NODE_NAME";
const ENV_POD_NAME: &str = "CONDUIT_PROXY_POD_NAME"; const ENV_POD_NAME: &str = "CONDUIT_PROXY_POD_NAME";
@ -157,6 +161,7 @@ const DEFAULT_PRIVATE_LISTENER: &str = "tcp://127.0.0.1:4140";
const DEFAULT_PUBLIC_LISTENER: &str = "tcp://0.0.0.0:4143"; const DEFAULT_PUBLIC_LISTENER: &str = "tcp://0.0.0.0:4143";
const DEFAULT_CONTROL_LISTENER: &str = "tcp://0.0.0.0:4190"; const DEFAULT_CONTROL_LISTENER: &str = "tcp://0.0.0.0:4190";
const DEFAULT_PRIVATE_CONNECT_TIMEOUT_MS: u64 = 20; const DEFAULT_PRIVATE_CONNECT_TIMEOUT_MS: u64 = 20;
const DEFAULT_BIND_TIMEOUT_MS: u64 = 10_000; // ten seconds, as in Linkerd.
const DEFAULT_RESOLV_CONF: &str = "/etc/resolv.conf"; const DEFAULT_RESOLV_CONF: &str = "/etc/resolv.conf";
// ===== impl Config ===== // ===== impl Config =====
@ -174,6 +179,7 @@ impl<'a> TryFrom<&'a Strings> for Config {
let private_forward = parse(strings, ENV_PRIVATE_FORWARD, str::parse); let private_forward = parse(strings, ENV_PRIVATE_FORWARD, str::parse);
let public_connect_timeout = parse(strings, ENV_PUBLIC_CONNECT_TIMEOUT, parse_number); let public_connect_timeout = parse(strings, ENV_PUBLIC_CONNECT_TIMEOUT, parse_number);
let private_connect_timeout = parse(strings, ENV_PRIVATE_CONNECT_TIMEOUT, parse_number); let private_connect_timeout = parse(strings, ENV_PRIVATE_CONNECT_TIMEOUT, parse_number);
let bind_timeout = parse(strings, ENV_BIND_TIMEOUT, parse_number);
let resolv_conf_path = strings.get(ENV_RESOLV_CONF); let resolv_conf_path = strings.get(ENV_RESOLV_CONF);
let event_buffer_capacity = parse(strings, ENV_EVENT_BUFFER_CAPACITY, parse_number); let event_buffer_capacity = parse(strings, ENV_EVENT_BUFFER_CAPACITY, parse_number);
let metrics_flush_interval_secs = let metrics_flush_interval_secs =
@ -226,6 +232,8 @@ impl<'a> TryFrom<&'a Strings> for Config {
.unwrap_or(DEFAULT_METRICS_FLUSH_INTERVAL_SECS)), .unwrap_or(DEFAULT_METRICS_FLUSH_INTERVAL_SECS)),
report_timeout: report_timeout:
Duration::from_secs(report_timeout?.unwrap_or(DEFAULT_REPORT_TIMEOUT_SECS)), Duration::from_secs(report_timeout?.unwrap_or(DEFAULT_REPORT_TIMEOUT_SECS)),
bind_timeout:
Duration::from_millis(bind_timeout?.unwrap_or(DEFAULT_BIND_TIMEOUT_MS)),
pod_name: pod_name?, pod_name: pod_name?,
pod_namespace: pod_namespace?, pod_namespace: pod_namespace?,
pod_zone: pod_zone?, pod_zone: pod_zone?,

View File

@ -223,7 +223,9 @@ where
bind, bind,
control, control,
config.default_destination_namespace().cloned(), config.default_destination_namespace().cloned(),
config.default_destination_zone().cloned()); config.default_destination_zone().cloned(),
config.bind_timeout,
);
let fut = serve( let fut = serve(
outbound_listener, outbound_listener,

View File

@ -1,4 +1,5 @@
use std::net::SocketAddr; use std::net::SocketAddr;
use std::time::Duration;
use futures::{Async, Poll}; use futures::{Async, Poll};
use http; use http;
@ -17,6 +18,7 @@ use control::{self, discovery};
use control::discovery::Bind as BindTrait; use control::discovery::Bind as BindTrait;
use ctx; use ctx;
use fully_qualified_authority::FullyQualifiedAuthority; use fully_qualified_authority::FullyQualifiedAuthority;
use timeout::Timeout;
type BindProtocol<B> = bind::BindProtocol<Arc<ctx::Proxy>, B>; type BindProtocol<B> = bind::BindProtocol<Arc<ctx::Proxy>, B>;
@ -25,6 +27,7 @@ pub struct Outbound<B> {
discovery: control::Control, discovery: control::Control,
default_namespace: Option<String>, default_namespace: Option<String>,
default_zone: Option<String>, default_zone: Option<String>,
bind_timeout: Duration,
} }
const MAX_IN_FLIGHT: usize = 10_000; const MAX_IN_FLIGHT: usize = 10_000;
@ -32,14 +35,18 @@ const MAX_IN_FLIGHT: usize = 10_000;
// ===== impl Outbound ===== // ===== impl Outbound =====
impl<B> Outbound<B> { impl<B> Outbound<B> {
pub fn new(bind: Bind<Arc<ctx::Proxy>, B>, discovery: control::Control, pub fn new(bind: Bind<Arc<ctx::Proxy>, B>,
default_namespace: Option<String>, default_zone: Option<String>) discovery: control::Control,
default_namespace: Option<String>,
default_zone: Option<String>,
bind_timeout: Duration,)
-> Outbound<B> { -> Outbound<B> {
Self { Self {
bind, bind,
discovery, discovery,
default_namespace, default_namespace,
default_zone, default_zone,
bind_timeout,
} }
} }
} }
@ -59,10 +66,10 @@ where
type Error = <Self::Service as tower::Service>::Error; type Error = <Self::Service as tower::Service>::Error;
type Key = (Destination, Protocol); type Key = (Destination, Protocol);
type RouteError = (); type RouteError = ();
type Service = InFlightLimit<Buffer<Balance< type Service = InFlightLimit<Timeout<Buffer<Balance<
load::WithPendingRequests<Discovery<B>>, load::WithPendingRequests<Discovery<B>>,
choose::PowerOfTwoChoices<rand::ThreadRng>, choose::PowerOfTwoChoices<rand::ThreadRng>
>>>; >>>>;
fn recognize(&self, req: &Self::Request) -> Option<Self::Key> { fn recognize(&self, req: &Self::Request) -> Option<Self::Key> {
let local = req.uri().authority_part().and_then(|authority| { let local = req.uri().authority_part().and_then(|authority| {
@ -131,11 +138,16 @@ where
let balance = tower_balance::power_of_two_choices(loaded, rand::thread_rng()); let balance = tower_balance::power_of_two_choices(loaded, rand::thread_rng());
Buffer::new(balance, self.bind.executor()) // use the same executor as the underlying `Bind` for the `Buffer` and
.map(|buffer| { // `Timeout`.
InFlightLimit::new(buffer, MAX_IN_FLIGHT) let handle = self.bind.executor();
})
.map_err(|_| {}) let buffer = Buffer::new(balance, handle).map_err(|_| {})?;
let timeout = Timeout::new(buffer, self.bind_timeout, handle);
Ok(InFlightLimit::new(timeout, MAX_IN_FLIGHT))
} }
} }

View File

@ -11,7 +11,6 @@ use tokio_core::reactor::{Timeout as ReactorTimeout, Handle};
use tokio_io; use tokio_io;
use tower::Service; use tower::Service;
/// A timeout that wraps an underlying operation. /// A timeout that wraps an underlying operation.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub struct Timeout<U> { pub struct Timeout<U> {
@ -53,7 +52,6 @@ impl<U> Timeout<U> {
impl<S, T, E> Service for Timeout<S> impl<S, T, E> Service for Timeout<S>
where where
S: Service<Response=T, Error=E>, S: Service<Response=T, Error=E>,
// E: Error,
{ {
type Request = S::Request; type Request = S::Request;
type Response = T; type Response = T;
@ -66,9 +64,8 @@ where
fn call(&mut self, req: Self::Request) -> Self::Future { fn call(&mut self, req: Self::Request) -> Self::Future {
let duration = self.duration; let duration = self.duration;
// TODO: should this panic or wrap the error?
let timeout = ReactorTimeout::new(duration, &self.handle) let timeout = ReactorTimeout::new(duration, &self.handle)
.expect("failed to create timeout!"); .expect("reactor gone");
let inner = self.inner.call(req); let inner = self.inner.call(req);
TimeoutFuture { TimeoutFuture {
inner, inner,
@ -82,7 +79,6 @@ where
impl<C> Connect for Timeout<C> impl<C> Connect for Timeout<C>
where where
C: Connect, C: Connect,
// C::Error: Error,
{ {
type Connected = C::Connected; type Connected = C::Connected;
type Error = TimeoutError<C::Error>; type Error = TimeoutError<C::Error>;
@ -90,9 +86,8 @@ where
fn connect(&self) -> Self::Future { fn connect(&self) -> Self::Future {
let duration = self.duration; let duration = self.duration;
// TODO: should this panic or wrap the error?
let timeout = ReactorTimeout::new(duration, &self.handle) let timeout = ReactorTimeout::new(duration, &self.handle)
.expect("failed to create timeout!"); .expect("reactor gone");
let inner = self.inner.connect(); let inner = self.inner.connect();
TimeoutFuture { TimeoutFuture {
inner, inner,

View File

@ -1,6 +1,8 @@
mod support; mod support;
use self::support::*; use self::support::*;
use std::thread;
#[test] #[test]
fn outbound_asks_controller_api() { fn outbound_asks_controller_api() {
let _ = env_logger::init(); let _ = env_logger::init();
@ -55,10 +57,36 @@ fn outbound_updates_newer_services() {
} }
#[test] #[test]
#[ignore] #[cfg_attr(not(feature = "flaky_tests"), ignore)]
fn outbound_times_out() { fn outbound_times_out() {
// Currently, the outbound router will wait forever until discovery tells let _ = env_logger::init();
// it where to send the request. It should probably time out eventually. let mut env = config::TestEnv::new();
// set the bind timeout to 100 ms.
env.put(config::ENV_BIND_TIMEOUT, "100".to_owned());
let srv = server::http1().route("/h1", "hello h1").run();
let addr = srv.addr.clone();
let ctrl = controller::new()
// when the proxy requests the destination, sleep for 500 ms, and then
// return the correct destination
.destination_fn("disco.test.svc.cluster.local", move || {
thread::sleep(Duration::from_millis(500));
Some(controller::destination_update(addr))
})
.run();
let proxy = proxy::new()
.controller(ctrl)
.outbound(srv)
.run_with_test_env(env);
let client = client::http2(proxy.outbound, "disco.test.svc.cluster.local");
let mut req = client.request_builder("/");
let rsp = client.request(req.method("GET"));
// the request should time out
assert_eq!(rsp.status(), http::StatusCode::INTERNAL_SERVER_ERROR);
} }
#[test] #[test]

View File

@ -2,6 +2,7 @@
use support::*; use support::*;
use std::fmt;
use std::net::IpAddr; use std::net::IpAddr;
use std::sync::{Arc, Mutex}; use std::sync::{Arc, Mutex};
@ -14,9 +15,11 @@ pub fn new() -> Controller {
Controller::new() Controller::new()
} }
struct Destination(Box<Fn() -> Option<pb::destination::Update> + Send>);
#[derive(Debug)] #[derive(Debug)]
pub struct Controller { pub struct Controller {
destinations: Vec<(String, Option<pb::destination::Update>)>, destinations: Vec<(String, Destination)>,
reports: Option<mpsc::UnboundedSender<pb::telemetry::ReportRequest>>, reports: Option<mpsc::UnboundedSender<pb::telemetry::ReportRequest>>,
} }
@ -35,14 +38,21 @@ impl Controller {
} }
pub fn destination(mut self, dest: &str, addr: SocketAddr) -> Self { pub fn destination(mut self, dest: &str, addr: SocketAddr) -> Self {
self.destination_fn(dest, move || Some(destination_update(addr)))
}
pub fn destination_fn<F>(mut self, dest: &str, f: F) -> Self
where
F: Fn() -> Option<pb::destination::Update> + Send + 'static,
{
self.destinations self.destinations
.push((dest.into(), Some(destination_update(addr)))); .push((dest.into(), Destination(Box::new(f))));
self self
} }
pub fn destination_close(mut self, dest: &str) -> Self { pub fn destination_close(mut self, dest: &str) -> Self {
self.destinations.push((dest.into(), None)); self.destination_fn(dest, || None)
self
} }
pub fn reports(&mut self) -> mpsc::UnboundedReceiver<pb::telemetry::ReportRequest> { pub fn reports(&mut self) -> mpsc::UnboundedReceiver<pb::telemetry::ReportRequest> {
@ -57,11 +67,17 @@ impl Controller {
} }
type Response = self::http::Response<GrpcBody>; type Response = self::http::Response<GrpcBody>;
type Destinations = Arc<Mutex<Vec<(String, Option<pb::destination::Update>)>>>; type Destinations = Arc<Mutex<Vec<(String, Destination)>>>;
const DESTINATION_GET: &str = "/conduit.proxy.destination.Destination/Get"; const DESTINATION_GET: &str = "/conduit.proxy.destination.Destination/Get";
const TELEMETRY_REPORT: &str = "/conduit.proxy.telemetry.Telemetry/Report"; const TELEMETRY_REPORT: &str = "/conduit.proxy.telemetry.Telemetry/Report";
impl fmt::Debug for Destination {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "Destination")
}
}
#[derive(Debug)] #[derive(Debug)]
struct Svc { struct Svc {
destinations: Destinations, destinations: Destinations,
@ -85,7 +101,8 @@ impl Svc {
let mut vec = destinations.lock().unwrap(); let mut vec = destinations.lock().unwrap();
//TODO: decode `_bytes` and compare with `.0` //TODO: decode `_bytes` and compare with `.0`
if !vec.is_empty() { if !vec.is_empty() {
vec.remove(0).1 let Destination(f) = vec.remove(0).1;
f()
} else { } else {
None None
} }
@ -249,7 +266,7 @@ fn run(controller: Controller) -> Listening {
} }
} }
fn destination_update(addr: SocketAddr) -> pb::destination::Update { pub fn destination_update(addr: SocketAddr) -> pb::destination::Update {
pb::destination::Update { pb::destination::Update {
update: Some(pb::destination::update::Update::Add( update: Some(pb::destination::update::Update::Add(
pb::destination::WeightedAddrSet { pb::destination::WeightedAddrSet {

View File

@ -58,7 +58,11 @@ impl Proxy {
} }
pub fn run(self) -> Listening { pub fn run(self) -> Listening {
run(self) self.run_with_test_env(config::TestEnv::new())
}
pub fn run_with_test_env(self, mut env: config::TestEnv) -> Listening {
run(self, env)
} }
} }
@ -90,7 +94,8 @@ impl conduit_proxy::GetOriginalDst for MockOriginalDst {
} }
} }
fn run(proxy: Proxy) -> Listening {
fn run(proxy: Proxy, mut env: config::TestEnv) -> Listening {
use self::conduit_proxy::config; use self::conduit_proxy::config;
let controller = proxy.controller.expect("proxy controller missing"); let controller = proxy.controller.expect("proxy controller missing");
@ -98,7 +103,6 @@ fn run(proxy: Proxy) -> Listening {
let outbound = proxy.outbound; let outbound = proxy.outbound;
let mut mock_orig_dst = DstInner::default(); let mut mock_orig_dst = DstInner::default();
let mut env = config::TestEnv::new();
env.put(config::ENV_CONTROL_URL, format!("tcp://{}", controller.addr)); env.put(config::ENV_CONTROL_URL, format!("tcp://{}", controller.addr));
env.put(config::ENV_PRIVATE_LISTENER, "tcp://127.0.0.1:0".to_owned()); env.put(config::ENV_PRIVATE_LISTENER, "tcp://127.0.0.1:0".to_owned());
if let Some(ref inbound) = inbound { if let Some(ref inbound) = inbound {