mirror of https://github.com/linkerd/linkerd2.git
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:
parent
6309741ae7
commit
694f691b71
|
@ -49,6 +49,9 @@ pub struct Config {
|
|||
/// Timeout after which to cancel telemetry reports.
|
||||
pub report_timeout: Duration,
|
||||
|
||||
/// Timeout after which to cancel binding a request.
|
||||
pub bind_timeout: Duration,
|
||||
|
||||
pub pod_name: Option<String>,
|
||||
pub pod_namespace: 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";
|
||||
const ENV_PRIVATE_CONNECT_TIMEOUT: &str = "CONDUIT_PROXY_PRIVATE_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_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_CONTROL_LISTENER: &str = "tcp://0.0.0.0:4190";
|
||||
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";
|
||||
|
||||
// ===== impl Config =====
|
||||
|
@ -174,6 +179,7 @@ impl<'a> TryFrom<&'a Strings> for Config {
|
|||
let private_forward = parse(strings, ENV_PRIVATE_FORWARD, str::parse);
|
||||
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 bind_timeout = parse(strings, ENV_BIND_TIMEOUT, parse_number);
|
||||
let resolv_conf_path = strings.get(ENV_RESOLV_CONF);
|
||||
let event_buffer_capacity = parse(strings, ENV_EVENT_BUFFER_CAPACITY, parse_number);
|
||||
let metrics_flush_interval_secs =
|
||||
|
@ -226,6 +232,8 @@ impl<'a> TryFrom<&'a Strings> for Config {
|
|||
.unwrap_or(DEFAULT_METRICS_FLUSH_INTERVAL_SECS)),
|
||||
report_timeout:
|
||||
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_namespace: pod_namespace?,
|
||||
pod_zone: pod_zone?,
|
||||
|
|
|
@ -223,7 +223,9 @@ where
|
|||
bind,
|
||||
control,
|
||||
config.default_destination_namespace().cloned(),
|
||||
config.default_destination_zone().cloned());
|
||||
config.default_destination_zone().cloned(),
|
||||
config.bind_timeout,
|
||||
);
|
||||
|
||||
let fut = serve(
|
||||
outbound_listener,
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
use std::net::SocketAddr;
|
||||
use std::time::Duration;
|
||||
|
||||
use futures::{Async, Poll};
|
||||
use http;
|
||||
|
@ -17,6 +18,7 @@ use control::{self, discovery};
|
|||
use control::discovery::Bind as BindTrait;
|
||||
use ctx;
|
||||
use fully_qualified_authority::FullyQualifiedAuthority;
|
||||
use timeout::Timeout;
|
||||
|
||||
type BindProtocol<B> = bind::BindProtocol<Arc<ctx::Proxy>, B>;
|
||||
|
||||
|
@ -25,6 +27,7 @@ pub struct Outbound<B> {
|
|||
discovery: control::Control,
|
||||
default_namespace: Option<String>,
|
||||
default_zone: Option<String>,
|
||||
bind_timeout: Duration,
|
||||
}
|
||||
|
||||
const MAX_IN_FLIGHT: usize = 10_000;
|
||||
|
@ -32,14 +35,18 @@ const MAX_IN_FLIGHT: usize = 10_000;
|
|||
// ===== impl Outbound =====
|
||||
|
||||
impl<B> Outbound<B> {
|
||||
pub fn new(bind: Bind<Arc<ctx::Proxy>, B>, discovery: control::Control,
|
||||
default_namespace: Option<String>, default_zone: Option<String>)
|
||||
pub fn new(bind: Bind<Arc<ctx::Proxy>, B>,
|
||||
discovery: control::Control,
|
||||
default_namespace: Option<String>,
|
||||
default_zone: Option<String>,
|
||||
bind_timeout: Duration,)
|
||||
-> Outbound<B> {
|
||||
Self {
|
||||
bind,
|
||||
discovery,
|
||||
default_namespace,
|
||||
default_zone,
|
||||
bind_timeout,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -59,10 +66,10 @@ where
|
|||
type Error = <Self::Service as tower::Service>::Error;
|
||||
type Key = (Destination, Protocol);
|
||||
type RouteError = ();
|
||||
type Service = InFlightLimit<Buffer<Balance<
|
||||
type Service = InFlightLimit<Timeout<Buffer<Balance<
|
||||
load::WithPendingRequests<Discovery<B>>,
|
||||
choose::PowerOfTwoChoices<rand::ThreadRng>,
|
||||
>>>;
|
||||
choose::PowerOfTwoChoices<rand::ThreadRng>
|
||||
>>>>;
|
||||
|
||||
fn recognize(&self, req: &Self::Request) -> Option<Self::Key> {
|
||||
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());
|
||||
|
||||
Buffer::new(balance, self.bind.executor())
|
||||
.map(|buffer| {
|
||||
InFlightLimit::new(buffer, MAX_IN_FLIGHT)
|
||||
})
|
||||
.map_err(|_| {})
|
||||
// use the same executor as the underlying `Bind` for the `Buffer` and
|
||||
// `Timeout`.
|
||||
let handle = self.bind.executor();
|
||||
|
||||
let buffer = Buffer::new(balance, handle).map_err(|_| {})?;
|
||||
|
||||
let timeout = Timeout::new(buffer, self.bind_timeout, handle);
|
||||
|
||||
Ok(InFlightLimit::new(timeout, MAX_IN_FLIGHT))
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -11,7 +11,6 @@ use tokio_core::reactor::{Timeout as ReactorTimeout, Handle};
|
|||
use tokio_io;
|
||||
use tower::Service;
|
||||
|
||||
|
||||
/// A timeout that wraps an underlying operation.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct Timeout<U> {
|
||||
|
@ -53,7 +52,6 @@ impl<U> Timeout<U> {
|
|||
impl<S, T, E> Service for Timeout<S>
|
||||
where
|
||||
S: Service<Response=T, Error=E>,
|
||||
// E: Error,
|
||||
{
|
||||
type Request = S::Request;
|
||||
type Response = T;
|
||||
|
@ -66,9 +64,8 @@ where
|
|||
|
||||
fn call(&mut self, req: Self::Request) -> Self::Future {
|
||||
let duration = self.duration;
|
||||
// TODO: should this panic or wrap the error?
|
||||
let timeout = ReactorTimeout::new(duration, &self.handle)
|
||||
.expect("failed to create timeout!");
|
||||
.expect("reactor gone");
|
||||
let inner = self.inner.call(req);
|
||||
TimeoutFuture {
|
||||
inner,
|
||||
|
@ -82,7 +79,6 @@ where
|
|||
impl<C> Connect for Timeout<C>
|
||||
where
|
||||
C: Connect,
|
||||
// C::Error: Error,
|
||||
{
|
||||
type Connected = C::Connected;
|
||||
type Error = TimeoutError<C::Error>;
|
||||
|
@ -90,9 +86,8 @@ where
|
|||
|
||||
fn connect(&self) -> Self::Future {
|
||||
let duration = self.duration;
|
||||
// TODO: should this panic or wrap the error?
|
||||
let timeout = ReactorTimeout::new(duration, &self.handle)
|
||||
.expect("failed to create timeout!");
|
||||
.expect("reactor gone");
|
||||
let inner = self.inner.connect();
|
||||
TimeoutFuture {
|
||||
inner,
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
mod support;
|
||||
use self::support::*;
|
||||
|
||||
use std::thread;
|
||||
|
||||
#[test]
|
||||
fn outbound_asks_controller_api() {
|
||||
let _ = env_logger::init();
|
||||
|
@ -55,10 +57,36 @@ fn outbound_updates_newer_services() {
|
|||
}
|
||||
|
||||
#[test]
|
||||
#[ignore]
|
||||
#[cfg_attr(not(feature = "flaky_tests"), ignore)]
|
||||
fn outbound_times_out() {
|
||||
// Currently, the outbound router will wait forever until discovery tells
|
||||
// it where to send the request. It should probably time out eventually.
|
||||
let _ = env_logger::init();
|
||||
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]
|
||||
|
|
|
@ -2,6 +2,7 @@
|
|||
|
||||
use support::*;
|
||||
|
||||
use std::fmt;
|
||||
use std::net::IpAddr;
|
||||
use std::sync::{Arc, Mutex};
|
||||
|
||||
|
@ -14,9 +15,11 @@ pub fn new() -> Controller {
|
|||
Controller::new()
|
||||
}
|
||||
|
||||
struct Destination(Box<Fn() -> Option<pb::destination::Update> + Send>);
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct Controller {
|
||||
destinations: Vec<(String, Option<pb::destination::Update>)>,
|
||||
destinations: Vec<(String, Destination)>,
|
||||
reports: Option<mpsc::UnboundedSender<pb::telemetry::ReportRequest>>,
|
||||
}
|
||||
|
||||
|
@ -35,14 +38,21 @@ impl Controller {
|
|||
}
|
||||
|
||||
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
|
||||
.push((dest.into(), Some(destination_update(addr))));
|
||||
.push((dest.into(), Destination(Box::new(f))));
|
||||
self
|
||||
}
|
||||
|
||||
|
||||
pub fn destination_close(mut self, dest: &str) -> Self {
|
||||
self.destinations.push((dest.into(), None));
|
||||
self
|
||||
self.destination_fn(dest, || None)
|
||||
}
|
||||
|
||||
pub fn reports(&mut self) -> mpsc::UnboundedReceiver<pb::telemetry::ReportRequest> {
|
||||
|
@ -57,11 +67,17 @@ impl Controller {
|
|||
}
|
||||
|
||||
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 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)]
|
||||
struct Svc {
|
||||
destinations: Destinations,
|
||||
|
@ -85,7 +101,8 @@ impl Svc {
|
|||
let mut vec = destinations.lock().unwrap();
|
||||
//TODO: decode `_bytes` and compare with `.0`
|
||||
if !vec.is_empty() {
|
||||
vec.remove(0).1
|
||||
let Destination(f) = vec.remove(0).1;
|
||||
f()
|
||||
} else {
|
||||
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 {
|
||||
update: Some(pb::destination::update::Update::Add(
|
||||
pb::destination::WeightedAddrSet {
|
||||
|
|
|
@ -58,7 +58,11 @@ impl Proxy {
|
|||
}
|
||||
|
||||
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;
|
||||
|
||||
let controller = proxy.controller.expect("proxy controller missing");
|
||||
|
@ -98,7 +103,6 @@ fn run(proxy: Proxy) -> Listening {
|
|||
let outbound = proxy.outbound;
|
||||
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_PRIVATE_LISTENER, "tcp://127.0.0.1:0".to_owned());
|
||||
if let Some(ref inbound) = inbound {
|
||||
|
|
Loading…
Reference in New Issue