Refactor errors (#46)
This patch does a bunch of refactoring, mostly around error types, but it also
paves the way to allow `Codec` to be used standalone.
* `Codec` (and `FramedRead` / `FramedWrite`) is broken out into a codec module.
* An h2-codec crate is created that re-exports the frame and codec modules.
* New error types are introduced in the internals:
  * `RecvError` represents errors caused by trying to receive a frame.
  * `SendError` represents errors caused by trying to send a frame.
  * `UserError` is an enum of potential errors caused by invalid usage
    by the user of the lib.
  * `ProtoError` is either a `Reason` or an `io::Error`. However it doesn't
    specify connection or stream level.
  * `h2::Error` is an opaque error type and is the only error type exposed
    by the public API (used to be `ConnectionError`).
There are misc code changes to enable this as well. The biggest is a new "sink"
API for `Codec`. It provides buffer which queues up a frame followed by flush
which writes everything that is queued. This departs from the `Sink` trait in
order to provide more accurate error values. For example, buffer can never fail
(but it will panic if `poll_ready` is not called first).
			
			
This commit is contained in:
		| @@ -1,10 +1,12 @@ | ||||
| use {client, server, frame, HeaderMap, ConnectionError}; | ||||
| use {client, server, frame, proto}; | ||||
| use frame::Reason; | ||||
| use codec::{RecvError, UserError}; | ||||
| use proto::*; | ||||
| use super::*; | ||||
|  | ||||
| use error::Reason::*; | ||||
| use futures::Sink; | ||||
| use http::HeaderMap; | ||||
|  | ||||
| use std::io; | ||||
| use std::marker::PhantomData; | ||||
|  | ||||
| #[derive(Debug)] | ||||
| @@ -41,14 +43,14 @@ pub(super) struct Recv<B, P> | ||||
|     /// Refused StreamId, this represents a frame that must be sent out. | ||||
|     refused: Option<StreamId>, | ||||
|  | ||||
|     _p: PhantomData<(B)>, | ||||
|     _p: PhantomData<B>, | ||||
| } | ||||
|  | ||||
| #[derive(Debug)] | ||||
| pub(super) enum Event<T> { | ||||
|     Headers(T), | ||||
|     Data(Bytes), | ||||
|     Trailers(::HeaderMap), | ||||
|     Trailers(HeaderMap), | ||||
| } | ||||
|  | ||||
| #[derive(Debug, Clone, Copy)] | ||||
| @@ -103,7 +105,7 @@ impl<B, P> Recv<B, P> | ||||
|     /// | ||||
|     /// Returns the stream state if successful. `None` if refused | ||||
|     pub fn open(&mut self, id: StreamId) | ||||
|         -> Result<Option<StreamId>, ConnectionError> | ||||
|         -> Result<Option<StreamId>, RecvError> | ||||
|     { | ||||
|         assert!(self.refused.is_none()); | ||||
|  | ||||
| @@ -123,7 +125,7 @@ impl<B, P> Recv<B, P> | ||||
|     pub fn recv_headers(&mut self, | ||||
|                         frame: frame::Headers, | ||||
|                         stream: &mut store::Ptr<B, P>) | ||||
|         -> Result<(), ProtoError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         trace!("opening stream; init_window={}", self.init_window_sz); | ||||
|         let is_initial = stream.state.recv_open(frame.is_end_stream())?; | ||||
| @@ -137,7 +139,7 @@ impl<B, P> Recv<B, P> | ||||
|                 self.next_stream_id = frame.stream_id(); | ||||
|                 self.next_stream_id.increment(); | ||||
|             } else { | ||||
|                 return Err(ProtoError::Connection(ProtocolError)); | ||||
|                 return Err(RecvError::Connection(ProtocolError)); | ||||
|             } | ||||
|  | ||||
|             // TODO: be smarter about this logic | ||||
| @@ -184,13 +186,13 @@ impl<B, P> Recv<B, P> | ||||
|     pub fn recv_trailers(&mut self, | ||||
|                          frame: frame::Headers, | ||||
|                          stream: &mut store::Ptr<B, P>) | ||||
|         -> Result<(), ProtoError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         // Transition the state | ||||
|         stream.state.recv_close()?; | ||||
|  | ||||
|         if stream.ensure_content_length_zero().is_err() { | ||||
|             return Err(ProtoError::Stream { | ||||
|             return Err(RecvError::Stream { | ||||
|                 id: stream.id, | ||||
|                 reason: ProtocolError, | ||||
|             }); | ||||
| @@ -205,11 +207,12 @@ impl<B, P> Recv<B, P> | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     /// Releases capacity back to the connection | ||||
|     pub fn release_capacity(&mut self, | ||||
|                             capacity: WindowSize, | ||||
|                             stream: &mut store::Ptr<B, P>, | ||||
|                             task: &mut Option<Task>) | ||||
|         -> Result<(), ConnectionError> | ||||
|         -> Result<(), UserError> | ||||
|     { | ||||
|         if capacity > stream.in_flight_recv_data { | ||||
|             // TODO: Handle error | ||||
| @@ -246,7 +249,7 @@ impl<B, P> Recv<B, P> | ||||
|     pub fn recv_data(&mut self, | ||||
|                      frame: frame::Data, | ||||
|                      stream: &mut store::Ptr<B, P>) | ||||
|         -> Result<(), ProtoError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         let sz = frame.payload().len(); | ||||
|  | ||||
| @@ -259,7 +262,7 @@ impl<B, P> Recv<B, P> | ||||
|         if !stream.state.is_recv_streaming() { | ||||
|             // Receiving a DATA frame when not expecting one is a protocol | ||||
|             // error. | ||||
|             return Err(ProtoError::Connection(ProtocolError)); | ||||
|             return Err(RecvError::Connection(ProtocolError)); | ||||
|         } | ||||
|  | ||||
|         trace!("recv_data; size={}; connection={}; stream={}", | ||||
| @@ -268,7 +271,7 @@ impl<B, P> Recv<B, P> | ||||
|         // Ensure that there is enough capacity on the connection before acting | ||||
|         // on the stream. | ||||
|         if self.flow.window_size() < sz || stream.recv_flow.window_size() < sz { | ||||
|             return Err(ProtoError::Connection(FlowControlError)); | ||||
|             return Err(RecvError::Connection(FlowControlError)); | ||||
|         } | ||||
|  | ||||
|         // Update connection level flow control | ||||
| @@ -281,7 +284,7 @@ impl<B, P> Recv<B, P> | ||||
|         stream.in_flight_recv_data += sz; | ||||
|  | ||||
|         if stream.dec_content_length(frame.payload().len()).is_err() { | ||||
|             return Err(ProtoError::Stream { | ||||
|             return Err(RecvError::Stream { | ||||
|                 id: stream.id, | ||||
|                 reason: ProtocolError, | ||||
|             }); | ||||
| @@ -289,14 +292,14 @@ impl<B, P> Recv<B, P> | ||||
|  | ||||
|         if frame.is_end_stream() { | ||||
|             if stream.ensure_content_length_zero().is_err() { | ||||
|                 return Err(ProtoError::Stream { | ||||
|                 return Err(RecvError::Stream { | ||||
|                     id: stream.id, | ||||
|                     reason: ProtocolError, | ||||
|                 }); | ||||
|             } | ||||
|  | ||||
|             if stream.state.recv_close().is_err() { | ||||
|                 return Err(ProtoError::Connection(ProtocolError)); | ||||
|                 return Err(RecvError::Connection(ProtocolError)); | ||||
|             } | ||||
|         } | ||||
|  | ||||
| @@ -314,13 +317,14 @@ impl<B, P> Recv<B, P> | ||||
|                              send: &Send<B, P>, | ||||
|                              stream: store::Key, | ||||
|                              store: &mut Store<B, P>) | ||||
|         -> Result<(), ConnectionError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         // First, make sure that the values are legit | ||||
|         self.ensure_can_reserve(frame.promised_id())?; | ||||
|  | ||||
|         // Make sure that the stream state is valid | ||||
|         store[stream].state.ensure_recv_open()?; | ||||
|         store[stream].state.ensure_recv_open() | ||||
|             .map_err(|e| e.into_connection_recv_error())?; | ||||
|  | ||||
|         // TODO: Streams in the reserved states do not count towards the concurrency | ||||
|         // limit. However, it seems like there should be a cap otherwise this | ||||
| @@ -361,18 +365,19 @@ impl<B, P> Recv<B, P> | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     pub fn ensure_not_idle(&self, id: StreamId) -> Result<(), ConnectionError> { | ||||
|     /// Ensures that `id` is not in the `Idle` state. | ||||
|     pub fn ensure_not_idle(&self, id: StreamId) -> Result<(), Reason> { | ||||
|         if id >= self.next_stream_id { | ||||
|             return Err(ProtocolError.into()); | ||||
|             return Err(ProtocolError); | ||||
|         } | ||||
|  | ||||
|         Ok(()) | ||||
|     } | ||||
|  | ||||
|     pub fn recv_reset(&mut self, frame: frame::Reset, stream: &mut Stream<B, P>) | ||||
|         -> Result<(), ConnectionError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         let err = ConnectionError::Proto(frame.reason()); | ||||
|         let err = proto::Error::Proto(frame.reason()); | ||||
|  | ||||
|         // Notify the stream | ||||
|         stream.state.recv_err(&err); | ||||
| @@ -381,7 +386,7 @@ impl<B, P> Recv<B, P> | ||||
|     } | ||||
|  | ||||
|     /// Handle a received error | ||||
|     pub fn recv_err(&mut self, err: &ConnectionError, stream: &mut Stream<B, P>) { | ||||
|     pub fn recv_err(&mut self, err: &proto::Error, stream: &mut Stream<B, P>) { | ||||
|         // Receive an error | ||||
|         stream.state.recv_err(err); | ||||
|  | ||||
| @@ -415,17 +420,17 @@ impl<B, P> Recv<B, P> | ||||
|  | ||||
|     /// Returns true if the remote peer can initiate a stream with the given ID. | ||||
|     fn ensure_can_open(&self, id: StreamId) | ||||
|         -> Result<(), ConnectionError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         if !P::is_server() { | ||||
|             // Remote is a server and cannot open streams. PushPromise is | ||||
|             // registered by reserving, so does not go through this path. | ||||
|             return Err(ProtocolError.into()); | ||||
|             return Err(RecvError::Connection(ProtocolError)); | ||||
|         } | ||||
|  | ||||
|         // Ensure that the ID is a valid server initiated ID | ||||
|         if !id.is_client_initiated() { | ||||
|             return Err(ProtocolError.into()); | ||||
|             return Err(RecvError::Connection(ProtocolError)); | ||||
|         } | ||||
|  | ||||
|         Ok(()) | ||||
| @@ -433,16 +438,16 @@ impl<B, P> Recv<B, P> | ||||
|  | ||||
|     /// Returns true if the remote peer can reserve a stream with the given ID. | ||||
|     fn ensure_can_reserve(&self, promised_id: StreamId) | ||||
|         -> Result<(), ConnectionError> | ||||
|         -> Result<(), RecvError> | ||||
|     { | ||||
|         // TODO: Are there other rules? | ||||
|         if P::is_server() { | ||||
|             // The remote is a client and cannot reserve | ||||
|             return Err(ProtocolError.into()); | ||||
|             return Err(RecvError::Connection(ProtocolError)); | ||||
|         } | ||||
|  | ||||
|         if !promised_id.is_server_initiated() { | ||||
|             return Err(ProtocolError.into()); | ||||
|             return Err(RecvError::Connection(ProtocolError)); | ||||
|         } | ||||
|  | ||||
|         Ok(()) | ||||
| @@ -450,31 +455,28 @@ impl<B, P> Recv<B, P> | ||||
|  | ||||
|     /// Send any pending refusals. | ||||
|     pub fn send_pending_refusal<T>(&mut self, dst: &mut Codec<T, Prioritized<B>>) | ||||
|         -> Poll<(), ConnectionError> | ||||
|         -> Poll<(), io::Error> | ||||
|         where T: AsyncWrite, | ||||
|     { | ||||
|         if let Some(stream_id) = self.refused.take() { | ||||
|         if let Some(stream_id) = self.refused { | ||||
|             try_ready!(dst.poll_ready()); | ||||
|  | ||||
|             // Create the RST_STREAM frame | ||||
|             let frame = frame::Reset::new(stream_id, RefusedStream); | ||||
|  | ||||
|             match dst.start_send(frame.into())? { | ||||
|                 AsyncSink::Ready => { | ||||
|                     self.reset(stream_id, RefusedStream); | ||||
|                     return Ok(Async::Ready(())); | ||||
|                 } | ||||
|                 AsyncSink::NotReady(_) => { | ||||
|                     self.refused = Some(stream_id); | ||||
|                     return Ok(Async::NotReady); | ||||
|                 } | ||||
|             } | ||||
|             // Buffer the frame | ||||
|             dst.buffer(frame.into()).ok().expect("invalid RST_STREAM frame"); | ||||
|         } | ||||
|  | ||||
|         self.refused = None; | ||||
|  | ||||
|         Ok(Async::Ready(())) | ||||
|     } | ||||
|  | ||||
|     pub fn poll_complete<T>(&mut self, | ||||
|                             store: &mut Store<B, P>, | ||||
|                             dst: &mut Codec<T, Prioritized<B>>) | ||||
|         -> Poll<(), ConnectionError> | ||||
|         -> Poll<(), io::Error> | ||||
|         where T: AsyncWrite, | ||||
|     { | ||||
|         // Send any pending connection level window updates | ||||
| @@ -488,7 +490,7 @@ impl<B, P> Recv<B, P> | ||||
|  | ||||
|     /// Send connection level window update | ||||
|     fn send_connection_window_update<T>(&mut self, dst: &mut Codec<T, Prioritized<B>>) | ||||
|         -> Poll<(), ConnectionError> | ||||
|         -> Poll<(), io::Error> | ||||
|         where T: AsyncWrite, | ||||
|     { | ||||
|         let incr = self.flow.unclaimed_capacity(); | ||||
| @@ -496,11 +498,14 @@ impl<B, P> Recv<B, P> | ||||
|         if incr > 0 { | ||||
|             let frame = frame::WindowUpdate::new(StreamId::zero(), incr); | ||||
|  | ||||
|             if dst.start_send(frame.into())?.is_ready() { | ||||
|                 self.flow.inc_window(incr).ok().expect("unexpected flow control state"); | ||||
|             } else { | ||||
|                 return Ok(Async::NotReady); | ||||
|             } | ||||
|             // Ensure the codec has capacity | ||||
|             try_ready!(dst.poll_ready()); | ||||
|  | ||||
|             // Buffer the WINDOW_UPDATE frame | ||||
|             dst.buffer(frame.into()).ok().expect("invalid WINDOW_UPDATE frame"); | ||||
|  | ||||
|             // Update flow control | ||||
|             self.flow.inc_window(incr).ok().expect("unexpected flow control state"); | ||||
|         } | ||||
|  | ||||
|         Ok(().into()) | ||||
| @@ -511,7 +516,7 @@ impl<B, P> Recv<B, P> | ||||
|     pub fn send_stream_window_updates<T>(&mut self, | ||||
|                                          store: &mut Store<B, P>, | ||||
|                                          dst: &mut Codec<T, Prioritized<B>>) | ||||
|         -> Poll<(), ConnectionError> | ||||
|         -> Poll<(), io::Error> | ||||
|         where T: AsyncWrite, | ||||
|     { | ||||
|         loop { | ||||
| @@ -534,10 +539,11 @@ impl<B, P> Recv<B, P> | ||||
|             let incr = stream.recv_flow.unclaimed_capacity(); | ||||
|  | ||||
|             if incr > 0 { | ||||
|                 // Create the WINDOW_UPDATE frame | ||||
|                 let frame = frame::WindowUpdate::new(stream.id, incr); | ||||
|                 let res = dst.start_send(frame.into())?; | ||||
|  | ||||
|                 assert!(res.is_ready()); | ||||
|                 // Buffer it | ||||
|                 dst.buffer(frame.into()).ok().expect("invalid WINDOW_UPDATE frame"); | ||||
|             } | ||||
|         } | ||||
|     } | ||||
| @@ -548,8 +554,9 @@ impl<B, P> Recv<B, P> | ||||
|     } | ||||
|  | ||||
|     pub fn poll_data(&mut self, stream: &mut Stream<B, P>) | ||||
|         -> Poll<Option<Bytes>, ConnectionError> | ||||
|         -> Poll<Option<Bytes>, proto::Error> | ||||
|     { | ||||
|         // TODO: Return error when the stream is reset | ||||
|         match stream.pending_recv.pop_front(&mut self.buffer) { | ||||
|             Some(Event::Data(payload)) => { | ||||
|                 Ok(Some(payload).into()) | ||||
| @@ -575,7 +582,7 @@ impl<B, P> Recv<B, P> | ||||
|     } | ||||
|  | ||||
|     pub fn poll_trailers(&mut self, stream: &mut Stream<B, P>) | ||||
|         -> Poll<Option<HeaderMap>, ConnectionError> | ||||
|         -> Poll<Option<HeaderMap>, proto::Error> | ||||
|     { | ||||
|         match stream.pending_recv.pop_front(&mut self.buffer) { | ||||
|             Some(Event::Trailers(trailers)) => { | ||||
| @@ -599,10 +606,6 @@ impl<B, P> Recv<B, P> | ||||
|             } | ||||
|         } | ||||
|     } | ||||
|  | ||||
|     fn reset(&mut self, _stream_id: StreamId, _reason: Reason) { | ||||
|         unimplemented!(); | ||||
|     } | ||||
| } | ||||
|  | ||||
| impl<B> Recv<B, server::Peer> | ||||
| @@ -610,15 +613,10 @@ impl<B> Recv<B, server::Peer> | ||||
| { | ||||
|     /// TODO: Should this fn return `Result`? | ||||
|     pub fn take_request(&mut self, stream: &mut store::Ptr<B, server::Peer>) | ||||
|         -> Result<Request<()>, ConnectionError> | ||||
|         -> Request<()> | ||||
|     { | ||||
|         match stream.pending_recv.pop_front(&mut self.buffer) { | ||||
|             Some(Event::Headers(request)) => Ok(request), | ||||
|                 /* | ||||
|                 // TODO: This error should probably be caught on receipt of the | ||||
|                 // frame vs. now. | ||||
|                 Ok(server::Peer::convert_poll_message(frame)?) | ||||
|                 */ | ||||
|             Some(Event::Headers(request)) => request, | ||||
|             _ => panic!(), | ||||
|         } | ||||
|     } | ||||
| @@ -628,7 +626,7 @@ impl<B> Recv<B, client::Peer> | ||||
|     where B: Buf, | ||||
| { | ||||
|     pub fn poll_response(&mut self, stream: &mut store::Ptr<B, client::Peer>) | ||||
|         -> Poll<Response<()>, ConnectionError> { | ||||
|         -> Poll<Response<()>, proto::Error> { | ||||
|         // If the buffer is not empty, then the first frame must be a HEADERS | ||||
|         // frame or the user violated the contract. | ||||
|         match stream.pending_recv.pop_front(&mut self.buffer) { | ||||
|   | ||||
		Reference in New Issue
	
	Block a user