Switch Service request to a generic. (#109)

This changes the Service request type to a generic instead of an associated
type. This is more appropriate as requests are inputs to the service.

This change enables a single implementation of `Service` to accept many
kinds of request types. This also enables requests to be references.

Fixes #99
This commit is contained in:
Carl Lerche 2018-11-01 12:28:10 -07:00 committed by GitHub
parent b95c8d1030
commit 75eecc476c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 517 additions and 578 deletions

View File

@ -100,16 +100,16 @@ fn run<D, C>(
executor: &runtime::TaskExecutor,
) -> impl Future<Item = (), Error = ()>
where
D: Discover<Request = Req, Response = Rsp> + Send + 'static,
D: Discover + Send + 'static,
D::Key: Send,
D::Service: Send,
D::Service: Service<Req, Response = Rsp> + Send,
D::Error: Send,
D::DiscoverError: Send,
<D::Service as Service>::Future: Send,
<D::Service as Service<Req>>::Future: Send,
C: lb::Choose<D::Key, D::Service> + Send + 'static,
{
println!("{}", name);
let t0 = Instant::now();
compute_histo(SendRequests::new(lb, REQUESTS, CONCURRENCY, executor))
.map(move |h| report(&h, t0.elapsed()))
.map_err(|_| {})
@ -180,8 +180,7 @@ struct Rsp {
latency: Duration,
}
impl Service for DelayService {
type Request = Req;
impl Service<Req> for DelayService {
type Response = Rsp;
type Error = timer::Error;
type Future = Delay;
@ -217,13 +216,10 @@ impl Future for Delay {
impl Discover for Disco {
type Key = usize;
type Request = Req;
type Response = Rsp;
type Error = tower_in_flight_limit::Error<tower_buffer::Error<timer::Error>>;
type Service = InFlightLimit<Buffer<DelayService>>;
type DiscoverError = ();
type Error = ();
type Service = InFlightLimit<Buffer<DelayService, Req>>;
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::DiscoverError> {
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::Error> {
match self.changes.pop_front() {
Some(Change::Insert(k, svc)) => {
let svc = Buffer::new(svc, &self.executor).unwrap();
@ -236,26 +232,31 @@ impl Discover for Disco {
}
}
type DemoService<D, C> =
InFlightLimit<
Buffer<
lb::Balance<D, C>,
Req>>;
struct SendRequests<D, C>
where
D: Discover<Request = Req, Response = Rsp>,
D: Discover,
D::Service: Service<Req>,
C: lb::Choose<D::Key, D::Service>,
{
send_remaining: usize,
lb: InFlightLimit<Buffer<lb::Balance<D, C>>>,
responses: stream::FuturesUnordered<
tower_in_flight_limit::ResponseFuture<tower_buffer::ResponseFuture<lb::Balance<D, C>>>,
>,
lb: DemoService<D, C>,
responses: stream::FuturesUnordered<<DemoService<D, C> as Service<Req>>::Future>,
}
impl<D, C> SendRequests<D, C>
where
D: Discover<Request = Req, Response = Rsp> + Send + 'static,
D: Discover + Send + 'static,
D::Service: Service<Req>,
D::Key: Send,
D::Service: Send,
D::Error: Send,
D::DiscoverError: Send,
<D::Service as Service>::Future: Send,
<D::Service as Service<Req>>::Future: Send,
C: lb::Choose<D::Key, D::Service> + Send + 'static,
{
pub fn new(
@ -274,12 +275,12 @@ where
impl<D, C> Stream for SendRequests<D, C>
where
D: Discover<Request = Req, Response = Rsp>,
D: Discover,
D::Service: Service<Req>,
C: lb::Choose<D::Key, D::Service>,
{
type Item = Rsp;
type Error =
tower_in_flight_limit::Error<tower_buffer::Error<<lb::Balance<D, C> as Service>::Error>>;
type Item = <DemoService<D, C> as Service<Req>>::Response;
type Error = <DemoService<D, C> as Service<Req>>::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
debug!(

View File

@ -156,8 +156,10 @@ where
/// Polls `discover` for updates, adding new items to `not_ready`.
///
/// Removals may alter the order of either `ready` or `not_ready`.
fn update_from_discover(&mut self)
-> Result<(), Error<<D::Service as Service>::Error, D::DiscoverError>>
fn update_from_discover<Request>(&mut self)
-> Result<(), Error<<D::Service as Service<Request>>::Error, D::Error>>
where
D::Service: Service<Request>
{
debug!("updating from discover");
use tower_discover::Change::*;
@ -191,8 +193,10 @@ where
///
/// When `poll_ready` returns ready, the service is removed from `not_ready` and inserted
/// into `ready`, potentially altering the order of `ready` and/or `not_ready`.
fn promote_to_ready(&mut self)
-> Result<(), Error<<D::Service as Service>::Error, D::DiscoverError>>
fn promote_to_ready<Request>(&mut self)
-> Result<(), Error<<D::Service as Service<Request>>::Error, D::Error>>
where
D::Service: Service<Request>,
{
let n = self.not_ready.len();
if n == 0 {
@ -231,8 +235,10 @@ where
///
/// If the service exists in `ready` and does not poll as ready, it is moved to
/// `not_ready`, potentially altering the order of `ready` and/or `not_ready`.
fn poll_ready_index(&mut self, idx: usize)
-> Option<Poll<(), Error<<D::Service as Service>::Error, D::DiscoverError>>>
fn poll_ready_index<Request>(&mut self, idx: usize)
-> Option<Poll<(), Error<<D::Service as Service<Request>>::Error, D::Error>>>
where
D::Service: Service<Request>,
{
match self.ready.get_index_mut(idx) {
None => return None,
@ -253,8 +259,10 @@ where
/// Chooses the next service to which a request will be dispatched.
///
/// Ensures that .
fn choose_and_poll_ready(&mut self)
-> Poll<(), Error<<D::Service as Service>::Error, D::DiscoverError>>
fn choose_and_poll_ready<Request>(&mut self)
-> Poll<(), Error<<D::Service as Service<Request>>::Error, D::Error>>
where
D::Service: Service<Request>,
{
loop {
let n = self.ready.len();
@ -277,15 +285,15 @@ where
}
}
impl<D, C> Service for Balance<D, C>
impl<D, C, Request> Service<Request> for Balance<D, C>
where
D: Discover,
D::Service: Service<Request>,
C: Choose<D::Key, D::Service>,
{
type Request = <D::Service as Service>::Request;
type Response = <D::Service as Service>::Response;
type Error = Error<<D::Service as Service>::Error, D::DiscoverError>;
type Future = ResponseFuture<<D::Service as Service>::Future, D::DiscoverError>;
type Response = <D::Service as Service<Request>>::Response;
type Error = Error<<D::Service as Service<Request>>::Error, D::Error>;
type Future = ResponseFuture<<D::Service as Service<Request>>::Future, D::Error>;
/// Prepares the balancer to process a request.
///
@ -310,7 +318,7 @@ where
self.choose_and_poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let idx = self.chosen_ready_index.take().expect("not ready");
let (_, svc) = self.ready.get_index_mut(idx).expect("invalid chosen ready index");
self.dispatched_ready_index = Some(idx);
@ -388,21 +396,17 @@ mod tests {
impl Discover for ReluctantDisco {
type Key = usize;
type Request = ();
type Response = ();
type Error = ();
type Service = ReluctantService;
type DiscoverError = ();
type Error = ();
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::DiscoverError> {
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::Error> {
let r = self.0.pop_front().map(Async::Ready).unwrap_or(Async::NotReady);
debug!("polling disco: {:?}", r.is_ready());
Ok(r)
}
}
impl Service for ReluctantService {
type Request = ();
impl Service<()> for ReluctantService {
type Response = ();
type Error = ();
type Future = future::FutureResult<(), ()>;

View File

@ -29,8 +29,11 @@ impl<T, M: Copy> Load for Constant<T, M> {
}
}
impl<S: Service, M: Copy> Service for Constant<S, M> {
type Request = S::Request;
impl<S, M, Request> Service<Request> for Constant<S, M>
where
S: Service<Request>,
M: Copy,
{
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
@ -39,7 +42,7 @@ impl<S: Service, M: Copy> Service for Constant<S, M> {
self.inner.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
self.inner.call(req)
}
}
@ -47,14 +50,11 @@ impl<S: Service, M: Copy> Service for Constant<S, M> {
/// Proxies `Discover` such that all changes are wrapped with a constant load.
impl<D: Discover, M: Copy> Discover for Constant<D, M> {
type Key = D::Key;
type Request = D::Request;
type Response = D::Response;
type Error = D::Error;
type Service = Constant<D::Service, M>;
type DiscoverError = D::DiscoverError;
type Error = D::Error;
/// Yields the next discovery change set.
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::DiscoverError> {
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::Error> {
use self::Change::*;
let change = match try_ready!(self.inner.poll()) {

View File

@ -81,11 +81,13 @@ const NANOS_PER_MILLI: f64 = 1_000_000.0;
// ===== impl PeakEwma =====
impl<D, I> WithPeakEwma<D, I>
where
D: Discover,
I: Instrument<Handle, D::Response>,
{
pub fn new(discover: D, decay: Duration, instrument: I) -> Self {
pub fn new<Request>(discover: D, decay: Duration, instrument: I) -> Self
where
D: Discover,
D::Service: Service<Request>,
I: Instrument<Handle, <D::Service as Service<Request>>::Response>
{
WithPeakEwma {
discover,
decay_ns: nanos(decay),
@ -97,16 +99,13 @@ where
impl<D, I> Discover for WithPeakEwma<D, I>
where
D: Discover,
I: Instrument<Handle, D::Response>,
I: Clone,
{
type Key = D::Key;
type Request = D::Request;
type Response = I::Output;
type Error = D::Error;
type Service = PeakEwma<D::Service, I>;
type DiscoverError = D::DiscoverError;
type Error = D::Error;
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::DiscoverError> {
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::Error> {
use self::Change::*;
let change = match try_ready!(self.discover.poll()) {
@ -120,11 +119,7 @@ where
// ===== impl PeakEwma =====
impl<S, I> PeakEwma<S, I>
where
S: Service,
I: Instrument<Handle, S::Response>,
{
impl<S, I> PeakEwma<S, I> {
fn new(service: S, decay_ns: f64, instrument: I) -> Self {
Self {
service,
@ -143,12 +138,11 @@ where
}
}
impl<S, I> Service for PeakEwma<S, I>
impl<S, I, Request> Service<Request> for PeakEwma<S, I>
where
S: Service,
S: Service<Request>,
I: Instrument<Handle, S::Response>,
{
type Request = S::Request;
type Response = I::Output;
type Error = S::Error;
type Future = InstrumentFuture<S::Future, I, Handle>;
@ -157,7 +151,7 @@ where
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
InstrumentFuture::new(self.instrument.clone(), self.handle(), self.service.call(req))
}
}
@ -306,8 +300,7 @@ mod tests {
use super::*;
struct Svc;
impl Service for Svc {
type Request = ();
impl Service<()> for Svc {
type Response = ();
type Error = ();
type Future = future::FutureResult<(), ()>;

View File

@ -8,11 +8,7 @@ use super::{Instrument, InstrumentFuture, NoInstrument};
/// Expresses load based on the number of currently-pending requests.
#[derive(Debug)]
pub struct PendingRequests<S, I = NoInstrument>
where
S: Service,
I: Instrument<Handle, S::Response>,
{
pub struct PendingRequests<S, I = NoInstrument> {
service: S,
ref_count: RefCount,
instrument: I,
@ -25,11 +21,7 @@ struct RefCount(Arc<()>);
/// Wraps `inner`'s services with `PendingRequests`.
#[derive(Debug)]
pub struct WithPendingRequests<D, I = NoInstrument>
where
D: Discover,
I: Instrument<Handle, D::Response>,
{
pub struct WithPendingRequests<D, I = NoInstrument> {
discover: D,
instrument: I,
}
@ -43,11 +35,7 @@ pub struct Handle(RefCount);
// ===== impl PendingRequests =====
impl<S, I> PendingRequests<S, I>
where
S: Service,
I: Instrument<Handle, S::Response>,
{
impl<S, I> PendingRequests<S, I> {
fn new(service: S, instrument: I) -> Self {
Self {
service,
@ -61,11 +49,7 @@ where
}
}
impl<S, I> Load for PendingRequests<S, I>
where
S: Service,
I: Instrument<Handle, S::Response>,
{
impl<S, I> Load for PendingRequests<S, I> {
type Metric = Count;
fn load(&self) -> Count {
@ -74,12 +58,11 @@ where
}
}
impl<S, I> Service for PendingRequests<S, I>
impl<S, I, Request> Service<Request> for PendingRequests<S, I>
where
S: Service,
S: Service<Request>,
I: Instrument<Handle, S::Response>,
{
type Request = S::Request;
type Response = I::Output;
type Error = S::Error;
type Future = InstrumentFuture<S::Future, I, Handle>;
@ -88,19 +71,20 @@ where
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
InstrumentFuture::new(self.instrument.clone(), self.handle(), self.service.call(req))
}
}
// ===== impl WithPendingRequests =====
impl<D, I> WithPendingRequests<D, I>
where
D: Discover,
I: Instrument<Handle, D::Response>,
{
pub fn new(discover: D, instrument: I) -> Self {
impl<D, I> WithPendingRequests<D, I> {
pub fn new<Request>(discover: D, instrument: I) -> Self
where
D: Discover,
D::Service: Service<Request>,
I: Instrument<Handle, <D::Service as Service<Request>>::Response>,
{
Self { discover, instrument }
}
}
@ -108,17 +92,14 @@ where
impl<D, I> Discover for WithPendingRequests<D, I>
where
D: Discover,
I: Instrument<Handle, D::Response>,
I: Clone,
{
type Key = D::Key;
type Request = D::Request;
type Response = I::Output;
type Error = D::Error;
type Service = PendingRequests<D::Service, I>;
type DiscoverError = D::DiscoverError;
type Error = D::Error;
/// Yields the next discovery change set.
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::DiscoverError> {
fn poll(&mut self) -> Poll<Change<D::Key, Self::Service>, D::Error> {
use self::Change::*;
let change = match try_ready!(self.discover.poll()) {
@ -144,8 +125,7 @@ mod tests {
use super::*;
struct Svc;
impl Service for Svc {
type Request = ();
impl Service<()> for Svc {
type Response = ();
type Error = ();
type Future = future::FutureResult<(), ()>;

View File

@ -27,18 +27,16 @@ use std::sync::atomic::Ordering;
/// Adds a buffer in front of an inner service.
///
/// See crate level documentation for more details.
pub struct Buffer<T>
where T: Service,
pub struct Buffer<T, Request>
where T: Service<Request>,
{
tx: UnboundedSender<Message<T>>,
tx: UnboundedSender<Message<Request, T::Future>>,
state: Arc<State>,
}
/// Future eventually completed with the response to the original request.
pub struct ResponseFuture<T>
where T: Service,
{
state: ResponseState<T::Future>,
pub struct ResponseFuture<T> {
state: ResponseState<T>,
}
/// Errors produced by `Buffer`.
@ -50,11 +48,11 @@ pub enum Error<T> {
/// Task that handles processing the buffer. This type should not be used
/// directly, instead `Buffer` requires an `Executor` that can accept this task.
pub struct Worker<T>
where T: Service,
pub struct Worker<T, Request>
where T: Service<Request>,
{
current_message: Option<Message<T>>,
rx: UnboundedReceiver<Message<T>>,
current_message: Option<Message<Request, T::Future>>,
rx: UnboundedReceiver<Message<Request, T::Future>>,
service: T,
state: Arc<State>,
}
@ -67,9 +65,9 @@ pub struct SpawnError<T> {
/// Message sent over buffer
#[derive(Debug)]
struct Message<T: Service> {
request: T::Request,
tx: oneshot::Sender<T::Future>,
struct Message<Request, Fut> {
request: Request,
tx: oneshot::Sender<Fut>,
}
/// State shared between `Buffer` and `Worker`
@ -82,8 +80,9 @@ enum ResponseState<T> {
Poll(T),
}
impl<T> Buffer<T>
where T: Service,
impl<T, Request> Buffer<T, Request>
where
T: Service<Request>,
{
/// Creates a new `Buffer` wrapping `service`.
///
@ -91,7 +90,8 @@ where T: Service,
/// draining the buffer and dispatching the requests to the internal
/// service.
pub fn new<E>(service: T, executor: &E) -> Result<Self, SpawnError<T>>
where E: Executor<Worker<T>>,
where
E: Executor<Worker<T, Request>>,
{
let (tx, rx) = mpsc::unbounded();
@ -117,13 +117,13 @@ where T: Service,
}
}
impl<T> Service for Buffer<T>
where T: Service,
impl<T, Request> Service<Request> for Buffer<T, Request>
where
T: Service<Request>,
{
type Request = T::Request;
type Response = T::Response;
type Error = Error<T::Error>;
type Future = ResponseFuture<T>;
type Future = ResponseFuture<T::Future>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
// If the inner service has errored, then we error here.
@ -136,7 +136,7 @@ where T: Service,
}
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let (tx, rx) = oneshot::channel();
let sent = self.tx.unbounded_send(Message {
@ -152,8 +152,9 @@ where T: Service,
}
}
impl<T> Clone for Buffer<T>
where T: Service
impl<T, Request> Clone for Buffer<T, Request>
where
T: Service<Request>,
{
fn clone(&self) -> Self {
Self {
@ -166,9 +167,10 @@ where T: Service
// ===== impl ResponseFuture =====
impl<T> Future for ResponseFuture<T>
where T: Service
where
T: Future,
{
type Item = T::Response;
type Item = T::Item;
type Error = Error<T::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
@ -197,11 +199,12 @@ where T: Service
// ===== impl Worker =====
impl<T> Worker<T>
where T: Service
impl<T, Request> Worker<T, Request>
where
T: Service<Request>,
{
/// Return the next queued Message that hasn't been canceled.
fn poll_next_msg(&mut self) -> Poll<Option<Message<T>>, ()> {
fn poll_next_msg(&mut self) -> Poll<Option<Message<Request, T::Future>>, ()> {
if let Some(mut msg) = self.current_message.take() {
// poll_cancel returns Async::Ready is the receiver is dropped.
// Returning NotReady means it is still alive, so we should still
@ -223,8 +226,9 @@ where T: Service
}
}
impl<T> Future for Worker<T>
where T: Service,
impl<T, Request> Future for Worker<T, Request>
where
T: Service<Request>,
{
type Item = ();
type Error = ();

View File

@ -61,8 +61,10 @@ type Handle = tower_mock::Handle<&'static str, &'static str, ()>;
struct Exec;
impl futures::future::Executor<Worker<Mock>> for Exec {
fn execute(&self, fut: Worker<Mock>) -> Result<(), futures::future::ExecuteError<Worker<Mock>>> {
impl futures::future::Executor<Worker<Mock, &'static str>> for Exec {
fn execute(&self, fut: Worker<Mock, &'static str>)
-> Result<(), futures::future::ExecuteError<Worker<Mock, &'static str>>>
{
thread::spawn(move || {
fut.wait().unwrap();
});
@ -70,7 +72,7 @@ impl futures::future::Executor<Worker<Mock>> for Exec {
}
}
fn new_service() -> (Buffer<Mock>, Handle) {
fn new_service() -> (Buffer<Mock, &'static str>, Handle) {
let (service, handle) = Mock::new();
let service = Buffer::new(service, &Exec).unwrap();
(service, handle)

View File

@ -24,25 +24,13 @@ pub trait Discover {
/// NewService key
type Key: Hash + Eq;
/// Requests handled by the discovered services
type Request;
/// Responses given by the discovered services
type Response;
/// Errors produced by the discovered services
type Error;
/// The discovered `Service` instance.
type Service: Service<Request = Self::Request,
Response = Self::Response,
Error = Self::Error>;
type Service;
/// Error produced during discovery
type DiscoverError;
type Error;
/// Yields the next discovery change set.
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::DiscoverError>;
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::Error>;
}
/// A change in the service set
@ -62,11 +50,13 @@ pub struct List<T> {
// ===== impl List =====
impl<T, U> List<T>
where T: Iterator<Item = U>,
U: Service,
where
T: Iterator<Item = U>
{
pub fn new<I>(services: I) -> List<T>
where I: IntoIterator<Item = U, IntoIter = T>,
pub fn new<I, Request>(services: I) -> List<T>
where
I: IntoIterator<Item = U, IntoIter = T>,
U: Service<Request>
{
List { inner: services.into_iter().enumerate() }
}
@ -74,16 +64,12 @@ where T: Iterator<Item = U>,
impl<T, U> Discover for List<T>
where T: Iterator<Item = U>,
U: Service,
{
type Key = usize;
type Request = U::Request;
type Response = U::Response;
type Error = U::Error;
type Service = U;
type DiscoverError = ();
type Error = ();
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::DiscoverError> {
fn poll(&mut self) -> Poll<Change<Self::Key, Self::Service>, Self::Error> {
match self.inner.next() {
Some((i, service)) => Ok(Change::Insert(i, service).into()),
None => Ok(Async::NotReady),

View File

@ -21,17 +21,19 @@ pub struct Filter<T, U> {
counts: Arc<Counts>,
}
pub struct ResponseFuture<T, S>
where S: Service,
pub struct ResponseFuture<T, S, Request>
where
S: Service<Request>,
{
inner: Option<ResponseInner<T, S>>,
inner: Option<ResponseInner<T, S, Request>>,
}
#[derive(Debug)]
struct ResponseInner<T, S>
where S: Service,
struct ResponseInner<T, S, Request>
where
S: Service<Request>,
{
state: State<S::Request, S::Future>,
state: State<Request, S::Future>,
check: T,
service: S,
counts: Arc<Counts>,
@ -51,11 +53,11 @@ pub enum Error<T, U> {
}
/// Checks a request
pub trait Predicate<T> {
pub trait Predicate<Request> {
type Error;
type Future: Future<Item = (), Error = Self::Error>;
fn check(&mut self, request: &T) -> Self::Future;
fn check(&mut self, request: &Request) -> Self::Future;
}
#[derive(Debug)]
@ -68,20 +70,21 @@ struct Counts {
}
#[derive(Debug)]
enum State<T, U> {
Check(T),
WaitReady(T),
enum State<Request, U> {
Check(Request),
WaitReady(Request),
WaitResponse(U),
NoCapacity,
}
// ===== impl Filter =====
impl<T, U> Filter<T, U>
where T: Service + Clone,
U: Predicate<T::Request>,
{
pub fn new(inner: T, predicate: U, buffer: usize) -> Self {
impl<T, U> Filter<T, U> {
pub fn new<Request>(inner: T, predicate: U, buffer: usize) -> Self
where
T: Service<Request> + Clone,
U: Predicate<Request>,
{
let counts = Counts {
task: AtomicTask::new(),
rem: AtomicUsize::new(buffer),
@ -95,14 +98,13 @@ where T: Service + Clone,
}
}
impl<T, U> Service for Filter<T, U>
where T: Service + Clone,
U: Predicate<T::Request>,
impl<T, U, Request> Service<Request> for Filter<T, U>
where T: Service<Request> + Clone,
U: Predicate<Request>,
{
type Request = T::Request;
type Response = T::Response;
type Error = Error<U::Error, T::Error>;
type Future = ResponseFuture<U::Future, T>;
type Future = ResponseFuture<U::Future, T, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.counts.task.register();
@ -118,7 +120,7 @@ where T: Service + Clone,
}
}
fn call(&mut self, request: T::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let rem = self.counts.rem.load(SeqCst);
if rem == 0 {
@ -166,12 +168,12 @@ impl<F, T, U> Predicate<T> for F
// ===== impl ResponseFuture =====
impl<T, U> Future for ResponseFuture<T, U>
impl<T, S, Request> Future for ResponseFuture<T, S, Request>
where T: Future,
U: Service,
S: Service<Request>,
{
type Item = U::Response;
type Error = Error<T::Error, U::Error>;
type Item = S::Response;
type Error = Error<T::Error, S::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
match self.inner {
@ -181,11 +183,11 @@ where T: Future,
}
}
impl<T, S> fmt::Debug for ResponseFuture<T, S>
impl<T, S, Request> fmt::Debug for ResponseFuture<T, S, Request>
where T: fmt::Debug,
S: Service + fmt::Debug,
S::Request: fmt::Debug,
S: Service<Request> + fmt::Debug,
S::Future: fmt::Debug,
Request: fmt::Debug,
{
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
fmt.debug_struct("ResponseFuture")
@ -196,9 +198,9 @@ where T: fmt::Debug,
// ===== impl ResponseInner =====
impl<T, U> ResponseInner<T, U>
impl<T, S, Request> ResponseInner<T, S, Request>
where T: Future,
U: Service,
S: Service<Request>,
{
fn inc_rem(&self) {
if 0 == self.counts.rem.fetch_add(1, SeqCst) {
@ -206,7 +208,7 @@ where T: Future,
}
}
fn poll(&mut self) -> Poll<U::Response, Error<T::Error, U::Error>> {
fn poll(&mut self) -> Poll<S::Response, Error<T::Error, S::Error>> {
use self::State::*;
loop {

View File

@ -49,7 +49,10 @@ struct Shared {
impl<T> InFlightLimit<T> {
/// Create a new rate limiter
pub fn new(inner: T, max: usize) -> Self {
pub fn new<Request>(inner: T, max: usize) -> Self
where
T: Service<Request>,
{
InFlightLimit {
inner,
state: State {
@ -79,10 +82,10 @@ impl<T> InFlightLimit<T> {
}
}
impl<S> Service for InFlightLimit<S>
where S: Service
impl<S, Request> Service<Request> for InFlightLimit<S>
where
S: Service<Request>,
{
type Request = S::Request;
type Response = S::Response;
type Error = Error<S::Error>;
type Future = ResponseFuture<S::Future>;
@ -105,7 +108,7 @@ where S: Service
.map_err(Error::Upstream)
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
// In this implementation, `poll_ready` is not expected to be called
// first (though, it might have been).
if self.state.reserved {

View File

@ -100,8 +100,7 @@ impl<T, U, E> Mock<T, U, E> {
}
}
impl<T, U, E> Service for Mock<T, U, E> {
type Request = T;
impl<T, U, E> Service<T> for Mock<T, U, E> {
type Response = U;
type Error = Error<E>;
type Future = ResponseFuture<U, E>;
@ -133,7 +132,7 @@ impl<T, U, E> Service for Mock<T, U, E> {
}
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: T) -> Self::Future {
// Make sure that the service has capacity
let mut state = self.state.lock().unwrap();

View File

@ -54,7 +54,10 @@ enum State {
impl<T> RateLimit<T> {
/// Create a new rate limiter
pub fn new(inner: T, rate: Rate, timer: Timer) -> Self {
pub fn new<Request>(inner: T, rate: Rate, timer: Timer) -> Self
where
T: Service<Request>,
{
let state = State::Ready {
until: Instant::now(),
rem: rate.num,
@ -98,10 +101,9 @@ impl Rate {
}
}
impl<S> Service for RateLimit<S>
where S: Service
impl<S, Request> Service<Request> for RateLimit<S>
where S: Service<Request>
{
type Request = S::Request;
type Response = S::Response;
type Error = Error<S::Error>;
type Future = ResponseFuture<S::Future>;
@ -125,7 +127,7 @@ where S: Service
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
match self.state {
State::Ready { mut until, mut rem } => {
let now = Instant::now();

View File

@ -8,11 +8,12 @@ use tower_service::{Service, NewService};
use std::{error, fmt};
pub struct Reconnect<T>
where T: NewService,
pub struct Reconnect<T, Request>
where
T: NewService<Request>,
{
new_service: T,
state: State<T>,
state: State<T, Request>,
}
#[derive(Debug)]
@ -22,15 +23,20 @@ pub enum Error<T, U> {
NotReady,
}
pub struct ResponseFuture<T>
where T: NewService
pub struct ResponseFuture<T, Request>
where
// TODO:
// This struct should just be generic over the response future, but
// doing so would require changing the future's error type
T: NewService<Request>,
{
inner: Option<<T::Service as Service>::Future>,
inner: Option<<T::Service as Service<Request>>::Future>,
}
#[derive(Debug)]
enum State<T>
where T: NewService
enum State<T, Request>
where
T: NewService<Request>
{
Idle,
Connecting(T::Future),
@ -39,8 +45,9 @@ where T: NewService
// ===== impl Reconnect =====
impl<T> Reconnect<T>
where T: NewService,
impl<T, Request> Reconnect<T, Request>
where
T: NewService<Request>,
{
pub fn new(new_service: T) -> Self {
Reconnect {
@ -50,13 +57,13 @@ where T: NewService,
}
}
impl<T> Service for Reconnect<T>
where T: NewService
impl<T, Request> Service<Request> for Reconnect<T, Request>
where
T: NewService<Request>
{
type Request = T::Request;
type Response = T::Response;
type Error = Error<T::Error, T::InitError>;
type Future = ResponseFuture<T>;
type Future = ResponseFuture<T, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
use self::State::*;
@ -116,7 +123,7 @@ where T: NewService
ret
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
use self::State::*;
trace!("call");
@ -131,10 +138,12 @@ where T: NewService
}
}
impl<T> fmt::Debug for Reconnect<T>
where T: NewService + fmt::Debug,
T::Future: fmt::Debug,
T::Service: fmt::Debug,
impl<T, Request> fmt::Debug for Reconnect<T, Request>
where
T: NewService<Request> + fmt::Debug,
T::Future: fmt::Debug,
T::Service: fmt::Debug,
Request: fmt::Debug,
{
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
fmt.debug_struct("Reconnect")
@ -146,7 +155,10 @@ where T: NewService + fmt::Debug,
// ===== impl ResponseFuture =====
impl<T: NewService> Future for ResponseFuture<T> {
impl<T, Request> Future for ResponseFuture<T, Request>
where
T: NewService<Request>,
{
type Item = T::Response;
type Error = Error<T::Error, T::InitError>;

View File

@ -15,8 +15,12 @@ pub struct Retry<P, S> {
}
#[derive(Debug)]
pub struct ResponseFuture<P: Policy<S::Request, S::Response, S::Error>, S: Service> {
request: Option<S::Request>,
pub struct ResponseFuture<P, S, Request>
where
P: Policy<Request, S::Response, S::Error>,
S: Service<Request>,
{
request: Option<Request>,
retry: Retry<P, S>,
state: State<S::Future, P::Future, S::Response, S::Error>,
}
@ -40,12 +44,12 @@ pub trait Policy<Req, Res, E>: Sized {
// ===== impl Retry =====
impl<P, S> Retry<P, S>
where
P: Policy<S::Request, S::Response, S::Error> + Clone,
S: Service + Clone,
{
pub fn new(policy: P, service: S) -> Self {
impl<P, S> Retry<P, S> {
pub fn new<Request>(policy: P, service: S) -> Self
where
P: Policy<Request, S::Response, S::Error> + Clone,
S: Service<Request> + Clone,
{
Retry {
policy,
service,
@ -53,21 +57,20 @@ where
}
}
impl<P, S> Service for Retry<P, S>
impl<P, S, Request> Service<Request> for Retry<P, S>
where
P: Policy<S::Request, S::Response, S::Error> + Clone,
S: Service + Clone,
P: Policy<Request, S::Response, S::Error> + Clone,
S: Service<Request> + Clone,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Future = ResponseFuture<P, S>;
type Future = ResponseFuture<P, S, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let cloned = self.policy.clone_request(&request);
let future = self.service.call(request);
ResponseFuture {
@ -80,10 +83,10 @@ where
// ===== impl ResponseFuture =====
impl<P, S> Future for ResponseFuture<P, S>
impl<P, S, Request> Future for ResponseFuture<P, S, Request>
where
P: Policy<S::Request, S::Response, S::Error> + Clone,
S: Service + Clone,
P: Policy<Request, S::Response, S::Error> + Clone,
S: Service<Request> + Clone,
{
type Item = S::Response;
type Error = S::Error;

View File

@ -21,10 +21,7 @@ pub struct Router<T> {
}
/// Matches the request with a route
pub trait Recognize: 'static {
/// Request being matched
type Request;
pub trait Recognize<Request>: 'static {
/// Inner service's response
type Response;
@ -35,7 +32,7 @@ pub trait Recognize: 'static {
type RouteError;
/// The destination service
type Service: Service<Request = Self::Request,
type Service: Service<Request,
Response = Self::Response,
Error = Self::Error>;
@ -50,14 +47,14 @@ pub trait Recognize: 'static {
/// service should determine the buffering strategy used to handle the
/// request until the request can be processed. This behavior enables
/// punting all buffering decisions to the inner service.
fn recognize(&mut self, request: &Self::Request)
fn recognize(&mut self, request: &Request)
-> Result<&mut Self::Service, Self::RouteError>;
}
pub struct ResponseFuture<T>
where T: Recognize,
pub struct ResponseFuture<T, Request>
where T: Recognize<Request>,
{
state: ResponseState<T>,
state: ResponseState<T, Request>,
}
/// Error produced by the `Router` service
@ -75,14 +72,14 @@ pub enum Error<T, U> {
NotReady,
}
enum ResponseState<T>
where T: Recognize
enum ResponseState<T, Request>
where T: Recognize<Request>
{
Dispatched(<T::Service as Service>::Future),
Dispatched(<T::Service as Service<Request>>::Future),
RouteError(T::RouteError),
Queued {
service: BorrowGuard<T::Service>,
request: T::Request,
request: Request,
},
NotReady,
Invalid,
@ -90,22 +87,22 @@ where T: Recognize
// ===== impl Router =====
impl<T> Router<T>
where T: Recognize
{
impl<T> Router<T> {
/// Create a new router
pub fn new(recognize: T) -> Self {
pub fn new<Request>(recognize: T) -> Self
where
T: Recognize<Request>,
{
Router { recognize: Borrow::new(recognize) }
}
}
impl<T> Service for Router<T>
where T: Recognize,
impl<T, Request> Service<Request> for Router<T>
where T: Recognize<Request>,
{
type Request = T::Request;
type Response = T::Response;
type Error = Error<T::Error, T::RouteError>;
type Future = ResponseFuture<T>;
type Future = ResponseFuture<T, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
// Checks if there is an outstanding borrow (i.e. there is an in-flight
@ -116,7 +113,7 @@ where T: Recognize,
self.recognize.poll_ready().map_err(|_| panic!())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let borrow = match self.recognize.try_borrow() {
Ok(borrow) => borrow,
Err(_) => {
@ -151,8 +148,8 @@ where T: Recognize,
// ===== impl ResponseFuture =====
impl<T> Future for ResponseFuture<T>
where T: Recognize,
impl<T, Request> Future for ResponseFuture<T, Request>
where T: Recognize<Request>,
{
type Item = T::Response;
type Error = Error<T::Error, T::RouteError>;

View File

@ -103,16 +103,15 @@ impl<T> MapRecognize<T> {
}
}
impl<T> Recognize for MapRecognize<T>
where T: Service<Request=String, Response=String, Error = ()> + 'static,
impl<T> Recognize<String> for MapRecognize<T>
where T: Service<String, Response=String, Error = ()> + 'static,
{
type Request = String;
type Response = String;
type Error = ();
type RouteError = ();
type Service = T;
fn recognize(&mut self, request: &Self::Request)
fn recognize(&mut self, request: &String)
-> Result<&mut Self::Service, Self::RouteError>
{
match self.map.get_mut(request) {
@ -143,8 +142,7 @@ impl StringService {
}
}
impl Service for StringService {
type Request = String;
impl Service<String> for StringService {
type Response = String;
type Error = ();
type Future = FutureResult<Self::Response, Self::Error>;
@ -153,7 +151,7 @@ impl Service for StringService {
Ok(Async::Ready(()))
}
fn call(&mut self, _: Self::Request) -> Self::Future {
fn call(&mut self, _: String) -> Self::Future {
future::result(self.string.clone())
}
}
@ -177,8 +175,7 @@ impl MaybeService {
}
}
impl Service for MaybeService {
type Request = String;
impl Service<String> for MaybeService {
type Response = String;
type Error = ();
type Future = FutureResult<Self::Response, Self::Error>;
@ -191,7 +188,7 @@ impl Service for MaybeService {
}
}
fn call(&mut self, _: Self::Request) -> Self::Future {
fn call(&mut self, _: String) -> Self::Future {
match self.string.clone() {
Some(string) => future::ok(string),
None => future::err(()),

View File

@ -17,6 +17,7 @@ extern crate futures;
use futures::{Future, IntoFuture, Poll};
use std::marker::PhantomData;
use std::rc::Rc;
use std::sync::Arc;
@ -47,11 +48,10 @@ use std::sync::Arc;
/// As an example, here is how an HTTP request is processed by a server:
///
/// ```rust,ignore
/// impl Service for HelloWorld {
/// type Request = http::Request;
/// impl Service<http::Request> for HelloWorld {
/// type Response = http::Response;
/// type Error = http::Error;
/// type Future = Box<Future<Item = Self::Response, Error = http::Error>>;
/// type Future = Box<Future<Item = Self::Response, Error = Self::Error>>;
///
/// fn poll_ready(&mut self) -> Poll<(), Self::Error> {
/// Ok(Async::Ready(()))
@ -122,11 +122,11 @@ use std::sync::Arc;
/// }
/// }
///
/// impl<T> Service for Timeout<T>
/// where T: Service,
/// T::Error: From<Expired>,
/// impl<T, Request> Service<Request> for Timeout<T>
/// where
/// T: Service<Request>,
/// T::Error: From<Expired>,
/// {
/// type Request = T::Request;
/// type Response = T::Response;
/// type Error = T::Error;
/// type Future = Box<Future<Item = Self::Response, Error = Self::Error>>;
@ -135,7 +135,7 @@ use std::sync::Arc;
/// Ok(Async::Ready(()))
/// }
///
/// fn call(&mut self, req: Self::Req) -> Self::Future {
/// fn call(&mut self, req: Request) -> Self::Future {
/// let timeout = self.timer.sleep(self.delay)
/// .and_then(|_| Err(Self::Error::from(Expired)));
///
@ -162,10 +162,7 @@ use std::sync::Arc;
/// `Service` provides a mechanism by which the caller is able to coordinate
/// readiness. `Service::poll_ready` returns `Ready` if the service expects that
/// it is able to process a request.
pub trait Service {
/// Requests handled by the service.
type Request;
pub trait Service<Request> {
/// Responses given by the service.
type Response;
@ -177,8 +174,11 @@ pub trait Service {
type Future: Future<Item = Self::Response, Error = Self::Error>;
/// A future yielding the service when it is ready to accept a request.
fn ready(self) -> Ready<Self> where Self: Sized {
Ready { inner: Some(self) }
fn ready(self) -> Ready<Self, Request> where Self: Sized {
Ready {
inner: Some(self),
_p: PhantomData,
}
}
/// Returns `Ready` when the service is able to process requests.
@ -201,14 +201,15 @@ pub trait Service {
///
/// Calling `call` without calling `poll_ready` is permitted. The
/// implementation must be resilient to this fact.
fn call(&mut self, req: Self::Request) -> Self::Future;
fn call(&mut self, req: Request) -> Self::Future;
}
/// Future yielding a `Service` once the service is ready to process a request
///
/// `Ready` values are produced by `Service::ready`.
pub struct Ready<T> {
pub struct Ready<T, Request> {
inner: Option<T>,
_p: PhantomData<fn() -> Request>,
}
/// Creates new `Service` values.
@ -218,10 +219,7 @@ pub struct Ready<T> {
/// accepts new TCP streams, obtains a new `Service` value using the
/// `NewService` trait, and uses that new `Service` value to process inbound
/// requests on that new TCP stream.
pub trait NewService {
/// Requests handled by the service
type Request;
pub trait NewService<Request> {
/// Responses given by the service
type Response;
@ -229,7 +227,7 @@ pub trait NewService {
type Error;
/// The `Service` value created by this factory
type Service: Service<Request = Self::Request, Response = Self::Response, Error = Self::Error>;
type Service: Service<Request, Response = Self::Response, Error = Self::Error>;
/// Errors produced while building a service.
type InitError;
@ -241,8 +239,8 @@ pub trait NewService {
fn new_service(&self) -> Self::Future;
}
impl<T> Future for Ready<T>
where T: Service,
impl<T, Request> Future for Ready<T, Request>
where T: Service<Request>,
{
type Item = T;
type Error = T::Error;
@ -259,12 +257,11 @@ where T: Service,
}
}
impl<F, R, E, S> NewService for F
impl<F, R, E, S, Request> NewService<Request> for F
where F: Fn() -> R,
R: IntoFuture<Item = S, Error = E>,
S: Service,
S: Service<Request>,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Service = S;
@ -276,8 +273,10 @@ impl<F, R, E, S> NewService for F
}
}
impl<S: NewService + ?Sized> NewService for Arc<S> {
type Request = S::Request;
impl<S, Request> NewService<Request> for Arc<S>
where
S: NewService<Request> + ?Sized,
{
type Response = S::Response;
type Error = S::Error;
type Service = S::Service;
@ -289,8 +288,10 @@ impl<S: NewService + ?Sized> NewService for Arc<S> {
}
}
impl<S: NewService + ?Sized> NewService for Rc<S> {
type Request = S::Request;
impl<S, Request> NewService<Request> for Rc<S>
where
S: NewService<Request> + ?Sized,
{
type Response = S::Response;
type Error = S::Error;
type Service = S::Service;
@ -302,8 +303,10 @@ impl<S: NewService + ?Sized> NewService for Rc<S> {
}
}
impl<'a, S: Service + 'a> Service for &'a mut S {
type Request = S::Request;
impl<'a, S, Request> Service<Request> for &'a mut S
where
S: Service<Request> + 'a
{
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
@ -312,13 +315,15 @@ impl<'a, S: Service + 'a> Service for &'a mut S {
(**self).poll_ready()
}
fn call(&mut self, request: S::Request) -> S::Future {
fn call(&mut self, request: Request) -> S::Future {
(**self).call(request)
}
}
impl<S: Service + ?Sized> Service for Box<S> {
type Request = S::Request;
impl<S, Request> Service<Request> for Box<S>
where
S: Service<Request> + ?Sized,
{
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
@ -327,7 +332,7 @@ impl<S: Service + ?Sized> Service for Box<S> {
(**self).poll_ready()
}
fn call(&mut self, request: S::Request) -> S::Future {
fn call(&mut self, request: Request) -> S::Future {
(**self).call(request)
}
}

View File

@ -49,10 +49,10 @@ impl<T> Timeout<T> {
}
}
impl<S> Service for Timeout<S>
where S: Service,
impl<S, Request> Service<Request> for Timeout<S>
where
S: Service<Request>,
{
type Request = S::Request;
type Response = S::Response;
type Error = Error<S::Error>;
type Future = ResponseFuture<S::Future>;
@ -62,7 +62,7 @@ where S: Service,
.map_err(Error::Inner)
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
ResponseFuture {
response: self.inner.call(request),
sleep: Delay::new(clock::now() + self.timeout),

View File

@ -27,10 +27,9 @@
//! f: F,
//! }
//!
//! impl<F> Service for ServiceFn<F>
//! impl<F> Service<String> for ServiceFn<F>
//! where F: Fn(String) -> String,
//! {
//! type Request = String;
//! type Response = String;
//! type Error = ();
//! type Future = FutureResult<String, ()>;
@ -69,10 +68,9 @@ use std::fmt;
///
/// See module level documentation for more details.
pub struct BoxService<T, U, E> {
inner: Box<Service<Request = T,
Response = U,
Error = E,
Future = BoxFuture<U, E>> + Send>,
inner: Box<Service<T, Response = U,
Error = E,
Future = BoxFuture<U, E>> + Send>,
}
/// A boxed `Future + Send` trait object.
@ -83,10 +81,9 @@ pub type BoxFuture<T, E> = Box<Future<Item = T, Error = E> + Send>;
/// A boxed `Service` trait object.
pub struct UnsyncBoxService<T, U, E> {
inner: Box<Service<Request = T,
Response = U,
Error = E,
Future = UnsyncBoxFuture<U, E>>>,
inner: Box<Service<T, Response = U,
Error = E,
Future = UnsyncBoxFuture<U, E>>>,
}
/// A boxed `Future` trait object.
@ -110,7 +107,7 @@ struct UnsyncBoxed<S> {
impl<T, U, E> BoxService<T, U, E>
{
pub fn new<S>(inner: S) -> Self
where S: Service<Request = T, Response = U, Error = E> + Send + 'static,
where S: Service<T, Response = U, Error = E> + Send + 'static,
S::Future: Send + 'static,
{
let inner = Box::new(Boxed { inner });
@ -118,9 +115,7 @@ impl<T, U, E> BoxService<T, U, E>
}
}
impl<T, U, E> Service for BoxService<T, U, E>
{
type Request = T;
impl<T, U, E> Service<T> for BoxService<T, U, E> {
type Response = U;
type Error = E;
type Future = BoxFuture<U, E>;
@ -147,10 +142,9 @@ where T: fmt::Debug,
// ===== impl UnsyncBoxService =====
impl<T, U, E> UnsyncBoxService<T, U, E>
{
impl<T, U, E> UnsyncBoxService<T, U, E> {
pub fn new<S>(inner: S) -> Self
where S: Service<Request = T, Response = U, Error = E> + 'static,
where S: Service<T, Response = U, Error = E> + 'static,
S::Future: 'static,
{
let inner = Box::new(UnsyncBoxed { inner });
@ -158,9 +152,7 @@ impl<T, U, E> UnsyncBoxService<T, U, E>
}
}
impl<T, U, E> Service for UnsyncBoxService<T, U, E>
{
type Request = T;
impl<T, U, E> Service<T> for UnsyncBoxService<T, U, E> {
type Response = U;
type Error = E;
type Future = UnsyncBoxFuture<U, E>;
@ -187,11 +179,10 @@ where T: fmt::Debug,
// ===== impl Boxed =====
impl<S> Service for Boxed<S>
where S: Service + 'static,
impl<S, Request> Service<Request> for Boxed<S>
where S: Service<Request> + 'static,
S::Future: Send + 'static,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Future = Box<Future<Item = S::Response,
@ -201,18 +192,17 @@ where S: Service + 'static,
self.inner.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
Box::new(self.inner.call(request))
}
}
// ===== impl UnsyncBoxed =====
impl<S> Service for UnsyncBoxed<S>
where S: Service + 'static,
impl<S, Request> Service<Request> for UnsyncBoxed<S>
where S: Service<Request> + 'static,
S::Future: 'static,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Future = Box<Future<Item = S::Response,
@ -222,7 +212,7 @@ where S: Service + 'static,
self.inner.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
Box::new(self.inner.call(request))
}
}

View File

@ -16,13 +16,12 @@ pub enum EitherService<A, B> {
B(B),
}
impl<A, B> Service for EitherService<A, B>
where A: Service,
B: Service<Request = A::Request,
impl<A, B, Request> Service<Request> for EitherService<A, B>
where A: Service<Request>,
B: Service<Request,
Response = A::Response,
Error = A::Error>,
{
type Request = A::Request;
type Response = A::Response;
type Error = A::Error;
type Future = Either<A::Future, B::Future>;
@ -36,7 +35,7 @@ where A: Service,
}
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
use self::EitherService::*;
match *self {

View File

@ -5,83 +5,60 @@ use tower_service::Service;
/// another service which completes successfully.
///
/// This is created by the `ServiceExt::and_then` method.
#[derive(Clone)]
pub struct AndThen<A, B> {
a: A,
b: B,
}
impl<A, B> AndThen<A, B>
where
A: Service,
B: Service<Request = A::Response, Error = A::Error> + Clone,
{
impl<A, B> AndThen<A, B> {
/// Create new `AndThen` combinator
pub fn new(a: A, b: B) -> AndThen<A, B> {
pub fn new<Request>(a: A, b: B) -> AndThen<A, B>
where
A: Service<Request>,
B: Service<A::Response, Error = A::Error> + Clone,
{
AndThen { a, b }
}
}
impl<A, B> Clone for AndThen<A, B>
impl<A, B, Request> Service<Request> for AndThen<A, B>
where
A: Service + Clone,
B: Service<Request = A::Response, Error = A::Error> + Clone,
A: Service<Request>,
B: Service<A::Response, Error = A::Error> + Clone,
{
fn clone(&self) -> Self {
AndThen {
a: self.a.clone(),
b: self.b.clone(),
}
}
}
impl<A, B> Service for AndThen<A, B>
where
A: Service,
B: Service<Request = A::Response, Error = A::Error> + Clone,
{
type Request = A::Request;
type Response = B::Response;
type Error = B::Error;
type Future = AndThenFuture<A, B>;
type Future = AndThenFuture<A, B, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
let _ = try_ready!(self.a.poll_ready());
self.b.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
AndThenFuture::new(self.a.call(req), self.b.clone())
fn call(&mut self, req: Request) -> Self::Future {
AndThenFuture {
fut_a: self.a.call(req),
b: self.b.clone(),
fut_b: None,
}
}
}
pub struct AndThenFuture<A, B>
pub struct AndThenFuture<A, B, Request>
where
A: Service,
B: Service<Request = A::Response, Error = A::Error>,
A: Service<Request>,
B: Service<A::Response, Error = A::Error>,
{
b: B,
fut_b: Option<B::Future>,
fut_a: A::Future,
}
impl<A, B> AndThenFuture<A, B>
impl<A, B, Request> Future for AndThenFuture<A, B, Request>
where
A: Service,
B: Service<Request = A::Response, Error = A::Error>,
{
fn new(fut_a: A::Future, b: B) -> Self {
AndThenFuture {
b,
fut_a,
fut_b: None,
}
}
}
impl<A, B> Future for AndThenFuture<A, B>
where
A: Service,
B: Service<Request = A::Response, Error = A::Error>,
A: Service<Request>,
B: Service<A::Response, Error = A::Error>,
{
type Item = B::Response;
type Error = B::Error;
@ -108,8 +85,7 @@ mod tests {
use ServiceExt;
struct Srv1(Rc<Cell<usize>>);
impl Service for Srv1 {
type Request = &'static str;
impl Service<&'static str> for Srv1 {
type Response = &'static str;
type Error = ();
type Future = FutureResult<Self::Response, ()>;
@ -119,7 +95,7 @@ mod tests {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: &'static str) -> Self::Future {
ok(req)
}
}
@ -127,8 +103,7 @@ mod tests {
#[derive(Clone)]
struct Srv2(Rc<Cell<usize>>);
impl Service for Srv2 {
type Request = &'static str;
impl Service<&'static str> for Srv2 {
type Response = (&'static str, &'static str);
type Error = ();
type Future = FutureResult<Self::Response, ()>;
@ -138,7 +113,7 @@ mod tests {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: &'static str) -> Self::Future {
ok((req, "srv2"))
}
}

View File

@ -4,20 +4,23 @@ use futures::{Future, IntoFuture, Poll};
use tower_service::Service;
/// `Apply` service combinator
pub struct Apply<T, F, R, Req> {
pub struct Apply<T, F, In, Out, Request>
where
T: Service<Request>,
{
service: T,
f: F,
_r: PhantomData<Fn(Req) -> R>,
_r: PhantomData<Fn((In, Request)) -> Out>,
}
impl<T, F, R, Req> Apply<T, F, R, Req>
impl<T, F, In, Out, Request> Apply<T, F, In, Out, Request>
where
T: Service<Error = R::Error> + Clone,
F: Fn(Req, T) -> R,
R: IntoFuture,
T: Service<Request>,
F: Fn(In, T) -> Out,
Out: IntoFuture,
{
/// Create new `Apply` combinator
pub fn new(f: F, service: T) -> Self {
pub(crate) fn new(f: F, service: T) -> Self {
Self {
service,
f,
@ -26,11 +29,10 @@ where
}
}
impl<T, F, R, Req> Clone for Apply<T, F, R, Req>
impl<T, F, In, Out, Request> Clone for Apply<T, F, In, Out, Request>
where
T: Service<Error = R::Error> + Clone,
F: Fn(Req, T) -> R + Clone,
R: IntoFuture,
T: Service<Request> + Clone,
F: Clone,
{
fn clone(&self) -> Self {
Apply {
@ -41,22 +43,21 @@ where
}
}
impl<T, F, R, Req> Service for Apply<T, F, R, Req>
impl<T, F, In, Out, Request> Service<In> for Apply<T, F, In, Out, Request>
where
T: Service<Error = R::Error> + Clone,
F: Fn(Req, T) -> R,
R: IntoFuture,
T: Service<Request, Error = Out::Error> + Clone,
F: Fn(In, T) -> Out,
Out: IntoFuture,
{
type Request = Req;
type Response = <R::Future as Future>::Item;
type Error = <R::Future as Future>::Error;
type Future = R::Future;
type Response = <Out::Future as Future>::Item;
type Error = <Out::Future as Future>::Error;
type Future = Out::Future;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: In) -> Self::Future {
let service = self.service.clone();
(self.f)(req, service).into_future()
}
@ -72,8 +73,7 @@ mod tests {
#[derive(Clone)]
struct Srv;
impl Service for Srv {
type Request = ();
impl Service<()> for Srv {
type Response = ();
type Error = ();
type Future = FutureResult<(), ()>;

View File

@ -6,16 +6,17 @@ use tower_service::Service;
/// Service for the `from_err` combinator, changing the error type of a service.
///
/// This is created by the `ServiceExt::from_err` method.
pub struct FromErr<A, E>
where
A: Service,
{
pub struct FromErr<A, E> {
service: A,
_e: PhantomData<E>,
}
impl<A: Service, E: From<A::Error>> FromErr<A, E> {
pub(crate) fn new(service: A) -> Self {
impl<A, E> FromErr<A, E> {
pub(crate) fn new<Request>(service: A) -> Self
where
A: Service<Request>,
E: From<A::Error>,
{
FromErr {
service,
_e: PhantomData,
@ -25,7 +26,7 @@ impl<A: Service, E: From<A::Error>> FromErr<A, E> {
impl<A, E> Clone for FromErr<A, E>
where
A: Service + Clone,
A: Clone,
{
fn clone(&self) -> Self {
FromErr {
@ -35,21 +36,20 @@ where
}
}
impl<A, E> Service for FromErr<A, E>
impl<A, E, Request> Service<Request> for FromErr<A, E>
where
A: Service,
A: Service<Request>,
E: From<A::Error>,
{
type Request = A::Request;
type Response = A::Response;
type Error = E;
type Future = FromErrFuture<A, E>;
type Future = FromErrFuture<A::Future, E>;
fn poll_ready(&mut self) -> Poll<(), E> {
Ok(self.service.poll_ready().map_err(E::from)?)
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
FromErrFuture {
fut: self.service.call(req),
f: PhantomData,
@ -57,17 +57,17 @@ where
}
}
pub struct FromErrFuture<A: Service, E> {
fut: A::Future,
pub struct FromErrFuture<A, E> {
fut: A,
f: PhantomData<E>,
}
impl<A, E> Future for FromErrFuture<A, E>
where
A: Service,
A: Future,
E: From<A::Error>,
{
type Item = A::Response;
type Item = A::Item;
type Error = E;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
@ -84,8 +84,7 @@ mod tests {
struct Srv;
impl Service for Srv {
type Request = ();
impl Service<()> for Srv {
type Response = ();
type Error = ();
type Future = FutureResult<(), ()>;

View File

@ -1,84 +1,87 @@
use futures::{Async, Future, Poll};
use tower_service::Service;
use std::marker::PhantomData;
/// Service for the `map` combinator, changing the type of a service's response.
///
/// This is created by the `ServiceExt::map` method.
pub struct Map<T, F, R>
where
T: Service,
F: Fn(T::Response) -> R + Clone,
{
pub struct Map<T, F, R> {
service: T,
f: F,
_p: PhantomData<fn() -> R>,
}
impl<T, F, R> Map<T, F, R>
where
T: Service,
F: Fn(T::Response) -> R + Clone,
{
impl<T, F, R> Map<T, F, R> {
/// Create new `Map` combinator
pub fn new(service: T, f: F) -> Self {
Map { service, f }
pub fn new<Request>(service: T, f: F) -> Self
where
T: Service<Request>,
F: Fn(T::Response) -> R + Clone,
{
Map {
service,
f,
_p: PhantomData,
}
}
}
impl<T, F, R> Clone for Map<T, F, R>
where
T: Service + Clone,
F: Fn(T::Response) -> R + Clone,
T: Clone,
F: Clone,
{
fn clone(&self) -> Self {
Map {
service: self.service.clone(),
f: self.f.clone(),
_p: PhantomData,
}
}
}
impl<T, F, R> Service for Map<T, F, R>
impl<T, F, R, Request> Service<Request> for Map<T, F, R>
where
T: Service,
T: Service<Request>,
F: Fn(T::Response) -> R + Clone,
{
type Request = T::Request;
type Response = R;
type Error = T::Error;
type Future = MapFuture<T, F, R>;
type Future = MapFuture<T::Future, F, R>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
MapFuture::new(self.service.call(req), self.f.clone())
}
}
pub struct MapFuture<T, F, R>
where
T: Service,
F: Fn(T::Response) -> R,
T: Future,
F: Fn(T::Item) -> R,
{
f: F,
fut: T::Future,
fut: T,
}
impl<T, F, R> MapFuture<T, F, R>
where
T: Service,
F: Fn(T::Response) -> R,
T: Future,
F: Fn(T::Item) -> R,
{
fn new(fut: T::Future, f: F) -> Self {
fn new(fut: T, f: F) -> Self {
MapFuture { f, fut }
}
}
impl<T, F, R> Future for MapFuture<T, F, R>
where
T: Service,
F: Fn(T::Response) -> R,
T: Future,
F: Fn(T::Item) -> R,
{
type Item = R;
type Error = T::Error;
@ -98,8 +101,7 @@ mod tests {
struct Srv;
impl Service for Srv {
type Request = ();
impl Service<()> for Srv {
type Response = ();
type Error = ();
type Future = FutureResult<(), ()>;

View File

@ -1,86 +1,85 @@
use futures::{Future, Poll};
use tower_service::Service;
use std::marker::PhantomData;
/// Service for the `map_err` combinator, changing the type of a service's error.
///
/// This is created by the `ServiceExt::map_err` method.
pub struct MapErr<T, F, E>
where
T: Service,
F: Fn(T::Error) -> E + Clone,
{
pub struct MapErr<T, F, E> {
service: T,
f: F,
_p: PhantomData<E>,
}
impl<T, F, E> MapErr<T, F, E>
where
T: Service,
F: Fn(T::Error) -> E + Clone,
{
impl<T, F, E> MapErr<T, F, E> {
/// Create new `MapErr` combinator
pub fn new(service: T, f: F) -> Self {
MapErr { service, f }
pub fn new<Request>(service: T, f: F) -> Self
where
T: Service<Request>,
F: Fn(T::Error) -> E + Clone,
{
MapErr { service, f, _p: PhantomData }
}
}
impl<T, F, E> Clone for MapErr<T, F, E>
where
T: Service + Clone,
F: Fn(T::Error) -> E + Clone,
T: Clone,
F: Clone,
{
fn clone(&self) -> Self {
MapErr {
service: self.service.clone(),
f: self.f.clone(),
_p: PhantomData,
}
}
}
impl<T, F, E> Service for MapErr<T, F, E>
impl<T, F, E, Request> Service<Request> for MapErr<T, F, E>
where
T: Service,
T: Service<Request>,
F: Fn(T::Error) -> E + Clone,
{
type Request = T::Request;
type Response = T::Response;
type Error = E;
type Future = MapErrFuture<T, F, E>;
type Future = MapErrFuture<T::Future, F, E>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready().map_err(&self.f)
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
MapErrFuture::new(self.service.call(req), self.f.clone())
}
}
pub struct MapErrFuture<T, F, E>
where
T: Service,
T: Future,
F: Fn(T::Error) -> E,
{
f: F,
fut: T::Future,
fut: T,
}
impl<T, F, E> MapErrFuture<T, F, E>
where
T: Service,
T: Future,
F: Fn(T::Error) -> E,
{
fn new(fut: T::Future, f: F) -> Self {
fn new(fut: T, f: F) -> Self {
MapErrFuture { f, fut }
}
}
impl<T, F, E> Future for MapErrFuture<T, F, E>
where
T: Service,
T: Future,
F: Fn(T::Error) -> E,
{
type Item = T::Response;
type Item = T::Item;
type Error = E;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
@ -97,8 +96,7 @@ mod tests {
struct Srv;
impl Service for Srv {
type Request = ();
impl Service<()> for Srv {
type Response = ();
type Error = ();
type Future = FutureResult<(), ()>;

View File

@ -17,16 +17,19 @@ pub use self::map::Map;
pub use self::map_err::MapErr;
pub use self::then::Then;
impl<T: ?Sized> ServiceExt for T where T: Service {}
impl<T: ?Sized, Request> ServiceExt<Request> for T
where
T: Service<Request>
{}
/// An extension trait for `Service`s that provides a variety of convenient
/// adapters
pub trait ServiceExt: Service {
fn apply<F, R, Req>(self, f: F) -> Apply<Self, F, R, Req>
pub trait ServiceExt<Request>: Service<Request> {
fn apply<F, In, Out>(self, f: F) -> Apply<Self, F, In, Out, Request>
where
Self: Clone + Sized,
F: Fn(Req, Self) -> R,
R: IntoFuture<Error = Self::Error>,
Self: Service<Request> + Clone + Sized,
F: Fn(In, Self) -> Out,
Out: IntoFuture<Error = Self::Error>,
{
Apply::new(f, self)
}
@ -43,7 +46,7 @@ pub trait ServiceExt: Service {
fn and_then<B>(self, service: B) -> AndThen<Self, B>
where
Self: Sized,
B: Service<Request = Self::Response, Error = Self::Error> + Clone,
B: Service<Self::Response, Error = Self::Error> + Clone,
{
AndThen::new(self, service)
}
@ -69,7 +72,7 @@ pub trait ServiceExt: Service {
fn then<B>(self, service: B) -> Then<Self, B>
where
Self: Sized,
B: Service<Request = Result<Self::Response, Self::Error>, Error = Self::Error> + Clone,
B: Service<Result<Self::Response, Self::Error>, Error = Self::Error> + Clone,
{
Then::new(self, service)
}

View File

@ -10,21 +10,21 @@ pub struct Then<A, B> {
b: B,
}
impl<A, B> Then<A, B>
where
A: Service,
B: Service<Request = Result<A::Response, A::Error>, Error = A::Error> + Clone,
{
impl<A, B> Then<A, B> {
/// Create new `Then` combinator
pub fn new(a: A, b: B) -> Then<A, B> {
pub fn new<Request>(a: A, b: B) -> Then<A, B>
where
A: Service<Request>,
B: Service<Result<A::Response, A::Error>, Error = A::Error> + Clone,
{
Then { a, b }
}
}
impl<A, B> Clone for Then<A, B>
where
A: Service + Clone,
B: Service + Clone,
A: Clone,
B: Clone,
{
fn clone(&self) -> Self {
Then {
@ -34,54 +34,43 @@ where
}
}
impl<A, B> Service for Then<A, B>
impl<A, B, Request> Service<Request> for Then<A, B>
where
A: Service,
B: Service<Request = Result<A::Response, A::Error>, Error = A::Error> + Clone,
A: Service<Request>,
B: Service<Result<A::Response, A::Error>, Error = A::Error> + Clone,
{
type Request = A::Request;
type Response = B::Response;
type Error = B::Error;
type Future = ThenFuture<A, B>;
type Future = ThenFuture<A, B, Request>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
let _ = try_ready!(self.a.poll_ready());
self.b.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
ThenFuture::new(self.a.call(req), self.b.clone())
fn call(&mut self, req: Request) -> Self::Future {
ThenFuture {
fut_a: self.a.call(req),
b: self.b.clone(),
fut_b: None,
}
}
}
pub struct ThenFuture<A, B>
pub struct ThenFuture<A, B, Request>
where
A: Service,
B: Service<Request = Result<A::Response, A::Error>>,
A: Service<Request>,
B: Service<Result<A::Response, A::Error>>,
{
b: B,
fut_b: Option<B::Future>,
fut_a: A::Future,
}
impl<A, B> ThenFuture<A, B>
impl<A, B, Request> Future for ThenFuture<A, B, Request>
where
A: Service,
B: Service<Request = Result<A::Response, A::Error>>,
{
fn new(fut_a: A::Future, b: B) -> Self {
ThenFuture {
b,
fut_a,
fut_b: None,
}
}
}
impl<A, B> Future for ThenFuture<A, B>
where
A: Service,
B: Service<Request = Result<A::Response, A::Error>>,
A: Service<Request>,
B: Service<Result<A::Response, A::Error>>,
{
type Item = B::Response;
type Error = B::Error;
@ -116,8 +105,7 @@ mod tests {
use ServiceExt;
struct Srv1(Rc<Cell<usize>>);
impl Service for Srv1 {
type Request = Result<&'static str, &'static str>;
impl Service<Result<&'static str, &'static str>> for Srv1 {
type Response = &'static str;
type Error = ();
type Future = FutureResult<Self::Response, Self::Error>;
@ -127,7 +115,7 @@ mod tests {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Result<&'static str, &'static str>) -> Self::Future {
match req {
Ok(msg) => ok(msg),
Err(_) => err(()),
@ -138,8 +126,7 @@ mod tests {
#[derive(Clone)]
struct Srv2(Rc<Cell<usize>>);
impl Service for Srv2 {
type Request = Result<&'static str, ()>;
impl Service<Result<&'static str, ()>> for Srv2 {
type Response = (&'static str, &'static str);
type Error = ();
type Future = FutureResult<Self::Response, ()>;
@ -149,7 +136,7 @@ mod tests {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Result<&'static str, ()>) -> Self::Future {
match req {
Ok(msg) => ok((msg, "ok")),
Err(()) => ok(("srv2", "err")),

View File

@ -39,10 +39,9 @@ impl<T> OptionService<T> {
}
}
impl<T> Service for OptionService<T>
where T: Service,
impl<T, Request> Service<Request> for OptionService<T>
where T: Service<Request>,
{
type Request = T::Request;
type Response = T::Response;
type Error = Error<T::Error>;
type Future = ResponseFuture<T::Future>;
@ -55,7 +54,7 @@ where T: Service,
}
}
fn call(&mut self, request: Self::Request) -> Self::Future {
fn call(&mut self, request: Request) -> Self::Future {
let inner = self.inner.as_mut().map(|i| i.call(request));
ResponseFuture { inner }
}

View File

@ -8,22 +8,18 @@ pub struct NewServiceFn<T> {
// ===== impl NewServiceFn =====
impl<T, N> NewServiceFn<T>
where T: Fn() -> N,
N: Service,
{
impl<T> NewServiceFn<T> {
/// Returns a new `NewServiceFn` with the given closure.
pub fn new(f: T) -> Self {
NewServiceFn { f }
}
}
impl<T, R, S> NewService for NewServiceFn<T>
impl<T, R, S, Request> NewService<Request> for NewServiceFn<T>
where T: Fn() -> R,
R: IntoFuture<Item = S>,
S: Service,
S: Service<Request>,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Service = R::Item;

View File

@ -9,7 +9,7 @@ use tower_service::Service;
/// Binds new instances of a Service with a borrowed reference to the watched value.
pub trait Bind<T> {
type Service: Service;
type Service;
fn bind(&mut self, t: &T) -> Self::Service;
}
@ -56,18 +56,21 @@ impl<T, B: Bind<T>> WatchService<T, B> {
}
}
impl<T, B: Bind<T>> Service for WatchService<T, B> {
type Request = <B::Service as Service>::Request;
type Response = <B::Service as Service>::Response;
type Error = Error<<B::Service as Service>::Error>;
type Future = ResponseFuture<<B::Service as Service>::Future>;
impl<T, B, Request> Service<Request> for WatchService<T, B>
where
B: Bind<T>,
B::Service: Service<Request>,
{
type Response = <B::Service as Service<Request>>::Response;
type Error = Error<<B::Service as Service<Request>>::Error>;
type Future = ResponseFuture<<B::Service as Service<Request>>::Future>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
let _ = self.poll_rebind().map_err(Error::WatchError)?;
self.inner.poll_ready().map_err(Error::Inner)
}
fn call(&mut self, req: Self::Request) -> Self::Future {
fn call(&mut self, req: Request) -> Self::Future {
ResponseFuture(self.inner.call(req))
}
}
@ -76,7 +79,6 @@ impl<T, B: Bind<T>> Service for WatchService<T, B> {
impl<T, S, F> Bind<T> for F
where
S: Service,
for<'t> F: FnMut(&'t T) -> S,
{
type Service = S;
@ -109,8 +111,7 @@ mod tests {
#[test]
fn rebind() {
struct Svc(usize);
impl Service for Svc {
type Request = ();
impl Service<()> for Svc {
type Response = usize;
type Error = ();
type Future = future::FutureResult<usize, ()>;