Adopt external tower-grpc and tower-h2 deps #225)

The conduit repo includes several library projects that have since been
moved into external repos, including `tower-grpc` and `tower-h2`.

This change removes these vendored libraries in favor of using the new
external crates.
This commit is contained in:
Eliza Weisman 2018-02-01 11:57:02 -08:00 committed by Oliver Gould
parent 53299f6c78
commit b56cc883c1
68 changed files with 629 additions and 9613 deletions

733
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@ -1,11 +1,6 @@
[workspace]
members = [
"codegen",
"futures-mpsc-lossy",
"proxy",
"tower-router",
"tower-grpc",
"tower-grpc-build",
"tower-grpc-examples",
"tower-h2",
]

View File

@ -1,7 +0,0 @@
[package]
name = "codegen"
version = "0.2.0"
authors = ["Carl Lerche <me@carllerche.com>"]
[dependencies]
ordermap = "0.3.0"

File diff suppressed because it is too large Load Diff

View File

@ -1,240 +0,0 @@
extern crate codegen;
use codegen::Scope;
#[test]
fn empty_scope() {
let scope = Scope::new();
assert_eq!(scope.to_string(), "");
}
#[test]
fn single_struct() {
let mut scope = Scope::new();
scope.structure("Foo")
.field("one", "usize")
.field("two", "String");
let expect = r#"
struct Foo {
one: usize,
two: String,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn two_structs() {
let mut scope = Scope::new();
scope.structure("Foo")
.field("one", "usize")
.field("two", "String");
scope.structure("Bar")
.field("hello", "World");
let expect = r#"
struct Foo {
one: usize,
two: String,
}
struct Bar {
hello: World,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_with_derive() {
let mut scope = Scope::new();
scope.structure("Foo")
.derive("Debug").derive("Clone")
.field("one", "usize")
.field("two", "String");
let expect = r#"
#[derive(Debug, Clone)]
struct Foo {
one: usize,
two: String,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_with_generics_1() {
let mut scope = Scope::new();
scope.structure("Foo")
.generic("T")
.generic("U")
.field("one", "T")
.field("two", "U");
let expect = r#"
struct Foo<T, U> {
one: T,
two: U,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_with_generics_2() {
let mut scope = Scope::new();
scope.structure("Foo")
.generic("T, U")
.field("one", "T")
.field("two", "U");
let expect = r#"
struct Foo<T, U> {
one: T,
two: U,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_with_generics_3() {
let mut scope = Scope::new();
scope.structure("Foo")
.generic("T: Win, U")
.field("one", "T")
.field("two", "U");
let expect = r#"
struct Foo<T: Win, U> {
one: T,
two: U,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_where_clause_1() {
let mut scope = Scope::new();
scope.structure("Foo")
.generic("T")
.bound("T", "Foo")
.field("one", "T");
let expect = r#"
struct Foo<T>
where T: Foo,
{
one: T,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_where_clause_2() {
let mut scope = Scope::new();
scope.structure("Foo")
.generic("T, U")
.bound("T", "Foo")
.bound("U", "Baz")
.field("one", "T")
.field("two", "U");
let expect = r#"
struct Foo<T, U>
where T: Foo,
U: Baz,
{
one: T,
two: U,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_doc() {
let mut scope = Scope::new();
scope.structure("Foo")
.doc("Hello, this is a doc string\n\
that continues on another line.")
.field("one", "T");
let expect = r#"
/// Hello, this is a doc string
/// that continues on another line.
struct Foo {
one: T,
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_in_mod() {
let mut scope = Scope::new();
{
let module = scope.module("foo");
module.structure("Foo")
.doc("Hello some docs")
.derive("Debug")
.generic("T, U")
.bound("T", "SomeBound")
.bound("U", "SomeOtherBound")
.field("one", "T")
.field("two", "U")
;
}
let expect = r#"
mod foo {
/// Hello some docs
#[derive(Debug)]
struct Foo<T, U>
where T: SomeBound,
U: SomeOtherBound,
{
one: T,
two: U,
}
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}
#[test]
fn struct_mod_import() {
let mut scope = Scope::new();
scope.module("foo")
.import("bar", "Bar")
.structure("Foo")
.field("bar", "Bar")
;
let expect = r#"
mod foo {
use bar::Bar;
struct Foo {
bar: Bar,
}
}"#;
assert_eq!(scope.to_string(), &expect[1..]);
}

View File

@ -10,6 +10,7 @@ chrono = "0.4"
domain = "0.2.2"
env_logger = "0.4"
futures = "0.1"
h2 = "0.1"
http = "0.1"
httparse = "1.2"
hyper = { version = "0.11.15", features = ["compat"] }
@ -21,33 +22,34 @@ url = "1.5"
tokio-core = "0.1"
tokio-io = "0.1"
prost = "0.2"
prost-derive = "0.2"
prost-types = "0.2"
prost = "0.3.0"
prost-derive = "0.3.0"
prost-types = "0.3.0"
abstract-ns = "0.4"
ns-dns-tokio = "0.4"
#futures-watch = { git = "https://github.com/carllerche/better-future" }
h2 = { git = "https://github.com/carllerche/h2" }
tokio-connect = { git = "https://github.com/carllerche/tokio-connect" }
tower = { git = "https://github.com/tower-rs/tower" }
tower-balance = { git = "https://github.com/tower-rs/tower" }
tower-buffer = { git = "https://github.com/tower-rs/tower" }
tower-discover = { git = "https://github.com/tower-rs/tower" }
tower-grpc = { git = "https://github.com/tower-rs/tower-grpc" }
tower-h2 = { git = "https://github.com/tower-rs/tower-h2" }
tower-reconnect = { git = "https://github.com/tower-rs/tower" }
tower-util = { git = "https://github.com/tower-rs/tower" }
futures-mpsc-lossy = { path = "../futures-mpsc-lossy" }
tower-router = { path = "../tower-router" }
tower-grpc = { path = "../tower-grpc" }
tower-h2 = { path = "../tower-h2" }
[target.'cfg(target_os = "linux")'.dependencies]
libc = "0.2"
[build-dependencies]
tower-grpc-build = { path = "../tower-grpc-build" }
tower-grpc-build = { git = "https://github.com/tower-rs/tower-grpc" }
[dev-dependencies]
# Quickcheck 0.4.1, on crates.io, is missng useful Arbitrary implementations that exist on

View File

@ -1,4 +1,3 @@
use std::io;
use std::marker::PhantomData;
use std::net::SocketAddr;
use std::sync::Arc;
@ -7,13 +6,14 @@ use std::time::Duration;
use http;
use tokio_core::reactor::Handle;
use tower;
use tower_h2;
use tower_reconnect::{self, Reconnect};
use tower_reconnect::Reconnect;
use control;
use ctx;
use telemetry;
use transparency;
use telemetry::{self, sensor};
use transparency::{self, HttpBody};
use transport;
use ::timeout::Timeout;
@ -48,15 +48,15 @@ pub enum Protocol {
Http2
}
type Service<B> = Reconnect<
telemetry::sensor::NewHttp<
transparency::Client<
telemetry::sensor::Connect<transport::TimeoutConnect<transport::Connect>>,
B,
>,
B,
transparency::HttpBody,
>,
pub type Service<B> = Reconnect<NewHttp<B>>;
pub type NewHttp<B> = sensor::NewHttp<Client<B>, B, HttpBody>;
pub type HttpResponse = http::Response<sensor::http::ResponseBody<HttpBody>>;
pub type Client<B> = transparency::Client<
sensor::Connect<transport::TimeoutConnect<transport::Connect>>,
B,
>;
impl<B> Bind<(), B> {
@ -189,11 +189,8 @@ where
B: tower_h2::Body + 'static,
{
type Request = http::Request<B>;
type Response = http::Response<telemetry::sensor::http::ResponseBody<transparency::HttpBody>>;
type Error = tower_reconnect::Error<
tower_h2::client::Error,
tower_h2::client::ConnectError<transport::TimeoutError<io::Error>>,
>;
type Response = HttpResponse;
type Error = <Service<B> as tower::Service>::Error;
type Service = Service<B>;
type BindError = ();

View File

@ -1,63 +0,0 @@
use std::fmt;
use std::marker::PhantomData;
use bytes::{Buf, BufMut};
use prost::{DecodeError, Message};
use tower_grpc::client::codec::{Codec, DecodeBuf, EncodeBuf};
/// A protobuf codec.
pub struct Protobuf<T, U>(PhantomData<(T, U)>);
impl<T, U> Protobuf<T, U> {
pub fn new() -> Self {
Protobuf(PhantomData)
}
}
impl<T, U> Clone for Protobuf<T, U> {
fn clone(&self) -> Self {
Protobuf(PhantomData)
}
}
impl<T, U> fmt::Debug for Protobuf<T, U> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.write_str("Protobuf")
}
}
impl<T: Message, U: Message + Default> Codec for Protobuf<T, U> {
const CONTENT_TYPE: &'static str = "application/grpc+proto";
type Encode = T;
type Decode = U;
// never errors
type EncodeError = Void;
type DecodeError = DecodeError;
fn encode(&mut self, msg: Self::Encode, buf: &mut EncodeBuf) -> Result<(), Self::EncodeError> {
let len = msg.encoded_len();
if buf.remaining_mut() < len {
buf.reserve(len);
}
// prost says the only error from `Message::encode` is if there is not
// enough space in the buffer.
msg.encode(buf).expect("buf space was reserved");
Ok(())
}
fn decode(&mut self, buf: &mut DecodeBuf) -> Result<Self::Decode, Self::DecodeError> {
trace!("decode; bytes={}", buf.remaining());
match Message::decode(buf) {
Ok(msg) => Ok(msg),
Err(err) => {
debug!("decode error: {:?}", err);
Err(err)
}
}
}
}
/// Can never be instantiated.
pub enum Void {}

View File

@ -1,26 +1,21 @@
use std::collections::{HashSet, VecDeque};
use std::collections::hash_map::{Entry, HashMap};
use std::net::SocketAddr;
use std::fmt;
use futures::{Async, Future, Poll, Stream};
use futures::sync::mpsc;
use tower::Service;
use tower_h2::{HttpService, BoxBody, RecvBody};
use tower_discover::{Change, Discover};
use tower_grpc;
use tower_grpc as grpc;
use fully_qualified_authority::FullyQualifiedAuthority;
use super::codec::Protobuf;
use super::pb::common::{Destination, TcpAddress};
use super::pb::proxy::destination::Update as PbUpdate;
use super::pb::proxy::destination::client::Destination as DestinationSvc;
use super::pb::proxy::destination::client::destination_methods::Get as GetRpc;
use super::pb::proxy::destination::update::Update as PbUpdate2;
pub type ClientBody = ::tower_grpc::client::codec::EncodingBody<
Protobuf<Destination, PbUpdate>,
::tower_grpc::client::codec::Unary<Destination>,
>;
use super::pb::proxy::destination::client::{Destination as DestinationSvc};
/// A handle to start watching a destination for address changes.
#[derive(Clone, Debug)]
@ -41,18 +36,14 @@ pub struct Background {
rx: mpsc::UnboundedReceiver<(FullyQualifiedAuthority, mpsc::UnboundedSender<Update>)>,
}
type DiscoveryWatch<F> = DestinationSet<
tower_grpc::client::Streaming<
tower_grpc::client::ResponseFuture<Protobuf<Destination, PbUpdate>, F>,
tower_grpc::client::codec::DecodingBody<Protobuf<Destination, PbUpdate>>,
>,
>;
/// A future returned from `Background::work()`, doing the work of talking to
/// the controller destination API.
#[derive(Debug)]
pub struct DiscoveryWork<F> {
destinations: HashMap<FullyQualifiedAuthority, DiscoveryWatch<F>>,
// TODO: debug impl
pub struct DiscoveryWork<T: HttpService<ResponseBody = RecvBody>> {
destinations: HashMap<
FullyQualifiedAuthority,
DestinationSet<T>
>,
/// A queue of authorities that need to be reconnected.
reconnects: VecDeque<FullyQualifiedAuthority>,
/// The Destination.Get RPC client service.
@ -62,14 +53,44 @@ pub struct DiscoveryWork<F> {
rx: mpsc::UnboundedReceiver<(FullyQualifiedAuthority, mpsc::UnboundedSender<Update>)>,
}
#[derive(Debug)]
struct DestinationSet<R> {
struct DestinationSet<T: HttpService<ResponseBody = RecvBody>> {
addrs: HashSet<SocketAddr>,
needs_reconnect: bool,
rx: R,
rx: UpdateRx<T>,
txs: Vec<mpsc::UnboundedSender<Update>>,
}
/// Receiver for destination set updates.
///
/// The destination RPC returns a `ResponseFuture` whose item is a
/// `Response<Stream>`, so this type holds the state of that RPC call ---
/// either we're waiting for the future, or we have a stream --- and allows
/// us to implement `Stream` regardless of whether the RPC has returned yet
/// or not.
///
/// Polling an `UpdateRx` polls the wrapped future while we are
/// `Waiting`, and the `Stream` if we are `Streaming`. If the future is `Ready`,
/// then we switch states to `Streaming`.
enum UpdateRx<T: HttpService<ResponseBody = RecvBody>> {
Waiting(UpdateRsp<T::Future>),
Streaming(grpc::Streaming<PbUpdate, T::ResponseBody>),
}
type UpdateRsp<F> =
grpc::client::server_streaming::ResponseFuture<PbUpdate, F>;
/// Wraps the error types returned by `UpdateRx` polls.
///
/// An `UpdateRx` error is either the error type of the `Future` in the
/// `UpdateRx::Waiting` state, or the `Stream` in the `UpdateRx::Streaming`
/// state.
// TODO: impl Error?
#[derive(Debug)]
enum RxError<T> {
Future(grpc::Error<T>),
Stream(grpc::Error),
}
#[derive(Debug)]
enum Update {
Insert(SocketAddr),
@ -168,7 +189,10 @@ where
impl Background {
/// Bind this handle to start talking to the controller API.
pub fn work<F>(self) -> DiscoveryWork<F> {
pub fn work<T>(self) -> DiscoveryWork<T>
where T: HttpService<RequestBody = BoxBody, ResponseBody = RecvBody>,
T::Error: fmt::Debug,
{
DiscoveryWork {
destinations: HashMap::new(),
reconnects: VecDeque::new(),
@ -180,20 +204,12 @@ impl Background {
// ==== impl DiscoveryWork =====
impl<F> DiscoveryWork<F>
impl<T> DiscoveryWork<T>
where
F: Future<Item = ::http::Response<::tower_h2::RecvBody>>,
F::Error: ::std::fmt::Debug,
T: HttpService<RequestBody = BoxBody, ResponseBody = RecvBody>,
T::Error: fmt::Debug,
{
pub fn poll_rpc<S>(&mut self, client: &mut S)
where
S: Service<
Request = ::http::Request<ClientBody>,
Response = F::Item,
Error = F::Error,
Future = F,
>,
{
pub fn poll_rpc(&mut self, client: &mut T) {
// This loop is make sure any streams that were found disconnected
// in `poll_destinations` while the `rpc` service is ready should
// be reconnected now, otherwise the task would just sleep...
@ -208,15 +224,7 @@ where
}
}
fn poll_new_watches<S>(&mut self, mut client: &mut S)
where
S: Service<
Request = ::http::Request<ClientBody>,
Response = F::Item,
Error = F::Error,
Future = F,
>,
{
fn poll_new_watches(&mut self, client: &mut T) {
loop {
// if rpc service isn't ready, not much we can do...
match client.poll_ready() {
@ -239,8 +247,6 @@ where
continue;
}
let grpc = tower_grpc::Client::new(Protobuf::new(), &mut client);
let mut rpc = GetRpc::new(grpc);
// check for any new watches
match self.rx.poll() {
Ok(Async::Ready(Some((auth, tx)))) => {
@ -260,7 +266,11 @@ where
scheme: "k8s".into(),
path: vac.key().without_trailing_dot().into(),
};
let stream = DestinationSvc::new(&mut rpc).get(req);
// TODO: Can grpc::Request::new be removed?
let mut svc = DestinationSvc::new(client.lift_ref());
let response = svc.get(grpc::Request::new(req));
let stream = UpdateRx::Waiting(response);
vac.insert(DestinationSet {
addrs: HashSet::new(),
needs_reconnect: false,
@ -281,18 +291,8 @@ where
}
/// Tries to reconnect next watch stream. Returns true if reconnection started.
fn poll_reconnect<S>(&mut self, client: &mut S) -> bool
where
S: Service<
Request = ::http::Request<ClientBody>,
Response = F::Item,
Error = F::Error,
Future = F,
>,
{
fn poll_reconnect(&mut self, client: &mut T) -> bool {
debug_assert!(self.rpc_ready);
let grpc = tower_grpc::Client::new(Protobuf::new(), client);
let mut rpc = GetRpc::new(grpc);
while let Some(auth) = self.reconnects.pop_front() {
if let Some(set) = self.destinations.get_mut(&auth) {
@ -301,7 +301,9 @@ where
scheme: "k8s".into(),
path: auth.without_trailing_dot().into(),
};
set.rx = DestinationSvc::new(&mut rpc).get(req);
let mut svc = DestinationSvc::new(client.lift_ref());
let response = svc.get(grpc::Request::new(req));
set.rx = UpdateRx::Waiting(response);
set.needs_reconnect = false;
return true;
} else {
@ -317,6 +319,7 @@ where
continue;
}
let needs_reconnect = 'set: loop {
match set.rx.poll() {
Ok(Async::Ready(Some(update))) => match update.update {
Some(PbUpdate2::Add(a_set)) => for addr in a_set.addrs {
@ -356,6 +359,7 @@ where
break 'set true;
}
}
};
if needs_reconnect {
set.needs_reconnect = true;
@ -383,6 +387,33 @@ where
}
}
// ===== impl UpdateRx =====
impl<T> Stream for UpdateRx<T>
where T: HttpService<RequestBody = BoxBody, ResponseBody = RecvBody>,
T::Error: fmt::Debug,
{
type Item = PbUpdate;
type Error = RxError<T::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
// this is not ideal.
let stream = match *self {
UpdateRx::Waiting(ref mut future) => match future.poll() {
Ok(Async::Ready(response)) => response.into_inner(),
Ok(Async::NotReady) => return Ok(Async::NotReady),
Err(e) => return Err(RxError::Future(e)),
},
UpdateRx::Streaming(ref mut stream) =>
return stream.poll().map_err(RxError::Stream),
};
*self = UpdateRx::Streaming(stream);
self.poll()
}
}
// ===== impl RxError =====
fn pb_to_sock_addr(pb: TcpAddress) -> Option<SocketAddr> {
use super::pb::common::ip_address::Ip;
use std::net::{Ipv4Addr, Ipv6Addr};
@ -434,4 +465,4 @@ fn pb_to_sock_addr(pb: TcpAddress) -> Option<SocketAddr> {
},
None => None,
}
}
}

View File

@ -1,4 +1,3 @@
use std::marker::PhantomData;
use std::time::{Duration, Instant};
use bytes::Bytes;
@ -6,7 +5,7 @@ use futures::{future, Async, Future, Poll, Stream};
use h2;
use http;
use tokio_core::reactor::{
Handle,
Handle,
// TODO: would rather just have Backoff in a separate file so this
// renaming import is not necessary.
Timeout as ReactorTimeout
@ -21,7 +20,6 @@ use fully_qualified_authority::FullyQualifiedAuthority;
use transport::LookupAddressAndConnect;
use timeout::Timeout;
mod codec;
pub mod discovery;
mod observe;
pub mod pb;
@ -91,7 +89,7 @@ impl Background {
executor,
);
let h2_client = tower_h2::client::Client::new(
let h2_client = tower_h2::client::Connect::new(
connect,
h2::client::Builder::default(),
::logging::context_executor(ctx, executor.clone()),
@ -100,6 +98,7 @@ impl Background {
let reconnect = Reconnect::new(h2_client);
let backoff = Backoff::new(reconnect, Duration::from_secs(5), executor);
// TODO: Use AddOrigin in tower-http
AddOrigin::new(scheme, authority, backoff)
};
@ -108,11 +107,12 @@ impl Background {
let fut = future::poll_fn(move || {
trace!("poll rpc services");
disco.poll_rpc(&mut EnumService(&mut client, PhantomData));
telemetry.poll_rpc(&mut EnumService(&mut client, PhantomData));
disco.poll_rpc(&mut client);
telemetry.poll_rpc(&mut client);
Ok(Async::NotReady)
});
Box::new(fut)
}
}
@ -215,73 +215,3 @@ where
}
}
// ===== impl EnumService =====
struct EnumService<S, B>(S, PhantomData<B>);
impl<S, B> Service for EnumService<S, B>
where
S: Service<Request = http::Request<GrpcEncodingBody>>,
B: Into<GrpcEncodingBody>,
{
type Request = http::Request<B>;
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.0.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let (head, body) = req.into_parts();
self.0.call(http::Request::from_parts(head, body.into()))
}
}
enum GrpcEncodingBody {
TelemetryReport(self::telemetry::ClientBody),
DestinationGet(self::discovery::ClientBody),
}
impl tower_h2::Body for GrpcEncodingBody {
type Data = Bytes;
#[inline]
fn is_end_stream(&self) -> bool {
match *self {
GrpcEncodingBody::TelemetryReport(ref b) => b.is_end_stream(),
GrpcEncodingBody::DestinationGet(ref b) => b.is_end_stream(),
}
}
#[inline]
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error> {
match *self {
GrpcEncodingBody::TelemetryReport(ref mut b) => b.poll_data(),
GrpcEncodingBody::DestinationGet(ref mut b) => b.poll_data(),
}
}
#[inline]
fn poll_trailers(&mut self) -> Poll<Option<http::HeaderMap>, h2::Error> {
match *self {
GrpcEncodingBody::TelemetryReport(ref mut b) => b.poll_trailers(),
GrpcEncodingBody::DestinationGet(ref mut b) => b.poll_trailers(),
}
}
}
impl From<self::telemetry::ClientBody> for GrpcEncodingBody {
fn from(body: self::telemetry::ClientBody) -> Self {
GrpcEncodingBody::TelemetryReport(body)
}
}
impl From<self::discovery::ClientBody> for GrpcEncodingBody {
fn from(body: self::discovery::ClientBody) -> Self {
GrpcEncodingBody::DestinationGet(body)
}
}

View File

@ -3,11 +3,10 @@ use std::sync::{Arc, Mutex};
use futures::{future, Poll, Stream};
use futures_mpsc_lossy;
use ordermap::OrderMap;
use tower_grpc::{self, Request, Response};
use tower_grpc::codegen::server::grpc::ServerStreamingService;
use tower_grpc::{self as grpc, Response};
use control::pb::common::TapEvent;
use control::pb::proxy::tap::ObserveRequest;
use control::pb::proxy::tap::{server, ObserveRequest};
use convert::*;
use ctx;
use telemetry::Event;
@ -42,29 +41,23 @@ impl Observe {
}
}
impl ServerStreamingService for Observe {
type Request = ObserveRequest;
type Response = TapEvent;
type ResponseStream = TapEvents;
type Future = future::FutureResult<Response<Self::ResponseStream>, tower_grpc::Error>;
impl server::Tap for Observe {
type ObserveStream = TapEvents;
type ObserveFuture = future::FutureResult<Response<Self::ObserveStream>, grpc::Error>;
fn poll_ready(&mut self) -> Poll<(), tower_grpc::Error> {
Ok(().into())
}
fn call(&mut self, req: Request<ObserveRequest>) -> Self::Future {
fn observe(&mut self, req: grpc::Request<ObserveRequest>) -> Self::ObserveFuture {
if self.next_id == ::std::usize::MAX {
return future::err(tower_grpc::Error::Grpc(tower_grpc::Status::INTERNAL));
return future::err(grpc::Error::Grpc(grpc::Status::INTERNAL));
}
let (_, req) = req.into_http().into_parts();
let req = req.into_inner();
let (tap, rx) = match req.match_
.and_then(|m| Tap::new(&m, self.tap_capacity).ok())
{
Some(m) => m,
None => {
return future::err(tower_grpc::Error::Grpc(
tower_grpc::Status::INVALID_ARGUMENT,
return future::err(grpc::Error::Grpc(
grpc::Status::INVALID_ARGUMENT,
));
}
};
@ -77,7 +70,7 @@ impl ServerStreamingService for Observe {
tap_id
}
Err(_) => {
return future::err(tower_grpc::Error::Grpc(tower_grpc::Status::INTERNAL));
return future::err(grpc::Error::Grpc(grpc::Status::INTERNAL));
}
};
@ -88,13 +81,14 @@ impl ServerStreamingService for Observe {
remaining: req.limit as usize,
taps: self.taps.clone(),
};
future::ok(Response::new(events))
}
}
impl Stream for TapEvents {
type Item = TapEvent;
type Error = tower_grpc::Error;
type Error = grpc::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
loop {

View File

@ -176,7 +176,7 @@ impl<'a> TryFrom<&'a Event> for common::TapEvent {
stream: ctx.id as u64,
}),
method: Some((&ctx.method).into()),
scheme: ctx.uri.scheme_part().map(|s| s.as_str().into()),
scheme: ctx.uri.scheme_part().map(common::Scheme::from),
authority: ctx.uri
.authority_part()
.map(|a| a.as_str())
@ -312,6 +312,12 @@ impl<'a> From<&'a http::Method> for common::HttpMethod {
}
}
impl<'a> From<&'a http::uri::Scheme> for common::Scheme {
fn from(scheme: &'a http::uri::Scheme) -> Self {
scheme.as_ref().into()
}
}
impl<'a> From<&'a str> for common::scheme::Type {
fn from(s: &'a str) -> Self {
use self::common::scheme::*;

View File

@ -1,42 +1,32 @@
use std::fmt;
use std::time::{Duration, Instant};
use futures::{Async, Future, Stream};
use tower::Service;
use tower_grpc;
use tokio_core::reactor::Handle;
use tower_h2::{HttpService, BoxBody};
use tower_grpc as grpc;
use super::codec::Protobuf;
use super::pb::proxy::telemetry::{ReportRequest, ReportResponse};
use super::pb::proxy::telemetry::client::Telemetry as TelemetrySvc;
use super::pb::proxy::telemetry::client::telemetry_methods::Report as ReportRpc;
use ::timeout::{Timeout, TimeoutFuture};
pub type ClientBody = tower_grpc::client::codec::EncodingBody<
Protobuf<ReportRequest, ReportResponse>,
tower_grpc::client::codec::Unary<ReportRequest>,
>;
type TelemetryStream<F> = tower_grpc::client::BodyFuture<
tower_grpc::client::Unary<
tower_grpc::client::ResponseFuture<Protobuf<ReportRequest, ReportResponse>, TimeoutFuture<F>>,
Protobuf<ReportRequest, ReportResponse>,
>,
>;
type TelemetryStream<F, B> = grpc::client::unary::ResponseFuture<
ReportResponse, TimeoutFuture<F>, B>;
#[derive(Debug)]
pub struct Telemetry<T, F> {
pub struct Telemetry<T, S: HttpService> {
reports: T,
in_flight: Option<(Instant, TelemetryStream<F>)>,
in_flight: Option<(Instant, TelemetryStream<S::Future, S::ResponseBody>)>,
report_timeout: Duration,
handle: Handle,
}
impl<T, F> Telemetry<T, F>
impl<T, S> Telemetry<T, S>
where
S: HttpService<RequestBody = BoxBody, ResponseBody = ::tower_h2::RecvBody>,
S::Error: fmt::Debug,
T: Stream<Item = ReportRequest>,
T::Error: ::std::fmt::Debug,
F: Future<Item = ::http::Response<::tower_h2::RecvBody>>,
F::Error: ::std::fmt::Debug,
{
pub fn new(reports: T, report_timeout: Duration, handle: &Handle) -> Self {
Telemetry {
@ -47,18 +37,10 @@ where
}
}
pub fn poll_rpc<S>(&mut self, client: &mut S)
where
S: Service<
Request = ::http::Request<ClientBody>,
Response = F::Item,
Error = F::Error,
Future = F,
>,
pub fn poll_rpc(&mut self, client: &mut S)
{
let client = Timeout::new(client, self.report_timeout, &self.handle);
let grpc = tower_grpc::Client::new(Protobuf::new(), client);
let mut rpc = ReportRpc::new(grpc);
let client = Timeout::new(client.lift_ref(), self.report_timeout, &self.handle);
let mut svc = TelemetrySvc::new(client);
//let _ctxt = ::logging::context("Telemetry.Report".into());
@ -78,8 +60,7 @@ where
}
}
let controller_ready = self.in_flight.is_none() && match rpc.poll_ready() {
let controller_ready = self.in_flight.is_none() && match svc.poll_ready() {
Ok(Async::Ready(_)) => true,
Ok(Async::NotReady) => {
trace!("controller unavailable");
@ -119,7 +100,7 @@ where
report.server_transports.len(),
report.client_transports.len(),
);
let rep = TelemetrySvc::new(&mut rpc).report(report);
let rep = svc.report(grpc::Request::new(report));
self.in_flight = Some((Instant::now(), rep));
}
}

View File

@ -1,18 +1,14 @@
use std::io;
use std::net::{SocketAddr};
use std::sync::Arc;
use http;
use tower;
use tower_buffer::{self, Buffer};
use tower_h2;
use tower_reconnect::{self, Reconnect};
use tower_router::Recognize;
use bind;
use ctx;
use telemetry;
use transparency;
use transport;
type Bind<B> = bind::Bind<Arc<ctx::Proxy>, B>;
@ -21,11 +17,6 @@ pub struct Inbound<B> {
bind: Bind<B>,
}
type Client<B> = transparency::Client<
telemetry::sensor::Connect<transport::TimeoutConnect<transport::Connect>>,
B,
>;
// ===== impl Inbound =====
impl<B> Inbound<B> {
@ -42,16 +33,13 @@ where
B: tower_h2::Body + 'static,
{
type Request = http::Request<B>;
type Response = http::Response<telemetry::sensor::http::ResponseBody<transparency::HttpBody>>;
type Response = bind::HttpResponse;
type Error = tower_buffer::Error<
tower_reconnect::Error<
tower_h2::client::Error,
tower_h2::client::ConnectError<transport::TimeoutError<io::Error>>,
>,
<bind::Service<B> as tower::Service>::Error
>;
type Key = (SocketAddr, bind::Protocol);
type RouteError = ();
type Service = Buffer<Reconnect<telemetry::sensor::NewHttp<Client<B>, B, transparency::HttpBody>>>;
type Service = Buffer<bind::Service<B>>;
fn recognize(&self, req: &Self::Request) -> Option<Self::Key> {
let key = req.extensions()

View File

@ -76,7 +76,6 @@ mod tower_fn; // TODO: move to tower-fn
use bind::Bind;
use connection::BoundPort;
use control::pb::proxy::tap;
use inbound::Inbound;
use map_err::MapErr;
use transparency::{HttpBody, Server};
@ -245,12 +244,13 @@ where
thread::Builder::new()
.name("controller-client".into())
.spawn(move || {
use control::pb::proxy::tap::server::TapServer;
let mut core = Core::new().expect("initialize controller core");
let executor = core.handle();
let (taps, observe) = control::Observe::new(100);
let new_service = tap::server::Tap::new_service().observe(observe);
let new_service = TapServer::new(observe);
let server = serve_control(
control_listener,

View File

@ -1,32 +1,21 @@
use std::io;
use std::sync::Arc;
use http;
use tower_balance::{self, Balance};
use tower_buffer::{self, Buffer};
use tower;
use tower_balance::{self, choose, Balance};
use tower_buffer::Buffer;
use tower_h2;
use tower_reconnect;
use tower_router::Recognize;
use bind::{Bind, BindProtocol, Protocol};
use control;
use bind::{self, Bind, Protocol};
use control::{self, discovery};
use ctx;
use fully_qualified_authority::FullyQualifiedAuthority;
use telemetry;
use transparency;
use transport;
type Discovery<B> = control::discovery::Watch<BindProtocol<Arc<ctx::Proxy>, B>>;
type BindProtocol<B> = bind::BindProtocol<Arc<ctx::Proxy>, B>;
type Error = tower_buffer::Error<
tower_balance::Error<
tower_reconnect::Error<
tower_h2::client::Error,
tower_h2::client::ConnectError<transport::TimeoutError<io::Error>>,
>,
(),
>,
>;
type Discovery<B> = discovery::Watch<BindProtocol<B>>;
pub struct Outbound<B> {
bind: Bind<Arc<ctx::Proxy>, B>,
@ -55,11 +44,14 @@ where
B: tower_h2::Body + 'static,
{
type Request = http::Request<B>;
type Response = http::Response<telemetry::sensor::http::ResponseBody<transparency::HttpBody>>;
type Error = Error;
type Response = bind::HttpResponse;
type Error = <Self::Service as tower::Service>::Error;
type Key = (FullyQualifiedAuthority, Protocol);
type RouteError = ();
type Service = Buffer<Balance<Discovery<B>>>;
type Service = Buffer<Balance<
Discovery<B>,
choose::RoundRobin, // TODO: better load balancer.
>>;
fn recognize(&self, req: &Self::Request) -> Option<Self::Key> {
req.uri().authority_part().map(|authority| {
@ -97,7 +89,8 @@ where
self.bind.clone().with_protocol(protocol),
);
let balance = Balance::new(resolve);
// TODO: move to p2c lb.
let balance = tower_balance::round_robin(resolve);
// Wrap with buffering. This currently is an unbounded buffer,
// which is not ideal.

View File

@ -77,7 +77,11 @@ impl<N, A, B> NewHttp<N, A, B>
where
A: Body + 'static,
B: Body + 'static,
N: NewService<Request = http::Request<A>, Response = http::Response<B>, Error = client::Error>
N: NewService<
Request = http::Request<A>,
Response = http::Response<B>,
Error = client::Error,
>
+ 'static,
{
pub(super) fn new(
@ -100,7 +104,11 @@ impl<N, A, B> NewService for NewHttp<N, A, B>
where
A: Body + 'static,
B: Body + 'static,
N: NewService<Request = http::Request<A>, Response = http::Response<B>, Error = client::Error>
N: NewService<
Request = http::Request<A>,
Response = http::Response<B>,
Error = client::Error,
>
+ 'static,
{
type Request = N::Request;
@ -152,7 +160,11 @@ impl<S, A, B> Service for Http<S, A, B>
where
A: Body + 'static,
B: Body + 'static,
S: Service<Request = http::Request<A>, Response = http::Response<B>, Error = client::Error>
S: Service<
Request = http::Request<A>,
Response = http::Response<B>,
Error = client::Error,
>
+ 'static,
{
type Request = S::Request;
@ -197,7 +209,7 @@ where
impl<F, B> Future for Respond<F, B>
where
F: Future<Item = http::Response<B>, Error = client::Error>,
F: Future<Item = http::Response<B>, Error=client::Error>,
B: Body + 'static,
{
type Item = http::Response<ResponseBody<B>>;

View File

@ -260,7 +260,10 @@ impl<'a> TryFrom<&'a observe_request::match_::tcp::Netmask> for NetMatch {
impl HttpMatch {
fn matches(&self, req: &Arc<ctx::http::Request>) -> bool {
match *self {
HttpMatch::Scheme(ref m) => req.uri.scheme_part().map(|s| &**m == s).unwrap_or(false),
HttpMatch::Scheme(ref m) => req.uri
.scheme_part()
.map(|s| m == s.as_ref())
.unwrap_or(false),
HttpMatch::Method(ref m) => *m == req.method,

View File

@ -3,10 +3,12 @@ use futures::{Async, Future, Poll};
use std::error::Error;
use std::fmt;
use std::io;
use std::time::Duration;
use tokio_connect::Connect;
use tokio_core::reactor::{Timeout as ReactorTimeout, Handle};
use tokio_io;
use tower::Service;
@ -37,10 +39,9 @@ pub struct TimeoutFuture<F> {
//===== impl Timeout =====
impl<U> Timeout<U> {
/// Construct a new `Timeout` wrapping `inner`.
pub fn new(inner: U, duration: Duration, handle: &Handle) -> Self {
Timeout {
Timeout {
inner,
duration,
handle: handle.clone(),
@ -49,13 +50,13 @@ impl<U> Timeout<U> {
}
impl<S, T, E> Service for Timeout<S>
impl<S, T, E> Service for Timeout<S>
where
S: Service<Response=T, Error=E>,
// E: Error,
{
type Request = S::Request;
type Response = T;
type Response = T;
type Error = TimeoutError<E>;
type Future = TimeoutFuture<S::Future>;
@ -101,11 +102,51 @@ where
}
}
impl<C> io::Read for Timeout<C>
where
C: io::Read,
{
fn read(&mut self, buf: &mut [u8]) -> io::Result<usize> {
self.inner.read(buf)
}
}
impl<C> io::Write for Timeout<C>
where
C: io::Write,
{
fn write(&mut self, buf: &[u8]) -> io::Result<usize> {
self.inner.write(buf)
}
fn flush(&mut self) -> io::Result<()> {
self.inner.flush()
}
}
impl<C> tokio_io::AsyncRead for Timeout<C>
where
C: tokio_io::AsyncRead,
{
unsafe fn prepare_uninitialized_buffer(&self, buf: &mut [u8]) -> bool {
self.inner.prepare_uninitialized_buffer(buf)
}
}
impl<C> tokio_io::AsyncWrite for Timeout<C>
where
C: tokio_io::AsyncWrite,
{
fn shutdown(&mut self) -> Poll<(), io::Error> {
self.inner.shutdown()
}
}
//===== impl TimeoutError =====
impl<E> From<E> for TimeoutError<E> {
#[inline] fn from(error: E) -> Self {
#[inline]
fn from(error: E) -> Self {
TimeoutError::Error(error)
}
}
@ -116,7 +157,7 @@ where
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match *self {
TimeoutError::Timeout(ref duration) =>
TimeoutError::Timeout(ref duration) =>
// TODO: format the duration nicer.
write!(f, "operation timed out after {:?}", duration),
TimeoutError::Error(ref err) =>
@ -125,10 +166,10 @@ where
}
}
impl<E> Error for TimeoutError<E>
where
impl<E> Error for TimeoutError<E>
where
E: Error
{
{
fn cause(&self) -> Option<&Error> {
match *self {
TimeoutError::Error(ref err) => Some(err),
@ -160,7 +201,7 @@ where
Err(TimeoutError::Timeout(self.duration))
} else {
Ok(Async::NotReady)
}
}
}
}
@ -176,4 +217,4 @@ where
.field("duration", &self.duration)
.finish()
}
}
}

View File

@ -23,7 +23,7 @@ where
B: tower_h2::Body,
{
Http1(hyper::Client<HyperConnect<C>, BodyStream<B>>),
Http2(tower_h2::client::Client<C, Handle, B>),
Http2(tower_h2::client::Connect<C, Handle, B>),
}
/// A `Future` returned from `Client::new_service()`.
@ -48,6 +48,7 @@ where
pub struct ClientService<C, B>
where
B: tower_h2::Body,
C: Connect,
{
inner: ClientServiceInner<C, B>,
}
@ -55,9 +56,14 @@ where
enum ClientServiceInner<C, B>
where
B: tower_h2::Body,
C: Connect
{
Http1(hyper::Client<HyperConnect<C>, BodyStream<B>>),
Http2(tower_h2::client::Service<C, Handle, B>),
Http2(tower_h2::client::Connection<
<C as Connect>::Connected,
Handle,
B
>),
}
impl<C, B> Client<C, B>
@ -83,7 +89,7 @@ where
// h2 currently doesn't handle PUSH_PROMISE that well, so we just
// disable it for now.
h2_builder.enable_push(false);
let h2 = tower_h2::client::Client::new(connect, h2_builder, executor);
let h2 = tower_h2::client::Connect::new(connect, h2_builder, executor);
Client {
inner: ClientInner::Http2(h2),

View File

@ -24,7 +24,6 @@ pub struct LookupAddressAndConnect {
}
pub type TimeoutConnect<C> = timeout::Timeout<C>;
pub type TimeoutError<E> = timeout::TimeoutError<E>;
// ===== impl Connect =====

View File

@ -1,5 +1,5 @@
mod connect;
mod so_original_dst;
pub use self::connect::{Connect, LookupAddressAndConnect, TimeoutConnect, TimeoutError};
pub use self::connect::{Connect, LookupAddressAndConnect, TimeoutConnect};
pub use self::so_original_dst::{GetOriginalDst, SoOriginalDst};

View File

@ -124,7 +124,7 @@ fn run(addr: SocketAddr, version: Run) -> Sender {
.map_err(|e| println!("client error: {:?}", e)))
},
Run::Http2 => {
let h2 = tower_h2::Client::<Conn, Handle, ()>::new(
let h2 = tower_h2::client::Connect::<Conn, Handle, ()>::new(
conn,
Default::default(),
reactor.clone(),

View File

@ -1,8 +0,0 @@
[package]
name = "tower-grpc-build"
version = "0.2.0"
authors = ["Carl Lerche <me@carllerche.com>"]
[dependencies]
codegen = { version = "0.2", path = "../codegen" }
prost-build = "0.2"

View File

@ -1,270 +0,0 @@
use prost_build;
#[allow(unused)]
use std::ascii::AsciiExt;
use std::fmt::{self, Write};
/// Generates service code
pub struct ServiceGenerator;
struct State {
mod_levels: usize,
}
// ===== impl ServiceGenerator =====
impl ServiceGenerator {
pub fn generate(&self, service: &prost_build::Service, buf: &mut String) -> fmt::Result {
let mut state = State {
mod_levels: 0,
};
state.generate(service, buf)
}
}
// ===== impl State =====
impl State {
pub fn generate(&mut self, service: &prost_build::Service, buf: &mut String) -> fmt::Result {
// Generate code in a client module
write!(buf, "pub mod client {{\n")?;
self.mod_levels = 1;
self.generate_svc(&service, buf)?;
write!(buf, "\n pub mod {}_methods {{", service.name.to_ascii_lowercase())?;
self.mod_levels = 2;
self.generate_rpcs(&service.methods, buf)?;
buf.push_str(" }\n"); // end `mod name_methods {`
buf.push_str("}"); // end `mod client {`
Ok(())
}
/// Generates a generic struct to represent the gRPC service.
fn generate_svc(&self, service: &prost_build::Service, buf: &mut String) -> fmt::Result {
// Generic names used to identify concrete service implementations
// contained by the service client struct.
let mut rpc_generics = vec![];
let mut all_generics = vec![];
let mut struct_fields = vec![];
let mut new_args = vec![];
let mut new_fields = vec![];
let mut where_bounds = vec![];
for method in &service.methods {
let rpc_name = format!("{}Rpc", method.proto_name);
let arg = self.input_name(method);
let returns = self.svc_returns(method);
rpc_generics.push(rpc_name.clone());
all_generics.push(rpc_name.clone());
struct_fields.push(format!("{}: {},", method.name, rpc_name));
where_bounds.push(format!("{}: ::tower::Service<\
Request=::tower_grpc::Request<{}>,\
Response=::tower_grpc::Response<{}>,\
>,", rpc_name, arg, returns));
if method.server_streaming {
all_generics.push(returns.clone());
where_bounds.push(format!("{}: ::futures::Stream<Item={}, Error=::tower_grpc::Error<::h2::Error>>,", returns, self.output_name(method)));
}
new_args.push(format!("{}: {}", method.name, rpc_name));
new_fields.push(format!("{},", method.name));
}
write!(buf, r##"
#[derive(Debug)]
pub struct {name}<{rpc_generics}> {{
{struct_fields}
}}
impl<{all_generics}> {name}<{rpc_generics}>
where
{where_bounds}
{{
pub fn new({new_args}) -> Self {{
{name} {{
{new_fields}
}}
}}
"##,
name=service.name,
all_generics=all_generics.join(", "),
rpc_generics=rpc_generics.join(", "),
where_bounds=where_bounds.join("\n"),
struct_fields=struct_fields.join("\n "),
new_args=new_args.join(", "),
new_fields=new_fields.join("\n "),
)?;
self.generate_svc_methods(&service, buf)?;
buf.push_str(" }\n"); // close `impl {name} {`
Ok(())
}
/// Generates the inherent methods on the generic service struct.
fn generate_svc_methods(&self, service: &prost_build::Service, buf: &mut String) -> fmt::Result {
for method in &service.methods {
let path = format!("/{}.{}/{}", service.package, service.proto_name, method.proto_name);
let svc_returns = self.svc_returns(method);
let rpc_name = format!("{}Rpc", method.proto_name);
let (returns, map_fut) = if method.server_streaming {
let returns = format!(
"::tower_grpc::client::Streaming<{rpc_name}::Future, {returns}>",
rpc_name=rpc_name,
returns=svc_returns,
);
(returns, "::tower_grpc::client::Streaming::map_future(fut)")
} else {
let returns = format!(
"::tower_grpc::client::BodyFuture<{rpc_name}::Future>",
rpc_name=rpc_name,
);
(returns, "::tower_grpc::client::BodyFuture::new(fut)")
};
write!(buf, r##"
pub fn {method}(&mut self, req: {arg}) -> {returns} {{
let req = ::tower_grpc::Request::new("{path}", req);
let fut = self.{method}.call(req);
{fut}
}}
"##,
method=method.name,
arg=self.input_name(method),
returns=returns,
path=path,
fut=map_fut,
)?;
}
Ok(())
}
/// Generates the individual RPCs as `tower::Service`.
fn generate_rpcs(&self, methods: &[prost_build::Method], buf: &mut String) -> fmt::Result {
for method in methods {
let input = self.input_name(method);
let output = self.output_name(method);
let call_stream = if method.client_streaming {
unimplemented!("generate client streaming");
} else {
"::tower_grpc::client::codec::Unary"
};
let where_bounds = format!(r##"
C: ::tower_grpc::client::Codec<Encode={input}, Decode={output}>,
S: ::tower::Service<
Request=::tower_grpc::Request<
{call_stream}<{input}>
>,
Response=::tower_grpc::Response<
::tower_grpc::client::codec::DecodingBody<C>
>,
Error=::tower_grpc::Error<E>
>,"##,
call_stream=call_stream,
input=input,
output=output,
);
let returns = self.rpc_returns(method);
let fut = if method.server_streaming {
"S::Future"
} else {
"::tower_grpc::client::Unary<S::Future, C>"
};
let call = if method.server_streaming {
"fut"
} else {
"::tower_grpc::client::Unary::map_future(fut)"
};
write!(buf, r##"
#[derive(Debug)]
pub struct {name}<S> {{
service: S,
}}
impl<S, C, E> {name}<S>
where{where_bounds}
{{
pub fn new(service: S) -> Self {{
{name} {{
service,
}}
}}
}}
impl<S, C, E> ::tower::Service for {name}<S>
where{where_bounds}
{{
type Request = ::tower_grpc::Request<{input}>;
type Response = ::tower_grpc::Response<{returns}>;
type Error = S::Error;
type Future = {fut};
fn poll_ready(&mut self) -> ::futures::Poll<(), Self::Error> {{
::tower::Service::poll_ready(&mut self.service)
}}
fn call(&mut self, req: Self::Request) -> Self::Future {{
let fut = ::tower::Service::call(&mut self.service, {req});
{call}
}}
}}
"##,
name=method.proto_name,
where_bounds=where_bounds,
input=input,
returns=returns,
fut=fut,
req=if method.client_streaming { "req" } else { "req.into_unary()" },
call=call,
)?;
}
Ok(())
}
fn supers(&self) -> String {
(0..self.mod_levels)
.map(|_| "super::")
.collect::<Vec<_>>()
.concat()
}
fn input_name(&self, method: &prost_build::Method) -> String {
format!("{}{}", self.supers(), method.input_type)
}
fn output_name(&self, method: &prost_build::Method) -> String {
format!("{}{}", self.supers(), method.output_type)
}
fn svc_returns(&self, method: &prost_build::Method) -> String {
if method.server_streaming {
format!("{}Returns", method.proto_name)
} else {
self.output_name(method)
}
}
fn rpc_returns(&self, method: &prost_build::Method) -> String {
if method.server_streaming {
"::tower_grpc::client::codec::DecodingBody<C>".into()
} else {
self.output_name(method)
}
}
}

View File

@ -1,92 +0,0 @@
extern crate codegen;
extern crate prost_build;
mod client;
mod server;
use std::io;
use std::cell::RefCell;
use std::fmt::Write;
use std::path::Path;
use std::rc::Rc;
/// Code generation configuration
pub struct Config {
prost: prost_build::Config,
inner: Rc<RefCell<Inner>>,
}
struct Inner {
build_client: bool,
build_server: bool,
}
struct ServiceGenerator {
client: client::ServiceGenerator,
server: server::ServiceGenerator,
inner: Rc<RefCell<Inner>>,
}
impl Config {
/// Returns a new `Config` with default values.
pub fn new() -> Self {
let mut prost = prost_build::Config::new();
let inner = Rc::new(RefCell::new(Inner {
// Enable client code gen by default
build_client: true,
// Disable server code gen by default
build_server: false,
}));
// Set the service generator
prost.service_generator(Box::new(ServiceGenerator {
client: client::ServiceGenerator,
server: server::ServiceGenerator,
inner: inner.clone(),
}));
Config {
prost,
inner,
}
}
/// Enable gRPC client code generation
pub fn enable_client(&mut self, enable: bool) -> &mut Self {
self.inner.borrow_mut().build_client = enable;
self
}
/// Enable gRPC server code generation
pub fn enable_server(&mut self, enable: bool) -> &mut Self {
self.inner.borrow_mut().build_server = enable;
self
}
/// Generate code
pub fn build<P>(&self, protos: &[P], includes: &[P]) -> io::Result<()>
where P: AsRef<Path>,
{
self.prost.compile_protos(protos, includes)
}
}
impl prost_build::ServiceGenerator for ServiceGenerator {
fn generate(&self, service: prost_build::Service, buf: &mut String) {
let inner = self.inner.borrow();
if inner.build_client {
// Add an extra new line to separate messages
write!(buf, "\n").unwrap();
self.client.generate(&service, buf).unwrap();
}
if inner.build_server {
write!(buf, "\n\n").unwrap();
self.server.generate(&service, buf).unwrap();
}
}
}

View File

@ -1,723 +0,0 @@
use codegen;
use prost_build;
#[allow(unused)]
use std::ascii::AsciiExt;
use std::fmt;
fn super_import(ty: &str) -> (String, &str) {
let mut v: Vec<&str> = ty.split("::").collect();
v.insert(0, "super");
let last = v.pop().unwrap_or(ty);
(v.join("::"), last)
}
fn unqualified(ty: &str) -> &str {
ty.rsplit("::").next().unwrap_or(ty)
}
/// Generates service code
pub struct ServiceGenerator;
impl ServiceGenerator {
fn define(&self, service: &prost_build::Service) -> codegen::Scope {
// Name of the support module. This is the module that will contain all
// the extra types for the service to avoid potential name conflicts
// with other services.
let support_name = service.name.to_ascii_lowercase();
// Create support module for the service
let mut support = codegen::Module::new(&support_name);
support.vis("pub")
.import("::tower_grpc::codegen::server", "*")
.import("super", &service.name)
;
// Create a structure for the service
let mut service_struct = codegen::Struct::new(&service.name);
service_struct
.vis("pub")
.derive("Debug")
;
// A fully not implemented service
let mut service_not_implemented_ty = codegen::Type::new(&service.name);
// Create a service implementation for the service struct
let mut service_impl = codegen::Impl::new(&service.name);
service_impl.impl_trait("tower::Service")
.associate_type("Request", "http::Request<tower_h2::RecvBody>")
.associate_type("Error", "h2::Error")
// `Response` and `Future` associated type comes later!
;
let mut service_call_block = codegen::Block::new("match request.uri().path()");
// Create a clone impl for the service struct
let mut clone_impl = codegen::Impl::new(&service.name);
clone_impl.impl_trait("Clone")
;
let mut clone_block = codegen::Block::new(&service.name);
// A `NewService` type. This is a wrapper around the service struct, but
// also adds builder functions to help define the service.
let mut new_service = codegen::Struct::new("NewService");
new_service
.vis("pub")
.derive("Debug")
;
let mut new_service_builder_impl = codegen::Impl::new("NewService");
let mut new_service_trait_impl = codegen::Impl::new("NewService");
new_service_trait_impl
.impl_trait("tower::NewService")
.associate_type("Request", "http::Request<tower_h2::RecvBody>")
.associate_type("Error", "h2::Error")
.associate_type("InitError", "h2::Error")
.associate_type("Future", "futures::FutureResult<Self::Service, Self::Error>")
// `Response` and `Service` associated type comes later!
;
// A fully not implemented `NewService`
let mut new_service_not_implemented_ty = codegen::Type::new("NewService");
let mut new_block = codegen::Block::new(&format!("inner: {}", service.name));
// Create a response future as an enumeration of the service methods
// response futures.
let mut response_fut = codegen::Struct::new("ResponseFuture");
response_fut
.vis("pub")
;
let mut response_impl = codegen::Impl::new("ResponseFuture");
response_impl.impl_trait("futures::Future")
.associate_type("Error", "h2::Error")
// `Item` associated type comes later
;
let mut response_debug_impl = codegen::Impl::new("ResponseFuture");
response_debug_impl.impl_trait("fmt::Debug")
.function("fmt")
.arg_ref_self()
.arg("fmt", "&mut fmt::Formatter")
.ret("fmt::Result")
.line("write!(fmt, \"ResponseFuture\")")
;
let mut response_block = codegen::Block::new(&format!("match self.kind"));
// Create a response body type as an enumeration of the service methods
// response bodies.
let mut response_body = codegen::Struct::new("ResponseBody");
response_body
.vis("pub")
;
let mut response_body_body_imp = codegen::Impl::new("ResponseBody");
response_body_body_imp
.impl_trait("tower_h2::Body")
.associate_type("Data", "bytes::Bytes")
;
let mut response_body_debug_impl = codegen::Impl::new("ResponseBody");
response_body_debug_impl.impl_trait("fmt::Debug")
.function("fmt")
.arg_ref_self()
.arg("fmt", "&mut fmt::Formatter")
.ret("fmt::Result")
.line("write!(fmt, \"ResponseBody\")")
;
;
let mut is_end_stream_block = codegen::Block::new("match self.kind");
let mut poll_data_block = codegen::Block::new("match self.kind");
let mut poll_trailers_block = codegen::Block::new("match self.kind");
// Create the `Kind` enum. This is by the various types above in order
// to contain the inner types for each of the possible service methods.
let mut kind_enum = codegen::Enum::new("Kind");
kind_enum
.vis("pub(super)")
.derive("Debug")
;
for method in &service.methods {
// ===== service struct =====
// Push a generic representing the method service
service_struct.generic(&method.proto_name);
let service_bound = service_bound_for(method);
// Bound a bound requiring that the service is of the appropriate
// type.
service_struct.bound(&method.proto_name, &service_bound);
// Push a field to hold the service
service_struct.field(&method.name, field_type_for(&method));
let ty = unimplemented_type_for(method);
service_not_implemented_ty.generic(&ty);
new_service_not_implemented_ty.generic(&ty);
// ===== Service impl =====
service_impl.generic(&method.proto_name);
service_impl.target_generic(&method.proto_name);
service_impl.bound(&method.proto_name, &service_bound);
// The service method path.
let match_line = format!("\"/{}.{}/{}\" =>",
service.package,
service.proto_name,
method.proto_name);
// Match the service path
let mut route_block = codegen::Block::new(&match_line);
route_block
.line(&format!("let response = self.{}.call(request);", method.name))
.line(&format!("{}::ResponseFuture {{ kind: Ok({}(response)) }}", support_name, method.proto_name))
;
service_call_block.block(route_block);
// ===== Clone impl =====
clone_impl.generic(&method.proto_name);
clone_impl.target_generic(&method.proto_name);
clone_impl.bound(&method.proto_name, &service_bound);
clone_block.line(format!("{name}: self.{name}.clone(),", name = method.name));
// ===== NewService =====
new_service.generic(&method.proto_name);
new_service.bound(&method.proto_name, &service_bound);
new_service_builder_impl
.generic(&method.proto_name)
.target_generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
;
new_service_trait_impl
.generic(&method.proto_name)
.target_generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
;
new_block.line(&format!("{}: {},", method.name, new_unimplemented_for(&method)));
// ===== ResponseFuture =====
// Push a generic on the response future type
response_fut
.generic(&method.proto_name)
.bound(&method.proto_name, &service_bound);
response_impl
.generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
.target_generic(&method.proto_name);
response_debug_impl
.generic(&method.proto_name)
.target_generic(&method.proto_name)
.bound(&method.proto_name, &service_bound);
let match_line = format!("Ok({}(ref mut fut)) =>", method.proto_name);
let mut try_ready_block = codegen::Block::new("let response = match fut.poll()?");
try_ready_block
.line("futures::Async::Ready(v) => v,")
.line("futures::Async::NotReady => return Ok(futures::Async::NotReady)")
.after(";")
;
let mut match_block = codegen::Block::new(&match_line);
match_block
.block(try_ready_block)
.line("let (head, body) = response.into_parts();")
.line(&format!("let body = ResponseBody {{ kind: Ok({}(body)) }};", method.proto_name))
.line("let response = http::Response::from_parts(head, body);")
.line("Ok(response.into())")
;
response_block.block(match_block);
// ===== ResponseBody =====
response_body
.generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
;
response_body_body_imp
.generic(&method.proto_name)
.target_generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
;
response_body_debug_impl
.generic(&method.proto_name)
.target_generic(&method.proto_name)
.bound(&method.proto_name, &service_bound)
;
is_end_stream_block
.line(&format!("Ok({}(ref v)) => v.is_end_stream(),", method.proto_name));
poll_data_block
.line(&format!("Ok({}(ref mut v)) => v.poll_data(),", method.proto_name));
poll_trailers_block
.line(&format!("Ok({}(ref mut v)) => v.poll_trailers(),", method.proto_name));
// ===== Kind =====
// Push a response kind variant
kind_enum.generic(&method.proto_name);
kind_enum.variant(&method.proto_name)
.tuple(&method.proto_name)
;
// ===== support module =====
let (input_path, input_type) = super_import(&method.input_type);
let (output_path, output_type) = super_import(&method.output_type);
// Import the request and response types
support.import(&input_path, input_type);
support.import(&output_path, output_type);
}
// ===== service impl =====
// An impl block that contains the `new_service` function. This block is
// fixed to `NotImplemented` services for all methods.
let mut service_struct_new = codegen::Impl::new(service_not_implemented_ty);
service_struct_new.function("new_service")
.vis("pub")
.ret(&new_service_not_implemented_ty.path(&support_name))
.line(&format!("{}::NewService::new()", &support_name))
;
// ===== Service impl =====
// Add the Future service associated type
let mut http_response_ty = codegen::Type::new("http::Response");
http_response_ty.generic(response_body.ty().path(&support_name));
service_impl
.associate_type("Response", &http_response_ty)
.associate_type("Future", response_fut.ty().path(&support_name));
service_impl.function("poll_ready")
.arg_mut_self()
.ret("futures::Poll<(), Self::Error>")
.line("Ok(().into())")
;
let mut catch_all_block = codegen::Block::new("_ =>");
catch_all_block
.line(&format!("{}::ResponseFuture {{ kind: Err(grpc::Status::UNIMPLEMENTED) }}", support_name));
service_call_block.block(catch_all_block);
service_impl.function("call")
.arg_mut_self()
.arg("request", "Self::Request")
.ret("Self::Future")
.line(&format!("use self::{}::Kind::*;", &support_name))
.line("")
.block(service_call_block)
;
// ===== Clone impl =====
clone_impl.function("clone")
.arg_ref_self()
.ret("Self")
.block(clone_block)
;
// ===== NewService =====
new_service
.field("inner", service_struct.ty())
;
// Generate all the builder functions
for (i, m1) in service.methods.iter().enumerate() {
let service_bound = service_bound_for(m1);
let mut ret = codegen::Type::new("NewService");
let mut build = codegen::Block::new(&format!("inner: {}", &service.name));
// Push all generics onto the return type
for (j, m2) in service.methods.iter().enumerate() {
if i == j {
ret.generic("T");
build.line(&format!("{}: service,", m2.name));
} else {
ret.generic(&m2.proto_name);
build.line(&format!("{name}: self.inner.{name},", name = m2.name));
}
}
new_service_builder_impl.function(&m1.name)
.vis("pub")
.generic("T")
.bound("T", service_bound)
.arg_self()
.arg("service", "T")
.ret(ret)
.line(new_service_line_for(&m1))
.line("")
.block({
let mut b = codegen::Block::new("NewService");
b.block(build);
b
})
;
}
let mut http_response_ty = codegen::Type::new("http::Response");
http_response_ty.generic(response_body.ty());
new_service_trait_impl
.associate_type("Response", &http_response_ty)
.associate_type("Service", service_struct.ty())
.function("new_service")
.arg_ref_self()
.ret("Self::Future")
.line("futures::ok(self.inner.clone())")
;
let mut new_service_not_implemented_impl = codegen::Impl::new(new_service_not_implemented_ty);
new_service_not_implemented_impl
.function("new")
.vis("pub(super)")
.ret("Self")
.block(codegen::Block::new("NewService")
.block(new_block).clone())
;
// ===== ResponseFuture =====
let mut ty = codegen::Type::new("Result");
ty.generic(response_fut_kind(service));
ty.generic("grpc::Status");
response_fut
.field("pub(super) kind", ty)
;
let mut http_response_ty = codegen::Type::new("http::Response");
http_response_ty.generic(response_body.ty());
response_impl
.associate_type("Item", &http_response_ty)
;
let mut match_block = codegen::Block::new("Err(ref status) =>");
match_block
.line("let body = ResponseBody { kind: Err(status.clone()) };")
.line("Ok(grpc::Response::new(body).into_http().into())")
;
response_block.block(match_block);
response_impl.function("poll")
.arg_mut_self()
.ret("futures::Poll<Self::Item, Self::Error>")
.line("use self::Kind::*;")
.line("")
.block(response_block)
;
// ===== ResponseBody =====
let mut ty = codegen::Type::new("Result");
ty.generic(response_body_kind(service));
ty.generic("grpc::Status");
response_body.field("kind", ty);
is_end_stream_block
.line("Err(_) => true,");
response_body_body_imp.function("is_end_stream")
.arg_ref_self()
.ret("bool")
.line("use self::Kind::*;")
.line("")
.block(is_end_stream_block)
;
poll_data_block
.line("Err(_) => Ok(None.into()),");
response_body_body_imp.function("poll_data")
.arg_mut_self()
.ret("futures::Poll<Option<Self::Data>, h2::Error>")
.line("use self::Kind::*;")
.line("")
.block(poll_data_block)
;
let mut match_block = codegen::Block::new("Err(ref status) =>");
match_block
.line("let mut map = http::HeaderMap::new();")
.line("map.insert(\"grpc-status\", status.to_header_value());")
.line("Ok(Some(map).into())")
;
poll_trailers_block.block(match_block);
response_body_body_imp.function("poll_trailers")
.arg_mut_self()
.ret("futures::Poll<Option<http::HeaderMap>, h2::Error>")
.line("use self::Kind::*;")
.line("")
.block(poll_trailers_block)
;
// ===== support module =====
support
.vis("pub")
.import("std", "fmt")
.push_structure(new_service)
.push_structure(response_fut)
.push_structure(response_body)
.push_enumeration(kind_enum)
.push_imp(new_service_not_implemented_impl)
.push_imp(new_service_builder_impl)
.push_imp(new_service_trait_impl)
.push_imp(response_impl)
.push_imp(response_debug_impl)
.push_imp(response_body_body_imp)
.push_imp(response_body_debug_impl)
;
// Create scope that contains the generated server code.
let mut scope = codegen::Scope::new();
{
let server = scope.module("server")
.vis("pub")
.import("::tower_grpc::codegen::server", "*");
for method in &service.methods {
let (input_path, input_type) = super_import(&method.input_type);
let (output_path, output_type) = super_import(&method.output_type);
// Import the request and response types
server.import(&input_path, input_type);
server.import(&output_path, output_type);
}
server.push_structure(service_struct)
.push_imp(service_struct_new)
.push_imp(service_impl)
.push_imp(clone_impl)
.push_module(support)
;
}
scope
}
pub fn generate(&self, service: &prost_build::Service, buf: &mut String) -> fmt::Result {
let scope = self.define(service);
let mut fmt = codegen::Formatter::new(buf);
scope.fmt(&mut fmt)
}
}
fn field_type_for(method: &prost_build::Method) -> codegen::Type {
let ty = match (method.client_streaming, method.server_streaming) {
(false, false) => {
format!("grpc::Grpc<grpc::Unary<{}, grpc::Decode<{}>>>",
method.proto_name, method.input_type)
}
(false, true) => {
format!("grpc::Grpc<grpc::ServerStreaming<{}, grpc::Decode<{}>>>",
method.proto_name, method.input_type)
}
(true, false) => {
format!("grpc::Grpc<grpc::ClientStreaming<{}>>",
method.proto_name)
}
(true, true) => {
format!("grpc::Grpc<{}>",
method.proto_name)
}
};
codegen::Type::from(ty)
}
fn service_bound_for(method: &prost_build::Method) -> codegen::Type {
let input_type = unqualified(&method.input_type);
let output_type = unqualified(&method.output_type);
let ty = match (method.client_streaming, method.server_streaming) {
(false, false) => {
format!("grpc::UnaryService<Request = {}, Response = {}>",
input_type, output_type)
}
(false, true) => {
format!("grpc::ServerStreamingService<Request = {}, Response = {}>",
input_type, output_type)
}
(true, false) => {
format!("grpc::ClientStreamingService<Request = {input}, RequestStream = grpc::Decode<{input}>, Response = {output}>",
input = input_type, output = output_type)
}
(true, true) => {
format!("grpc::GrpcService<Request = {input}, RequestStream = grpc::Decode<{input}>, Response = {output}>",
input = input_type, output = output_type)
}
};
codegen::Type::from(ty)
}
fn new_service_line_for(method: &prost_build::Method) -> &'static str {
match (method.client_streaming, method.server_streaming) {
(false, false) => {
"let service = grpc::Grpc::new(grpc::Unary::new(service));"
}
(false, true) => {
"let service = grpc::Grpc::new(grpc::ServerStreaming::new(service));"
}
(true, false) => {
"let service = grpc::Grpc::new(grpc::ClientStreaming::new(service));"
}
(true, true) => {
"let service = grpc::Grpc::new(service);"
}
}
}
fn unimplemented_type_for(method: &prost_build::Method) -> codegen::Type {
let ty = match (method.client_streaming, method.server_streaming) {
(false, false) => {
format!("grpc::NotImplemented<{}, {}>",
unqualified(&method.input_type), unqualified(&method.output_type))
}
(false, true) => {
format!("grpc::NotImplemented<{}, grpc::unary::Once<{}>>",
unqualified(&method.input_type), unqualified(&method.output_type))
}
(true, false) => {
format!("grpc::NotImplemented<grpc::Decode<{}>, {}>",
unqualified(&method.input_type), unqualified(&method.output_type))
}
(true, true) => {
format!("grpc::NotImplemented<grpc::Decode<{}>, grpc::unary::Once<{}>>",
unqualified(&method.input_type), unqualified(&method.output_type))
}
};
codegen::Type::from(ty)
}
fn new_unimplemented_for(method: &prost_build::Method) -> &'static str {
match (method.client_streaming, method.server_streaming) {
(false, false) => {
"grpc::Grpc::new(grpc::Unary::new(grpc::NotImplemented::new()))"
}
(false, true) => {
"grpc::Grpc::new(grpc::ServerStreaming::new(grpc::NotImplemented::new()))"
}
(true, false) => {
"grpc::Grpc::new(grpc::ClientStreaming::new(grpc::NotImplemented::new()))"
}
(true, true) => {
"grpc::Grpc::new(grpc::NotImplemented::new())"
}
}
}
// ===== Here be the crazy types =====
fn response_fut_kind(service: &prost_build::Service) -> String {
use std::fmt::Write;
// Handle theempty case...
if service.methods.is_empty() {
return "Kind".to_string();
}
let mut ret = "Kind<\n".to_string();
for method in &service.methods {
match (method.client_streaming, method.server_streaming) {
(false, false) => {
write!(&mut ret, " <grpc::Grpc<grpc::Unary<{}, grpc::Decode<{}>>> as tower::Service>::Future,\n",
method.proto_name, method.input_type).unwrap();
}
(false, true) => {
write!(&mut ret, " <grpc::Grpc<grpc::ServerStreaming<{}, grpc::Decode<{}>>> as tower::Service>::Future,\n",
method.proto_name, method.input_type).unwrap();
}
(true, false) => {
write!(&mut ret, " <grpc::Grpc<grpc::ClientStreaming<{}>> as tower::Service>::Future,\n",
method.proto_name).unwrap();
}
(true, true) => {
write!(&mut ret, " <grpc::Grpc<{}> as tower::Service>::Future,\n",
method.proto_name).unwrap();
}
}
}
ret.push_str(">");
ret
}
fn response_body_kind(service: &prost_build::Service) -> String {
use std::fmt::Write;
// Handle theempty case...
if service.methods.is_empty() {
return "Kind".to_string();
}
let mut ret = "Kind<\n".to_string();
for method in &service.methods {
match (method.client_streaming, method.server_streaming) {
(false, false) => {
write!(&mut ret, " grpc::Encode<<grpc::Unary<{}, grpc::Decode<{}>> as grpc::GrpcService>::ResponseStream>,\n",
method.proto_name, method.input_type).unwrap();
}
(false, true) => {
write!(&mut ret, " grpc::Encode<<grpc::ServerStreaming<{}, grpc::Decode<{}>> as grpc::GrpcService>::ResponseStream>,\n",
method.proto_name, method.input_type).unwrap();
}
(true, false) => {
write!(&mut ret, " grpc::Encode<<grpc::ClientStreaming<{}> as grpc::GrpcService>::ResponseStream>,\n",
method.proto_name).unwrap();
}
(true, true) => {
write!(&mut ret, " grpc::Encode<<{} as grpc::GrpcService>::ResponseStream>,\n",
method.proto_name).unwrap();
}
}
}
ret.push_str(">");
ret
}

View File

@ -1,32 +0,0 @@
[package]
name = "tower-grpc-examples"
version = "0.2.0"
authors = ["Carl Lerche <me@carllerche.com>"]
[[bin]]
name = "helloworld"
path = "src/helloworld.rs"
[[bin]]
name = "route_guide"
path = "src/routeguide/main.rs"
[dependencies]
futures = "0.1"
bytes = "0.4"
env_logger = "0.4"
log = "0.3"
prost = "0.2"
prost-derive = "0.2"
tokio-core = "0.1"
tower = { git = "https://github.com/tower-rs/tower" }
tower-h2 = { path = "../tower-h2" }
tower-grpc = { path = "../tower-grpc" }
# For the routeguide example
serde = "1.0"
serde_json = "1.0"
serde_derive = "1.0"
[build-dependencies]
tower-grpc-build = { path = "../tower-grpc-build" }

View File

@ -1,17 +0,0 @@
extern crate tower_grpc_build;
fn main() {
// Build helloworld
tower_grpc_build::Config::new()
.enable_server(true)
.enable_client(false)
.build(&["proto/helloworld/helloworld.proto"], &["proto/helloworld"])
.unwrap();
// Build routeguide
tower_grpc_build::Config::new()
.enable_server(true)
.enable_client(false)
.build(&["proto/routeguide/route_guide.proto"], &["proto/routeguide"])
.unwrap();
}

View File

@ -1,601 +0,0 @@
[{
"location": {
"latitude": 407838351,
"longitude": -746143763
},
"name": "Patriots Path, Mendham, NJ 07945, USA"
}, {
"location": {
"latitude": 408122808,
"longitude": -743999179
},
"name": "101 New Jersey 10, Whippany, NJ 07981, USA"
}, {
"location": {
"latitude": 413628156,
"longitude": -749015468
},
"name": "U.S. 6, Shohola, PA 18458, USA"
}, {
"location": {
"latitude": 419999544,
"longitude": -740371136
},
"name": "5 Conners Road, Kingston, NY 12401, USA"
}, {
"location": {
"latitude": 414008389,
"longitude": -743951297
},
"name": "Mid Hudson Psychiatric Center, New Hampton, NY 10958, USA"
}, {
"location": {
"latitude": 419611318,
"longitude": -746524769
},
"name": "287 Flugertown Road, Livingston Manor, NY 12758, USA"
}, {
"location": {
"latitude": 406109563,
"longitude": -742186778
},
"name": "4001 Tremley Point Road, Linden, NJ 07036, USA"
}, {
"location": {
"latitude": 416802456,
"longitude": -742370183
},
"name": "352 South Mountain Road, Wallkill, NY 12589, USA"
}, {
"location": {
"latitude": 412950425,
"longitude": -741077389
},
"name": "Bailey Turn Road, Harriman, NY 10926, USA"
}, {
"location": {
"latitude": 412144655,
"longitude": -743949739
},
"name": "193-199 Wawayanda Road, Hewitt, NJ 07421, USA"
}, {
"location": {
"latitude": 415736605,
"longitude": -742847522
},
"name": "406-496 Ward Avenue, Pine Bush, NY 12566, USA"
}, {
"location": {
"latitude": 413843930,
"longitude": -740501726
},
"name": "162 Merrill Road, Highland Mills, NY 10930, USA"
}, {
"location": {
"latitude": 410873075,
"longitude": -744459023
},
"name": "Clinton Road, West Milford, NJ 07480, USA"
}, {
"location": {
"latitude": 412346009,
"longitude": -744026814
},
"name": "16 Old Brook Lane, Warwick, NY 10990, USA"
}, {
"location": {
"latitude": 402948455,
"longitude": -747903913
},
"name": "3 Drake Lane, Pennington, NJ 08534, USA"
}, {
"location": {
"latitude": 406337092,
"longitude": -740122226
},
"name": "6324 8th Avenue, Brooklyn, NY 11220, USA"
}, {
"location": {
"latitude": 406421967,
"longitude": -747727624
},
"name": "1 Merck Access Road, Whitehouse Station, NJ 08889, USA"
}, {
"location": {
"latitude": 416318082,
"longitude": -749677716
},
"name": "78-98 Schalck Road, Narrowsburg, NY 12764, USA"
}, {
"location": {
"latitude": 415301720,
"longitude": -748416257
},
"name": "282 Lakeview Drive Road, Highland Lake, NY 12743, USA"
}, {
"location": {
"latitude": 402647019,
"longitude": -747071791
},
"name": "330 Evelyn Avenue, Hamilton Township, NJ 08619, USA"
}, {
"location": {
"latitude": 412567807,
"longitude": -741058078
},
"name": "New York State Reference Route 987E, Southfields, NY 10975, USA"
}, {
"location": {
"latitude": 416855156,
"longitude": -744420597
},
"name": "103-271 Tempaloni Road, Ellenville, NY 12428, USA"
}, {
"location": {
"latitude": 404663628,
"longitude": -744820157
},
"name": "1300 Airport Road, North Brunswick Township, NJ 08902, USA"
}, {
"location": {
"latitude": 407113723,
"longitude": -749746483
},
"name": ""
}, {
"location": {
"latitude": 402133926,
"longitude": -743613249
},
"name": ""
}, {
"location": {
"latitude": 400273442,
"longitude": -741220915
},
"name": ""
}, {
"location": {
"latitude": 411236786,
"longitude": -744070769
},
"name": ""
}, {
"location": {
"latitude": 411633782,
"longitude": -746784970
},
"name": "211-225 Plains Road, Augusta, NJ 07822, USA"
}, {
"location": {
"latitude": 415830701,
"longitude": -742952812
},
"name": ""
}, {
"location": {
"latitude": 413447164,
"longitude": -748712898
},
"name": "165 Pedersen Ridge Road, Milford, PA 18337, USA"
}, {
"location": {
"latitude": 405047245,
"longitude": -749800722
},
"name": "100-122 Locktown Road, Frenchtown, NJ 08825, USA"
}, {
"location": {
"latitude": 418858923,
"longitude": -746156790
},
"name": ""
}, {
"location": {
"latitude": 417951888,
"longitude": -748484944
},
"name": "650-652 Willi Hill Road, Swan Lake, NY 12783, USA"
}, {
"location": {
"latitude": 407033786,
"longitude": -743977337
},
"name": "26 East 3rd Street, New Providence, NJ 07974, USA"
}, {
"location": {
"latitude": 417548014,
"longitude": -740075041
},
"name": ""
}, {
"location": {
"latitude": 410395868,
"longitude": -744972325
},
"name": ""
}, {
"location": {
"latitude": 404615353,
"longitude": -745129803
},
"name": ""
}, {
"location": {
"latitude": 406589790,
"longitude": -743560121
},
"name": "611 Lawrence Avenue, Westfield, NJ 07090, USA"
}, {
"location": {
"latitude": 414653148,
"longitude": -740477477
},
"name": "18 Lannis Avenue, New Windsor, NY 12553, USA"
}, {
"location": {
"latitude": 405957808,
"longitude": -743255336
},
"name": "82-104 Amherst Avenue, Colonia, NJ 07067, USA"
}, {
"location": {
"latitude": 411733589,
"longitude": -741648093
},
"name": "170 Seven Lakes Drive, Sloatsburg, NY 10974, USA"
}, {
"location": {
"latitude": 412676291,
"longitude": -742606606
},
"name": "1270 Lakes Road, Monroe, NY 10950, USA"
}, {
"location": {
"latitude": 409224445,
"longitude": -748286738
},
"name": "509-535 Alphano Road, Great Meadows, NJ 07838, USA"
}, {
"location": {
"latitude": 406523420,
"longitude": -742135517
},
"name": "652 Garden Street, Elizabeth, NJ 07202, USA"
}, {
"location": {
"latitude": 401827388,
"longitude": -740294537
},
"name": "349 Sea Spray Court, Neptune City, NJ 07753, USA"
}, {
"location": {
"latitude": 410564152,
"longitude": -743685054
},
"name": "13-17 Stanley Street, West Milford, NJ 07480, USA"
}, {
"location": {
"latitude": 408472324,
"longitude": -740726046
},
"name": "47 Industrial Avenue, Teterboro, NJ 07608, USA"
}, {
"location": {
"latitude": 412452168,
"longitude": -740214052
},
"name": "5 White Oak Lane, Stony Point, NY 10980, USA"
}, {
"location": {
"latitude": 409146138,
"longitude": -746188906
},
"name": "Berkshire Valley Management Area Trail, Jefferson, NJ, USA"
}, {
"location": {
"latitude": 404701380,
"longitude": -744781745
},
"name": "1007 Jersey Avenue, New Brunswick, NJ 08901, USA"
}, {
"location": {
"latitude": 409642566,
"longitude": -746017679
},
"name": "6 East Emerald Isle Drive, Lake Hopatcong, NJ 07849, USA"
}, {
"location": {
"latitude": 408031728,
"longitude": -748645385
},
"name": "1358-1474 New Jersey 57, Port Murray, NJ 07865, USA"
}, {
"location": {
"latitude": 413700272,
"longitude": -742135189
},
"name": "367 Prospect Road, Chester, NY 10918, USA"
}, {
"location": {
"latitude": 404310607,
"longitude": -740282632
},
"name": "10 Simon Lake Drive, Atlantic Highlands, NJ 07716, USA"
}, {
"location": {
"latitude": 409319800,
"longitude": -746201391
},
"name": "11 Ward Street, Mount Arlington, NJ 07856, USA"
}, {
"location": {
"latitude": 406685311,
"longitude": -742108603
},
"name": "300-398 Jefferson Avenue, Elizabeth, NJ 07201, USA"
}, {
"location": {
"latitude": 419018117,
"longitude": -749142781
},
"name": "43 Dreher Road, Roscoe, NY 12776, USA"
}, {
"location": {
"latitude": 412856162,
"longitude": -745148837
},
"name": "Swan Street, Pine Island, NY 10969, USA"
}, {
"location": {
"latitude": 416560744,
"longitude": -746721964
},
"name": "66 Pleasantview Avenue, Monticello, NY 12701, USA"
}, {
"location": {
"latitude": 405314270,
"longitude": -749836354
},
"name": ""
}, {
"location": {
"latitude": 414219548,
"longitude": -743327440
},
"name": ""
}, {
"location": {
"latitude": 415534177,
"longitude": -742900616
},
"name": "565 Winding Hills Road, Montgomery, NY 12549, USA"
}, {
"location": {
"latitude": 406898530,
"longitude": -749127080
},
"name": "231 Rocky Run Road, Glen Gardner, NJ 08826, USA"
}, {
"location": {
"latitude": 407586880,
"longitude": -741670168
},
"name": "100 Mount Pleasant Avenue, Newark, NJ 07104, USA"
}, {
"location": {
"latitude": 400106455,
"longitude": -742870190
},
"name": "517-521 Huntington Drive, Manchester Township, NJ 08759, USA"
}, {
"location": {
"latitude": 400066188,
"longitude": -746793294
},
"name": ""
}, {
"location": {
"latitude": 418803880,
"longitude": -744102673
},
"name": "40 Mountain Road, Napanoch, NY 12458, USA"
}, {
"location": {
"latitude": 414204288,
"longitude": -747895140
},
"name": ""
}, {
"location": {
"latitude": 414777405,
"longitude": -740615601
},
"name": ""
}, {
"location": {
"latitude": 415464475,
"longitude": -747175374
},
"name": "48 North Road, Forestburgh, NY 12777, USA"
}, {
"location": {
"latitude": 404062378,
"longitude": -746376177
},
"name": ""
}, {
"location": {
"latitude": 405688272,
"longitude": -749285130
},
"name": ""
}, {
"location": {
"latitude": 400342070,
"longitude": -748788996
},
"name": ""
}, {
"location": {
"latitude": 401809022,
"longitude": -744157964
},
"name": ""
}, {
"location": {
"latitude": 404226644,
"longitude": -740517141
},
"name": "9 Thompson Avenue, Leonardo, NJ 07737, USA"
}, {
"location": {
"latitude": 410322033,
"longitude": -747871659
},
"name": ""
}, {
"location": {
"latitude": 407100674,
"longitude": -747742727
},
"name": ""
}, {
"location": {
"latitude": 418811433,
"longitude": -741718005
},
"name": "213 Bush Road, Stone Ridge, NY 12484, USA"
}, {
"location": {
"latitude": 415034302,
"longitude": -743850945
},
"name": ""
}, {
"location": {
"latitude": 411349992,
"longitude": -743694161
},
"name": ""
}, {
"location": {
"latitude": 404839914,
"longitude": -744759616
},
"name": "1-17 Bergen Court, New Brunswick, NJ 08901, USA"
}, {
"location": {
"latitude": 414638017,
"longitude": -745957854
},
"name": "35 Oakland Valley Road, Cuddebackville, NY 12729, USA"
}, {
"location": {
"latitude": 412127800,
"longitude": -740173578
},
"name": ""
}, {
"location": {
"latitude": 401263460,
"longitude": -747964303
},
"name": ""
}, {
"location": {
"latitude": 412843391,
"longitude": -749086026
},
"name": ""
}, {
"location": {
"latitude": 418512773,
"longitude": -743067823
},
"name": ""
}, {
"location": {
"latitude": 404318328,
"longitude": -740835638
},
"name": "42-102 Main Street, Belford, NJ 07718, USA"
}, {
"location": {
"latitude": 419020746,
"longitude": -741172328
},
"name": ""
}, {
"location": {
"latitude": 404080723,
"longitude": -746119569
},
"name": ""
}, {
"location": {
"latitude": 401012643,
"longitude": -744035134
},
"name": ""
}, {
"location": {
"latitude": 404306372,
"longitude": -741079661
},
"name": ""
}, {
"location": {
"latitude": 403966326,
"longitude": -748519297
},
"name": ""
}, {
"location": {
"latitude": 405002031,
"longitude": -748407866
},
"name": ""
}, {
"location": {
"latitude": 409532885,
"longitude": -742200683
},
"name": ""
}, {
"location": {
"latitude": 416851321,
"longitude": -742674555
},
"name": ""
}, {
"location": {
"latitude": 406411633,
"longitude": -741722051
},
"name": "3387 Richmond Terrace, Staten Island, NY 10303, USA"
}, {
"location": {
"latitude": 413069058,
"longitude": -744597778
},
"name": "261 Van Sickle Road, Goshen, NY 10924, USA"
}, {
"location": {
"latitude": 418465462,
"longitude": -746859398
},
"name": ""
}, {
"location": {
"latitude": 411733222,
"longitude": -744228360
},
"name": ""
}, {
"location": {
"latitude": 410248224,
"longitude": -747127767
},
"name": "3 Hasta Way, Newton, NJ 07860, USA"
}]

View File

@ -1,37 +0,0 @@
// Copyright 2015 gRPC authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
option java_multiple_files = true;
option java_package = "io.grpc.examples.helloworld";
option java_outer_classname = "HelloWorldProto";
package helloworld;
// The greeting service definition.
service Greeter {
// Sends a greeting
rpc SayHello (HelloRequest) returns (HelloReply) {}
}
// The request message containing the user's name.
message HelloRequest {
string name = 1;
}
// The response message containing the greetings
message HelloReply {
string message = 1;
}

View File

@ -1,110 +0,0 @@
// Copyright 2015 gRPC authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
syntax = "proto3";
option java_multiple_files = true;
option java_package = "io.grpc.examples.routeguide";
option java_outer_classname = "RouteGuideProto";
package routeguide;
// Interface exported by the server.
service RouteGuide {
// A simple RPC.
//
// Obtains the feature at a given position.
//
// A feature with an empty name is returned if there's no feature at the given
// position.
rpc GetFeature(Point) returns (Feature) {}
// A server-to-client streaming RPC.
//
// Obtains the Features available within the given Rectangle. Results are
// streamed rather than returned at once (e.g. in a response message with a
// repeated field), as the rectangle may cover a large area and contain a
// huge number of features.
rpc ListFeatures(Rectangle) returns (stream Feature) {}
// A client-to-server streaming RPC.
//
// Accepts a stream of Points on a route being traversed, returning a
// RouteSummary when traversal is completed.
rpc RecordRoute(stream Point) returns (RouteSummary) {}
// A Bidirectional streaming RPC.
//
// Accepts a stream of RouteNotes sent while a route is being traversed,
// while receiving other RouteNotes (e.g. from other users).
rpc RouteChat(stream RouteNote) returns (stream RouteNote) {}
}
// Points are represented as latitude-longitude pairs in the E7 representation
// (degrees multiplied by 10**7 and rounded to the nearest integer).
// Latitudes should be in the range +/- 90 degrees and longitude should be in
// the range +/- 180 degrees (inclusive).
message Point {
int32 latitude = 1;
int32 longitude = 2;
}
// A latitude-longitude rectangle, represented as two diagonally opposite
// points "lo" and "hi".
message Rectangle {
// One corner of the rectangle.
Point lo = 1;
// The other corner of the rectangle.
Point hi = 2;
}
// A feature names something at a given point.
//
// If a feature could not be named, the name is empty.
message Feature {
// The name of the feature.
string name = 1;
// The point where the feature is detected.
Point location = 2;
}
// A RouteNote is a message sent while at a given point.
message RouteNote {
// The location from which the message is sent.
Point location = 1;
// The message to be sent.
string message = 2;
}
// A RouteSummary is received in response to a RecordRoute rpc.
//
// It contains the number of individual points received, the number of
// detected features, and the total distance covered as the cumulative sum of
// the distance between each point.
message RouteSummary {
// The number of points received.
int32 point_count = 1;
// The number of known features passed while traversing the route.
int32 feature_count = 2;
// The distance covered in metres.
int32 distance = 3;
// The duration of the traversal in seconds.
int32 elapsed_time = 4;
}

View File

@ -1,79 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
#[macro_use]
extern crate log;
extern crate prost;
#[macro_use]
extern crate prost_derive;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
extern crate tower_grpc;
mod hello_world {
include!(concat!(env!("OUT_DIR"), "/helloworld.rs"));
}
use hello_world::{server, HelloRequest, HelloReply};
use futures::{future, Future, Stream, Poll};
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::Service;
use tower_h2::Server;
use tower_grpc::{Request, Response};
#[derive(Clone, Debug)]
struct Greet;
impl Service for Greet {
type Request = Request<HelloRequest>;
type Response = Response<HelloReply>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
println!("SayHello = {:?}", request);
let response = Response::new(HelloReply {
message: "Zomg, it works!".to_string(),
});
future::ok(response)
}
}
pub fn main() {
let _ = ::env_logger::init();
let mut core = Core::new().unwrap();
let reactor = core.handle();
let new_service = server::Greeter::new_service()
.say_hello(Greet)
;
let h2 = Server::new(new_service, Default::default(), reactor.clone());
let addr = "[::1]:50051".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}

View File

@ -1,155 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
#[macro_use]
extern crate log;
extern crate prost;
#[macro_use]
extern crate prost_derive;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
extern crate tower_grpc;
extern crate serde;
extern crate serde_json;
#[macro_use]
extern crate serde_derive;
mod routeguide {
include!(concat!(env!("OUT_DIR"), "/routeguide.rs"));
}
use routeguide::{server, Point, Rectangle, Feature, RouteSummary, RouteNote};
use futures::{future, Future, Stream, Poll};
use futures::sync::mpsc;
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::Service;
use tower_h2::Server;
use tower_grpc::{Request, Response};
#[derive(Debug, Deserialize)]
pub struct Route {
}
#[derive(Debug, Deserialize)]
pub struct Location {
latitude: i32,
longitude: i32,
}
/// Handles GetFeature requests
#[derive(Clone, Debug)]
struct GetFeature;
/// Handles ListFeatures requests
#[derive(Clone, Debug)]
struct ListFeatures;
/// Handles RecordRoute requests
#[derive(Clone, Debug)]
struct RecordRoute;
/// Handles RouteChat requests
#[derive(Clone, Debug)]
struct RouteChat;
impl Service for GetFeature {
type Request = Request<Point>;
type Response = Response<Feature>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
println!("GetFeature = {:?}", request);
let response = Response::new(Feature {
name: "This is my feature".to_string(),
location: Some(request.get_ref().clone()),
});
future::ok(response)
}
}
impl Service for ListFeatures {
type Request = Request<Rectangle>;
type Response = Response<Box<Stream<Item = Feature, Error = tower_grpc::Error>>>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
unimplemented!();
}
}
impl Service for RecordRoute {
type Request = Request<tower_grpc::protobuf::server::Decode<Point>>;
type Response = Response<RouteSummary>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
unimplemented!();
}
}
impl Service for RouteChat {
type Request = Request<tower_grpc::protobuf::server::Decode<RouteNote>>;
type Response = Response<Box<Stream<Item = RouteNote, Error = tower_grpc::Error>>>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
unimplemented!();
}
}
pub fn main() {
let _ = ::env_logger::init();
let mut core = Core::new().unwrap();
let reactor = core.handle();
let new_service = server::RouteGuide::new_service()
.get_feature(GetFeature)
.list_features(ListFeatures)
;
let h2 = Server::new(new_service, Default::default(), reactor.clone());
let addr = "[::1]:50051".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}

View File

@ -1,42 +0,0 @@
#![allow(unused_imports)]
use serde_json;
use std::{env, io};
use std::io::prelude::*;
use std::fs::File;
use std::path::Path;
#[derive(Debug, Deserialize)]
struct Feature {
location: Location,
name: String,
}
#[derive(Debug, Deserialize)]
struct Location {
latitude: i32,
longitude: i32,
}
pub fn load() -> Vec<::routeguide::Feature> {
let args: Vec<_> = env::args().collect();
assert_eq!(args.len(), 2, "unexpected arguments");
let mut file = File::open(&args[1]).ok().expect("failed to open data file");
let mut data = String::new();
file.read_to_string(&mut data).ok().expect("failed to read data file");
let decoded: Vec<Feature> = serde_json::from_str(&data).unwrap();
decoded.into_iter().map(|feature| {
::routeguide::Feature {
name: feature.name,
location: Some(::routeguide::Point {
longitude: feature.location.longitude,
latitude: feature.location.latitude,
}),
}
}).collect()
}

View File

@ -1,201 +0,0 @@
#![allow(dead_code)]
#![allow(unused_variables)]
extern crate bytes;
extern crate env_logger;
extern crate futures;
#[macro_use]
extern crate log;
extern crate prost;
#[macro_use]
extern crate prost_derive;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
extern crate tower_grpc;
extern crate serde;
extern crate serde_json;
#[macro_use]
extern crate serde_derive;
mod data;
mod routeguide {
include!(concat!(env!("OUT_DIR"), "/routeguide.rs"));
}
use routeguide::{server, Point, Rectangle, Feature, RouteSummary, RouteNote};
use futures::{future, Future, Stream, Sink, Poll};
use futures::sync::mpsc;
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::Service;
use tower_h2::Server;
use tower_grpc::{Request, Response};
use std::sync::Arc;
pub type Features = Arc<Vec<routeguide::Feature>>;
/// Handles GetFeature requests
#[derive(Clone, Debug)]
struct GetFeature(Features);
/// Handles ListFeatures requests
#[derive(Clone, Debug)]
struct ListFeatures(Features);
/// Handles RecordRoute requests
#[derive(Clone, Debug)]
struct RecordRoute;
/// Handles RouteChat requests
#[derive(Clone, Debug)]
struct RouteChat;
impl Service for GetFeature {
type Request = Request<Point>;
type Response = Response<Feature>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
println!("GetFeature = {:?}", request);
for feature in &self.0[..] {
if feature.location.as_ref() == Some(request.get_ref()) {
return future::ok(Response::new(feature.clone()));
}
}
// Otherwise, return some other feature?
let response = Response::new(Feature {
name: "".to_string(),
location: None,
});
future::ok(response)
}
}
impl Service for ListFeatures {
type Request = Request<Rectangle>;
type Response = Response<Box<Stream<Item = Feature, Error = tower_grpc::Error>>>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
use std::thread;
println!("ListFeatures = {:?}", request);
let (tx, rx) = mpsc::channel(4);
let features = self.0.clone();
thread::spawn(move || {
let mut tx = tx.wait();
for feature in &features[..] {
if in_range(feature.location.as_ref().unwrap(), request.get_ref()) {
tx.send(feature.clone()).unwrap();
}
}
});
let rx = rx.map_err(|_| unimplemented!());
future::ok(Response::new(Box::new(rx)))
}
}
impl Service for RecordRoute {
type Request = Request<tower_grpc::protobuf::server::Decode<Point>>;
type Response = Response<RouteSummary>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
unimplemented!();
}
}
impl Service for RouteChat {
type Request = Request<tower_grpc::protobuf::server::Decode<RouteNote>>;
type Response = Response<Box<Stream<Item = RouteNote, Error = tower_grpc::Error>>>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
unimplemented!();
}
}
fn in_range(point: &Point, rect: &Rectangle) -> bool {
use std::cmp;
let lo = rect.lo.as_ref().unwrap();
let hi = rect.hi.as_ref().unwrap();
let left = cmp::min(lo.longitude, hi.longitude);
let right = cmp::max(lo.longitude, hi.longitude);
let top = cmp::max(lo.latitude, hi.latitude);
let bottom = cmp::min(lo.latitude, hi.latitude);
point.longitude >= left &&
point.longitude <= right &&
point.latitude >= bottom &&
point.latitude <= top
}
pub fn main() {
let _ = ::env_logger::init();
// Load data file
let data = Arc::new(data::load());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let new_service = server::RouteGuide::new_service()
.get_feature(GetFeature(data.clone()))
.list_features(ListFeatures(data.clone()))
;
let h2 = Server::new(new_service, Default::default(), reactor.clone());
let addr = "127.0.0.1:10000".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
println!("listining on {:?}", addr);
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}

View File

@ -1,30 +0,0 @@
[package]
name = "tower-grpc"
version = "0.2.0"
authors = ["Sean McArthur <sean@seanmonstar.com>"]
[features]
default = ["protobuf"]
protobuf = ["prost"]
[dependencies]
bytes = "0.4"
futures = "0.1"
http = "0.1"
h2 = { git = "https://github.com/carllerche/h2" }
log = "0.3"
tower = { git = "https://github.com/tower-rs/tower" }
tower-h2 = { path = "../tower-h2" }
tower-router = { git = "https://github.com/tower-rs/tower" }
# For protobuf
prost = { version = "0.2", optional = true }
[dev-dependencies]
env_logger = "0.4"
tokio-connect = { git = "https://github.com/carllerche/tokio-connect" }
tokio-core = "0.1"
# For examples
prost = "0.2"
prost-derive = "0.2"

View File

@ -1,209 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
extern crate http;
extern crate h2;
extern crate tokio_connect;
extern crate tokio_core;
extern crate tower;
extern crate tower_grpc;
extern crate tower_h2;
use std::net::SocketAddr;
use bytes::{Buf, BufMut};
use futures::Future;
use tokio_connect::Connect;
use tokio_core::net::TcpStream;
use tokio_core::reactor::{Core, Handle};
use tower::{Service, NewService};
use self::helloworld::{Greeter, HelloRequest, HelloReply, SayHello};
// eventually generated?
mod helloworld {
use futures::{Future, Poll};
use tower::Service;
use tower_grpc;
use tower_grpc::client::Codec;
pub struct HelloRequest {
pub name: String,
}
pub struct HelloReply {
pub message: String,
}
pub struct Greeter<SayHelloRpc> {
say_hello: SayHelloRpc,
}
impl<SayHelloRpc> Greeter<SayHelloRpc>
where
SayHelloRpc: Service<
Request=tower_grpc::Request<HelloRequest>,
Response=tower_grpc::Response<HelloReply>,
>,
{
pub fn new(say_hello: SayHelloRpc) -> Self {
Greeter {
say_hello,
}
}
pub fn say_hello(&mut self, req: HelloRequest) -> ::futures::future::Map<SayHelloRpc::Future, fn(tower_grpc::Response<HelloReply>) -> HelloReply> {
let req = tower_grpc::Request::new("/helloworld.Greeter/SayHello", req);
self.say_hello.call(req)
.map(|res| {
res.into_http().into_parts().1
} as _)
}
}
pub struct SayHello<S> {
service: S,
}
impl<C, S> SayHello<S>
where
C: Codec<Encode=HelloRequest, Decode=HelloReply>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<HelloRequest>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
>,
{
pub fn new(service: S) -> Self {
SayHello {
service,
}
}
}
impl<C, S, E> Service for SayHello<S>
where
C: Codec<Encode=HelloRequest, Decode=HelloReply>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<HelloRequest>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
Error=tower_grpc::Error<E>
>,
{
type Request = tower_grpc::Request<HelloRequest>;
type Response = tower_grpc::Response<HelloReply>;
type Error = S::Error;
type Future = tower_grpc::client::Unary<S::Future, C>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let fut = self.service.call(req.into_unary());
tower_grpc::client::Unary::map_future(fut)
}
}
}
#[derive(Clone, Copy)]
pub struct StupidCodec;
impl tower_grpc::client::codec::Codec for StupidCodec {
const CONTENT_TYPE: &'static str = "application/proto+stupid";
type Encode = HelloRequest;
type Decode = HelloReply;
type EncodeError = ();
type DecodeError = ();
fn encode(&mut self, msg: Self::Encode, buf: &mut tower_grpc::client::codec::EncodeBuf) -> Result<(), Self::EncodeError> {
buf.reserve(msg.name.len());
buf.put(msg.name.as_bytes());
Ok(())
}
fn decode(&mut self, buf: &mut tower_grpc::client::codec::DecodeBuf) -> Result<Self::Decode, Self::DecodeError> {
let s = ::std::str::from_utf8(buf.bytes()).unwrap().to_string();
buf.advance(s.len());
Ok(HelloReply {
message: s,
})
}
}
struct Conn(SocketAddr, Handle);
impl Connect for Conn {
type Connected = TcpStream;
type Error = ::std::io::Error;
type Future = Box<Future<Item = TcpStream, Error = ::std::io::Error>>;
fn connect(&self) -> Self::Future {
let c = TcpStream::connect(&self.0, &self.1)
.and_then(|tcp| tcp.set_nodelay(true).map(move |_| tcp));
Box::new(c)
}
}
struct AddOrigin<S>(S);
impl<S, B> Service for AddOrigin<S>
where
S: Service<Request=http::Request<B>>,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
fn poll_ready(&mut self) -> ::futures::Poll<(), Self::Error> {
self.0.poll_ready()
}
fn call(&mut self, mut req: Self::Request) -> Self::Future {
use std::str::FromStr;
//TODO: use Uri.into_parts() to be more efficient
let full_uri = format!("http://127.0.0.1:8888{}", req.uri().path());
let new_uri = http::Uri::from_str(&full_uri).expect("example uri should work");
*req.uri_mut() = new_uri;
self.0.call(req)
}
}
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let addr = "[::1]:8888".parse().unwrap();
let conn = Conn(addr, reactor.clone());
let h2 = tower_h2::Client::new(conn, Default::default(), reactor);
let done = h2.new_service()
.map_err(|e| unimplemented!("h2 new_service error: {:?}", e))
.and_then(move |service| {
let service = AddOrigin(service);
let grpc = tower_grpc::Client::new(StupidCodec, service);
let say_hello = SayHello::new(grpc);
let mut greeter = Greeter::new(say_hello);
greeter.say_hello(HelloRequest {
name: String::from("world"),
})
})
.map(|reply| println!("Greeter.SayHello: {}", reply.message))
.map_err(|e| println!("error: {:?}", e));
let _ = core.run(done);
}

View File

@ -1,190 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
extern crate h2;
extern crate http;
#[macro_use]
extern crate log;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
use bytes::{Buf, BufMut, Bytes, BytesMut, IntoBuf, BigEndian};
use futures::{future, Async, Future, Poll, Stream};
use http::{Request, HeaderMap};
use http::header::HeaderValue;
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::{NewService, Service};
use tower_h2::{Body, Data, Server, RecvBody};
type Response = http::Response<HelloBody>;
struct HelloBody {
message: Bytes,
status: &'static str,
}
impl HelloBody {
fn new(body: Bytes) -> Self {
HelloBody {
message: body,
status: "0",
}
}
fn unimplemented() -> Self {
HelloBody {
message: Bytes::new(),
status: "12",
}
}
fn internal() -> Self {
HelloBody {
message: Bytes::new(),
status: "13",
}
}
}
impl Body for HelloBody {
type Data = Bytes;
fn poll_data(&mut self) -> Poll<Option<Bytes>, h2::Error> {
let data = self.message.split_off(0);
let data = if data.is_empty() {
None
} else {
Some(data)
};
Ok(Async::Ready(data))
}
fn poll_trailers(&mut self) -> Poll<Option<HeaderMap>, h2::Error> {
let mut map = HeaderMap::new();
map.insert("grpc-status", HeaderValue::from_static(self.status));
Ok(Async::Ready(Some(map)))
}
}
struct RecvBodyStream(tower_h2::RecvBody);
impl Stream for RecvBodyStream {
type Item = Data;
type Error = h2::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.0.poll_data()
}
}
const SAY_HELLO: &'static str = "/helloworld.Greeter/SayHello";
#[derive(Debug)]
struct Svc;
impl Service for Svc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type Future = Box<Future<Item=Response, Error=h2::Error>>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let mut rsp = http::Response::builder();
rsp.version(http::Version::HTTP_2);
if req.uri().path() != SAY_HELLO {
println!("unknown route");
let body = HelloBody::unimplemented();
let rsp = rsp.body(body).unwrap();
return Box::new(future::ok(rsp));
}
let hello = RecvBodyStream(req.into_parts().1);
// TODO: This is not great flow control management
Box::new(hello.map(Bytes::from).concat2().and_then(move |bytes| {
let mut buf = bytes.into_buf();
let compressed_byte = buf.get_u8();
if compressed_byte == 1 {
println!("compression not supported");
let body = HelloBody::unimplemented();
let rsp = rsp.body(body).unwrap();
return Ok(rsp);
} else if compressed_byte != 0 {
println!("grpc header looked busted");
let body = HelloBody::internal();
let rsp = rsp.body(body).unwrap();
return Ok(rsp);
}
let len = buf.get_u32::<BigEndian>() as usize;
if buf.remaining() != len {
println!("delimited message claims len={}, but body={}", len, buf.remaining());
let body = HelloBody::internal();
let rsp = rsp.body(body).unwrap();
return Ok(rsp);
}
let s = ::std::str::from_utf8(buf.bytes()).unwrap();
println!("HelloRequest = {}:{:?}", len, s);
let reply = format!("Hello, {}", s);
let mut bytes = BytesMut::with_capacity(reply.len() + 5);
bytes.put_u8(0);
bytes.put_u32::<BigEndian>(reply.len() as u32);
bytes.put_slice(reply.as_bytes());
let body = HelloBody::new(bytes.freeze());
let rsp = rsp.body(body).unwrap();
Ok(rsp)
}))
}
}
#[derive(Debug)]
struct NewSvc;
impl NewService for NewSvc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type InitError = ::std::io::Error;
type Service = Svc;
type Future = future::FutureResult<Svc, Self::InitError>;
fn new_service(&self) -> Self::Future {
future::ok(Svc)
}
}
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let h2 = Server::new(NewSvc, Default::default(), reactor.clone());
let addr = "127.0.0.1:9888".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
println!("Greeter listening on {}", addr);
core.run(serve).unwrap();
}

View File

@ -1,299 +0,0 @@
#![allow(unused_variables)]
extern crate env_logger;
#[macro_use]
extern crate futures;
#[macro_use]
extern crate log;
#[macro_use]
extern crate prost_derive;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
extern crate tower_grpc;
use futures::{future, Future, Stream, Poll};
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::Service;
use tower_grpc::{Request, Response};
use tower_h2::Server;
#[derive(Clone, Debug)]
struct Greet;
impl Service for Greet {
type Request = Request<HelloRequest>;
type Response = Response<HelloReply>;
type Error = tower_grpc::Error;
type Future = future::FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let response = Response::new(HelloReply {
message: "Zomg, it works!".to_string(),
});
future::ok(response)
}
}
pub fn main() {
let _ = ::env_logger::init();
let mut core = Core::new().unwrap();
let reactor = core.handle();
let new_service = server::Greeter::new_service()
.say_hello(Greet)
;
let h2 = Server::new(new_service, Default::default(), reactor.clone());
let addr = "[::1]:50051".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}
/// The request message containing the user's name.
#[derive(Clone, Debug, PartialEq, Message)]
pub struct HelloRequest {
#[prost(string, tag="1")]
pub name: String,
}
/// The response message containing the greetings
#[derive(Clone, Debug, PartialEq, Message)]
pub struct HelloReply {
#[prost(string, tag="1")]
pub message: String,
}
pub mod server {
use super::{HelloRequest, HelloReply};
use ::tower_grpc::codegen::server::*;
#[derive(Debug)]
pub struct Greeter<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
say_hello: grpc::Grpc<grpc::Unary<SayHello, grpc::Decode<HelloRequest>>>,
}
impl Greeter<grpc::NotImplemented<HelloRequest, HelloReply>>
{
pub fn new_service() -> greeter::NewService<grpc::NotImplemented<HelloRequest, HelloReply>> {
greeter::NewService::new()
}
}
impl<SayHello> Clone for Greeter<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
fn clone(&self) -> Self {
Greeter {
say_hello: self.say_hello.clone(),
}
}
}
// ===== impl Greeter service =====
impl<SayHello> tower::Service for Greeter<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
type Request = http::Request<::tower_h2::RecvBody>;
type Response = http::Response<greeter::ResponseBody<SayHello>>;
type Error = h2::Error;
type Future = greeter::ResponseFuture<SayHello>;
fn poll_ready(&mut self) -> futures::Poll<(), Self::Error> {
// Always ready
Ok(().into())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
use self::greeter::Kind::*;
println!("PATH={:?}", request.uri().path());
match request.uri().path() {
"/helloworld.Greeter/SayHello" => {
let response = self.say_hello.call(request);
greeter::ResponseFuture { kind: Ok(SayHello(response)) }
}
_ => {
greeter::ResponseFuture { kind: Err(grpc::Status::UNIMPLEMENTED) }
}
}
}
}
pub mod greeter {
use ::tower_grpc::codegen::server::*;
use super::{Greeter, HelloRequest, HelloReply};
use std::fmt;
#[derive(Debug)]
pub struct NewService<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
inner: Greeter<SayHello>,
}
pub struct ResponseFuture<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
pub(super) kind: Result<Kind<<grpc::Grpc<grpc::Unary<SayHello, grpc::Decode<HelloRequest>>> as tower::Service>::Future>, grpc::Status>,
}
pub struct ResponseBody<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
kind: Result<Kind<grpc::Encode<<grpc::Unary<SayHello, grpc::Decode<HelloRequest>> as grpc::GrpcService>::ResponseStream>>, grpc::Status>,
}
/// Enumeration of all the service methods
#[derive(Debug)]
pub(super) enum Kind<SayHello> {
SayHello(SayHello),
}
impl NewService<grpc::NotImplemented<HelloRequest, HelloReply>>
{
pub fn new() -> Self {
NewService {
inner: Greeter {
say_hello: grpc::Grpc::new(grpc::Unary::new(grpc::NotImplemented::new())),
},
}
}
}
impl<SayHello> NewService<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
/// Set the `say_hello` method
pub fn say_hello<T>(self, say_hello: T) -> NewService<T>
where T: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
let say_hello = grpc::Grpc::new(grpc::Unary::new(say_hello));
NewService {
inner: Greeter {
say_hello,
}
}
}
}
impl<SayHello> tower::NewService for NewService<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
type Request = http::Request<::tower_h2::RecvBody>;
type Response = http::Response<ResponseBody<SayHello>>;
type Error = h2::Error;
type Service = Greeter<SayHello>;
type InitError = h2::Error;
type Future = futures::FutureResult<Self::Service, Self::Error>;
fn new_service(&self) -> Self::Future {
futures::ok(self.inner.clone())
}
}
impl<SayHello> futures::Future for ResponseFuture<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
type Item = http::Response<ResponseBody<SayHello>>;
type Error = h2::Error;
fn poll(&mut self) -> futures::Poll<Self::Item, Self::Error> {
use self::Kind::*;
match self.kind {
Ok(SayHello(ref mut fut)) => {
let response = try_ready!(fut.poll());
let (head, body) = response.into_parts();
let body = ResponseBody { kind: Ok(SayHello(body)) };
let response = http::Response::from_parts(head, body);
Ok(response.into())
}
Err(ref status) => {
let body = ResponseBody { kind: Err(status.clone()) };
Ok(grpc::Response::new(body).into_http().into())
}
}
}
}
impl<SayHello> fmt::Debug for ResponseFuture<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
write!(fmt, "ResponseFuture")
}
}
impl<SayHello> ::tower_h2::Body for ResponseBody<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
type Data = bytes::Bytes;
fn is_end_stream(&self) -> bool {
use self::Kind::*;
match self.kind {
Ok(SayHello(ref v)) => v.is_end_stream(),
Err(_) => true,
}
}
fn poll_data(&mut self) -> futures::Poll<Option<Self::Data>, h2::Error> {
use self::Kind::*;
match self.kind {
Ok(SayHello(ref mut v)) => v.poll_data(),
Err(_) => Ok(None.into()),
}
}
fn poll_trailers(&mut self) -> futures::Poll<Option<http::HeaderMap>, h2::Error> {
use self::Kind::*;
match self.kind {
Ok(SayHello(ref mut v)) => v.poll_trailers(),
Err(ref status) => {
let mut map = http::HeaderMap::new();
map.insert("grpc-status", status.to_header_value());
Ok(Some(map).into())
}
}
}
}
impl<SayHello> fmt::Debug for ResponseBody<SayHello>
where SayHello: grpc::UnaryService<Request = HelloRequest, Response = HelloReply>,
{
fn fmt(&self, fmt: &mut fmt::Formatter) -> fmt::Result {
write!(fmt, "ResponseBody")
}
}
}
}

View File

@ -1,427 +0,0 @@
#![allow(dead_code)]
#![allow(unused_imports)]
#![allow(unused_variables)]
extern crate bytes;
extern crate env_logger;
extern crate futures;
extern crate http;
extern crate h2;
extern crate tokio_connect;
extern crate tokio_core;
extern crate tower;
extern crate tower_grpc;
extern crate tower_h2;
use std::net::SocketAddr;
use bytes::BytesMut;
use futures::{Future, Stream};
use tokio_connect::Connect;
use tokio_core::net::TcpStream;
use tokio_core::reactor::{Core, Handle};
use tower::{Service, NewService};
use self::routeguide::{GetFeature, ListFeatures, RouteGuide, Point, Feature, Rectangle, RouteNote};
// eventually generated?
mod routeguide {
use futures::{Future, Poll};
use tower::Service;
use tower_grpc;
use tower_grpc::client::Codec;
#[derive(Debug)]
pub struct Point {
pub latitude: i32,
pub longitude: i32,
}
#[derive(Debug)]
pub struct Rectangle {
pub lo: Point,
pub hi: Point,
}
#[derive(Debug)]
pub struct Feature {
pub name: String,
pub location: Point,
}
#[derive(Debug)]
pub struct RouteSummary {
pub point_count: i32,
pub feature_count: i32,
pub distance: i32,
pub elapsed_time: i32,
}
#[derive(Debug)]
pub struct RouteNote {
pub location: Point,
pub message: String,
}
// the full "service"
pub struct RouteGuide<GetFeatureRpc, ListFeaturesRpc, RecordRouteRpc, RouteChatRpc> {
get_feature: GetFeatureRpc,
list_features: ListFeaturesRpc,
record_route: RecordRouteRpc,
route_chat: RouteChatRpc,
}
impl<GetFeatureRpc, ListFeaturesRpc, RecordRouteRpc, RouteChatRpc, ListFeaturesStream>
RouteGuide<GetFeatureRpc, ListFeaturesRpc, RecordRouteRpc, RouteChatRpc>
where
GetFeatureRpc: Service<
Request=tower_grpc::Request<Point>,
Response=tower_grpc::Response<Feature>,
>,
ListFeaturesRpc: Service<
Request=tower_grpc::Request<Rectangle>,
Response=tower_grpc::Response<ListFeaturesStream>,
>,
ListFeaturesStream: ::futures::Stream<Item=Feature>,
{
pub fn new(get_feature: GetFeatureRpc, list_features: ListFeaturesRpc, record_route: RecordRouteRpc, route_chat: RouteChatRpc) -> Self {
RouteGuide {
get_feature,
list_features,
record_route,
route_chat,
}
}
pub fn get_feature(&mut self, req: Point) -> ::futures::future::Map<GetFeatureRpc::Future, fn(tower_grpc::Response<Feature>) -> Feature>
{
let req = tower_grpc::Request::new("/routeguide.RouteGuide/GetFeature", req);
self.get_feature.call(req)
.map(|res| {
res.into_http().into_parts().1
} as _)
}
//TODO: should this return a Stream wrapping the future?
pub fn list_features(&mut self, req: Rectangle) -> ::futures::future::Map<ListFeaturesRpc::Future, fn(tower_grpc::Response<ListFeaturesStream>) -> ListFeaturesStream>
{
let req = tower_grpc::Request::new("/routeguide.RouteGuide/GetFeature", req);
self.list_features.call(req)
.map(|res| {
res.into_http().into_parts().1
} as _)
}
}
// rpc methods
pub struct GetFeature<S> {
service: S,
}
impl<C, S, E> GetFeature<S>
where
C: Codec<Encode=Point, Decode=Feature>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<Point>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
Error=tower_grpc::Error<E>
>,
{
pub fn new(service: S) -> Self {
GetFeature {
service,
}
}
}
impl<C, S, E> Service for GetFeature<S>
where
C: Codec<Encode=Point, Decode=Feature>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<Point>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
Error=tower_grpc::Error<E>
>,
{
type Request = tower_grpc::Request<Point>;
type Response = tower_grpc::Response<Feature>;
type Error = S::Error;
type Future = tower_grpc::client::Unary<S::Future, C>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let fut = self.service.call(req.into_unary());
tower_grpc::client::Unary::map_future(fut)
}
}
pub struct ListFeatures<S> {
service: S,
}
impl<C, S, E> ListFeatures<S>
where
C: Codec<Encode=Rectangle, Decode=Feature>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<Rectangle>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
Error=tower_grpc::Error<E>
>,
{
pub fn new(service: S) -> Self {
ListFeatures {
service,
}
}
}
impl<C, S, E> Service for ListFeatures<S>
where
C: Codec<Encode=Rectangle, Decode=Feature>,
S: Service<
Request=tower_grpc::Request<
tower_grpc::client::codec::Unary<Rectangle>
>,
Response=tower_grpc::Response<
tower_grpc::client::codec::DecodingBody<C>
>,
Error=tower_grpc::Error<E>
>,
{
type Request = tower_grpc::Request<Rectangle>;
type Response = tower_grpc::Response<tower_grpc::client::codec::DecodingBody<C>>;
type Error = S::Error;
type Future = S::Future;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
}
fn call(&mut self, req: Self::Request) -> Self::Future {
self.service.call(req.into_unary())
}
}
pub struct RecordRoute<S> {
service: S,
}
pub struct RouteChat<S> {
service: S,
}
}
pub struct StupidCodec<T, U>(::std::marker::PhantomData<(T, U)>);
impl<T, U> StupidCodec<T, U> {
fn new() -> Self {
StupidCodec(::std::marker::PhantomData)
}
}
impl<T, U> Clone for StupidCodec<T, U> {
fn clone(&self) -> Self {
StupidCodec(::std::marker::PhantomData)
}
}
impl tower_grpc::client::Codec for StupidCodec<Point, Feature> {
const CONTENT_TYPE: &'static str = "application/proto+stupid";
type Encode = Point;
type Decode = Feature;
type EncodeError = ();
type DecodeError = ();
fn encode(&mut self, msg: Self::Encode, buf: &mut tower_grpc::client::codec::EncodeBuf) -> Result<(), Self::EncodeError> {
Ok(())
}
fn decode(&mut self, buf: &mut tower_grpc::client::codec::DecodeBuf) -> Result<Self::Decode, Self::DecodeError> {
Ok(Feature {
name: String::from("faked"),
location: Point {
longitude: 5,
latitude: 5,
}
})
}
}
impl tower_grpc::client::Codec for StupidCodec<Rectangle, Feature> {
const CONTENT_TYPE: &'static str = "application/proto+stupid";
type Encode = Rectangle;
type Decode = Feature;
type EncodeError = ();
type DecodeError = ();
fn encode(&mut self, msg: Self::Encode, buf: &mut tower_grpc::client::codec::EncodeBuf) -> Result<(), Self::EncodeError> {
Ok(())
}
fn decode(&mut self, buf: &mut tower_grpc::client::codec::DecodeBuf) -> Result<Self::Decode, Self::DecodeError> {
Ok(Feature {
name: String::from("faked"),
location: Point {
longitude: 5,
latitude: 5,
}
})
}
}
struct Conn(SocketAddr, Handle);
impl Connect for Conn {
type Connected = TcpStream;
type Error = ::std::io::Error;
type Future = Box<Future<Item = TcpStream, Error = ::std::io::Error>>;
fn connect(&self) -> Self::Future {
let c = TcpStream::connect(&self.0, &self.1)
.and_then(|tcp| tcp.set_nodelay(true).map(move |_| tcp));
Box::new(c)
}
}
struct AddOrigin<S>(S);
impl<S, B> Service for AddOrigin<S>
where
S: Service<Request=http::Request<B>>,
{
type Request = S::Request;
type Response = S::Response;
type Error = S::Error;
type Future = S::Future;
fn poll_ready(&mut self) -> ::futures::Poll<(), Self::Error> {
self.0.poll_ready()
}
fn call(&mut self, mut req: Self::Request) -> Self::Future {
use std::str::FromStr;
//TODO: use Uri.into_parts() to be more efficient
let full_uri = format!("http://127.0.0.1:8888{}", req.uri().path());
let new_uri = http::Uri::from_str(&full_uri).expect("example uri should work");
*req.uri_mut() = new_uri;
self.0.call(req)
}
}
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let addr = "[::1]:8888".parse().unwrap();
let conn = Conn(addr, reactor.clone());
let h2 = tower_h2::Client::new(conn, Default::default(), reactor);
let done = h2.new_service()
.map_err(|_e| unimplemented!("h2 new_service error"))
.and_then(move |orig_service| {
let service = AddOrigin(orig_service.clone_handle());
let grpc = tower_grpc::Client::new(StupidCodec::<Point, Feature>::new(), service);
let get_feature = GetFeature::new(grpc);
let service = AddOrigin(orig_service);
let grpc = tower_grpc::Client::new(StupidCodec::<Rectangle, Feature>::new(), service);
let list_features = ListFeatures::new(grpc);
let mut client = RouteGuide::new(get_feature, list_features, (), ());
let valid_feature = client.get_feature(Point {
latitude: 409146138,
longitude: -746188906,
}).map(|feature| {
println!("GetFeature: {:?}", feature);
}).map_err(|e| ("GetFeature", e));
let missing_feature = client.get_feature(Point {
latitude: 0,
longitude: 0,
}).map(|feature| {
println!("GetFeature: {:?}", feature);
}).map_err(|e| ("GetFeature", e));
let features_between = client.list_features(Rectangle {
lo: Point {
latitude: 400000000,
longitude: -750000000,
},
hi: Point {
latitude: 420000000,
longitude: -730000000,
}
}).and_then(|features| {
features.for_each(|feature| {
println!("ListFeatures: {:?}", feature);
Ok(())
}).map_err(|e| match e {
tower_grpc::Error::Inner(h2) => tower_grpc::Error::Inner(h2.into()),
tower_grpc::Error::Grpc(status) => tower_grpc::Error::Grpc(status),
})
}).map_err(|e| ("ListFeatures", e));
/*
let record_route = client.record_route(futures::stream::iter_ok::<_, ()>(vec![
Point {
longitude: 1,
latitude: 1,
},
Point {
longitude: 2,
latitude: 2,
},
])).map(|summary| {
println!("RecordRoute: {:?}", summary);
}).map_err(|e| ("RecordRoute", e));
let route_chat = client.route_chat(futures::stream::iter_ok::<_, ()>(vec![
RouteNote {
location: Point {
longitude: 55,
latitude: 55,
},
message: "First note".to_string(),
},
])).for_each(|_| {
Ok(())
}).map_err(|e| ("RouteChat", e));
*/
valid_feature
.join(missing_feature)
//.join(features_between)
//.join(record_route)
//.join(route_chat)
})
.map_err(|e| println!("error: {:?}", e));
let _ = core.run(done);
}

View File

@ -1,158 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
extern crate h2;
extern crate http;
#[macro_use]
extern crate log;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
use bytes::Bytes;
use futures::*;
use http::{Request, HeaderMap};
use http::header::HeaderValue;
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::{NewService, Service};
use tower_h2::{Body, Data, Server, RecvBody};
type Response = http::Response<GrpcBody>;
struct GrpcBody {
message: Bytes,
status: &'static str,
}
impl GrpcBody {
fn new(body: Bytes) -> Self {
GrpcBody {
message: body,
status: "0",
}
}
fn unimplemented() -> Self {
GrpcBody {
message: Bytes::new(),
status: "12",
}
}
}
impl Body for GrpcBody {
type Data = Bytes;
fn poll_data(&mut self) -> Poll<Option<Bytes>, h2::Error> {
let data = self.message.split_off(0);
let data = if data.is_empty() {
None
} else {
Some(data)
};
Ok(Async::Ready(data))
}
fn poll_trailers(&mut self) -> Poll<Option<HeaderMap>, h2::Error> {
let mut map = HeaderMap::new();
map.insert("grpc-status", HeaderValue::from_static(self.status));
Ok(Async::Ready(Some(map)))
}
}
struct RecvBodyStream(tower_h2::RecvBody);
impl Stream for RecvBodyStream {
type Item = Data;
type Error = h2::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.0.poll_data()
}
}
const GET_FEATURE: &'static str = "/routeguide.RouteGuide/GetFeature";
#[derive(Debug)]
struct Svc;
impl Service for Svc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type Future = Box<Future<Item=Response, Error=h2::Error>>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Request<RecvBody>) -> Self::Future {
let mut rsp = http::Response::builder();
rsp.version(http::Version::HTTP_2);
let (head, body) = req.into_parts();
match head.uri.path() {
GET_FEATURE => {
let body = RecvBodyStream(body);
// TODO: This is not great flow control management
Box::new(body.map(Bytes::from).concat2().and_then(move |bytes| {
let s = ::std::str::from_utf8(&bytes).unwrap();
println!("GetFeature = {:?}", s);
let body = GrpcBody::new("blah".into());
let rsp = rsp.body(body).unwrap();
Ok(rsp)
}))
},
_ => {
println!("unknown route");
let body = GrpcBody::unimplemented();
let rsp = rsp.body(body).unwrap();
Box::new(future::ok(rsp))
}
}
}
}
#[derive(Debug)]
struct NewSvc;
impl NewService for NewSvc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type InitError = ::std::io::Error;
type Service = Svc;
type Future = future::FutureResult<Svc, Self::InitError>;
fn new_service(&self) -> Self::Future {
future::ok(Svc)
}
}
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let h2 = Server::new(NewSvc, Default::default(), reactor.clone());
let addr = "[::1]:8888".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}

View File

@ -1,383 +0,0 @@
use std::collections::VecDeque;
use bytes::{Buf, BufMut, Bytes, BytesMut, BigEndian};
use futures::{Async, Stream, Poll};
use h2;
use http::header::HeaderMap;
use tower_h2::{self, Body, Data, RecvBody};
use ::Status;
use super::check_grpc_status;
/// A type used to encode and decode for a single RPC method.
pub trait Codec: Clone {
/// The content-type header for messages using this encoding.
///
/// Should be `application/grpc+yourencoding`.
const CONTENT_TYPE: &'static str;
/// The message to encode into bytes.
type Encode;
/// The message to decode from bytes.
type Decode;
/// An error that could occur during encoding.
type EncodeError;
/// An error that could occur during decoding.
type DecodeError;
/// Encode a message into the provided buffer.
fn encode(&mut self, item: Self::Encode, buf: &mut EncodeBuf) -> Result<(), Self::EncodeError>;
/// Decode a message from the buffer.
///
/// The buffer will contain exactly the bytes of a full message. There
/// is no need to get the length from the bytes, gRPC framing is handled
/// for you.
fn decode(&mut self, buf: &mut DecodeBuf) -> Result<Self::Decode, Self::DecodeError>;
}
/// A buffer to encode a message into.
#[derive(Debug)]
pub struct EncodeBuf<'a> {
pub(crate) bytes: &'a mut BytesMut,
}
/// A buffer to decode messages from.
#[derive(Debug)]
pub struct DecodeBuf<'a> {
pub(crate) bufs: &'a mut BytesList,
pub(crate) len: usize,
}
/// A mapping of a stream of encodable items to a stream of bytes.
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct EncodingBody<E, S> {
buf: BytesMut,
encoder: E,
stream: S,
}
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct DecodingBody<D> {
bufs: BytesList,
decoder: D,
state: DecodingState,
stream: RecvBody,
}
#[derive(Debug)]
enum DecodingState {
ReadHeader,
ReadBody {
compression: bool,
len: usize,
},
Trailers,
Done,
}
#[derive(Debug)]
pub(crate) struct BytesList {
pub(crate) bufs: VecDeque<Data>,
}
impl<'a> EncodeBuf<'a> {
#[inline]
pub fn reserve(&mut self, capacity: usize) {
self.bytes.reserve(capacity);
}
}
impl<'a> BufMut for EncodeBuf<'a> {
#[inline]
fn remaining_mut(&self) -> usize {
self.bytes.remaining_mut()
}
#[inline]
unsafe fn advance_mut(&mut self, cnt: usize) {
self.bytes.advance_mut(cnt)
}
#[inline]
unsafe fn bytes_mut(&mut self) -> &mut [u8] {
self.bytes.bytes_mut()
}
}
impl<'a> Buf for DecodeBuf<'a> {
#[inline]
fn remaining(&self) -> usize {
self.len
}
#[inline]
fn bytes(&self) -> &[u8] {
&self.bufs.bytes()[..self.len]
}
#[inline]
fn advance(&mut self, cnt: usize) {
assert!(cnt <= self.len);
self.bufs.advance(cnt);
self.len -= cnt;
}
}
impl<'a> Drop for DecodeBuf<'a> {
fn drop(&mut self) {
if self.len > 0 {
warn!("DecodeBuf was not advanced to end");
self.bufs.advance(self.len);
}
}
}
impl Buf for BytesList {
#[inline]
fn remaining(&self) -> usize {
self.bufs.iter().map(|buf| buf.remaining()).sum()
}
#[inline]
fn bytes(&self) -> &[u8] {
if self.bufs.is_empty() {
&[]
} else {
&self.bufs[0].bytes()
}
}
#[inline]
fn advance(&mut self, mut cnt: usize) {
while cnt > 0 {
{
let front = &mut self.bufs[0];
if front.remaining() > cnt {
front.advance(cnt);
return;
} else {
cnt -= front.remaining();
}
}
self.bufs.pop_front();
}
}
}
impl<E, S> EncodingBody<E, S> {
pub(crate) fn new(encoder: E, stream: S) -> Self {
EncodingBody {
buf: BytesMut::new(),
encoder,
stream,
}
}
}
impl<E, S> tower_h2::Body for EncodingBody<E, S>
where
S: Stream,
E: Codec<Encode=S::Item>,
{
type Data = Bytes;
fn is_end_stream(&self) -> bool {
false
}
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error> {
let item = try_ready!(self.stream.poll().map_err(|_| h2_err()));
if let Some(item) = item {
self.buf.reserve(5);
unsafe { self.buf.advance_mut(5); }
self.encoder.encode(item, &mut EncodeBuf {
bytes: &mut self.buf,
}).map_err(|_| h2_err())?;
// now that we know length, we can write the header
let len = self.buf.len() - 5;
assert!(len <= ::std::u32::MAX as usize);
{
let mut cursor = ::std::io::Cursor::new(&mut self.buf[..5]);
cursor.put_u8(0); // byte must be 0, reserve doesn't auto-zero
cursor.put_u32::<BigEndian>(len as u32);
}
Ok(Async::Ready(Some(self.buf.split_to(len + 5).freeze())))
} else {
Ok(Async::Ready(None))
}
}
}
fn grpc_status(trailers: &HeaderMap) -> Result<(), Status> {
match check_grpc_status(&trailers) {
Some(status) => if status.code() == ::Code::OK {
Ok(())
} else {
Err(status)
}
None => {
trace!("trailers missing grpc-status");
Err(Status::UNKNOWN)
}
}
}
impl<D> DecodingBody<D>
where
D: Codec,
{
pub(crate) fn new(decoder: D, stream: RecvBody) -> Self {
DecodingBody {
bufs: BytesList {
bufs: VecDeque::new(),
},
decoder,
state: DecodingState::ReadHeader,
stream,
}
}
fn decode(&mut self) -> Result<Option<D::Decode>, Status> {
if let DecodingState::ReadHeader = self.state {
if self.bufs.remaining() < 5 {
return Ok(None);
}
let is_compressed = match self.bufs.get_u8() {
0 => false,
1 => {
trace!("message compressed, compression not supported yet");
return Err(Status::UNIMPLEMENTED);
},
_ => {
trace!("unexpected compression flag");
return Err(Status::UNKNOWN);
}
};
let len = self.bufs.get_u32::<BigEndian>() as usize;
self.state = DecodingState::ReadBody {
compression: is_compressed,
len,
}
}
if let DecodingState::ReadBody { len, .. } = self.state {
if self.bufs.remaining() < len {
return Ok(None);
}
match self.decoder.decode(&mut DecodeBuf {
bufs: &mut self.bufs,
len,
}) {
Ok(msg) => {
self.state = DecodingState::ReadHeader;
return Ok(Some(msg));
},
Err(_) => {
debug!("decoder error");
return Err(Status::UNKNOWN);
}
}
}
Ok(None)
}
fn poll_inner(&mut self) -> Poll<Option<D::Decode>, ::Error<h2::Error>> {
loop {
match self.state {
DecodingState::Trailers | DecodingState::Done => break,
_ => (),
}
match self.decode() {
Ok(Some(val)) => return Ok(Async::Ready(Some(val))),
Ok(None) => (),
Err(status) => return Err(::Error::Grpc(status)),
}
let chunk = try_ready!(self.stream.poll_data());
if let Some(data) = chunk {
self.bufs.bufs.push_back(data);
} else {
if self.bufs.has_remaining() {
trace!("unexpected EOF decoding stream");
return Err(::Error::Grpc(Status::UNKNOWN))
} else {
self.state = DecodingState::Trailers;
break;
}
}
}
if let DecodingState::Trailers = self.state {
return if let Some(trailers) = try_ready!(self.stream.poll_trailers()) {
grpc_status(&trailers).map_err(::Error::Grpc)?;
self.state = DecodingState::Done;
Ok(Async::Ready(None))
} else {
trace!("receive body ended without trailers");
Err(::Error::Grpc(Status::UNKNOWN))
}
}
Ok(Async::Ready(None))
}
}
impl<D> Stream for DecodingBody<D>
where
D: Codec,
{
type Item = D::Decode;
type Error = ::Error<h2::Error>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
self.poll_inner()
.map_err(|err| {
self.state = DecodingState::Done;
err
})
}
}
fn h2_err() -> h2::Error {
unimplemented!("EncodingBody map_err")
}
/// Wraps a message to provide a `Stream` of just one item.
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct Unary<T> {
item: Option<T>,
}
impl<T> Unary<T> {
pub fn new(item: T) -> Self {
Unary {
item: Some(item),
}
}
}
impl<T> Stream for Unary<T> {
type Item = T;
type Error = self::inner::Void;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
Ok(Async::Ready(self.item.take()))
}
}
mod inner {
pub struct Void(Void_);
enum Void_ {}
}

View File

@ -1,260 +0,0 @@
pub mod codec;
use futures::{Async, Future, Poll, Stream};
use http;
use http::header::{HeaderMap, HeaderValue};
use tower::Service;
use tower_h2::RecvBody;
pub use self::codec::Codec;
use self::codec::{DecodingBody, EncodingBody};
use ::Status;
/// A gRPC client wrapping a `Service` over `h2`.
#[derive(Debug)]
pub struct Client<C, S> {
codec: C,
service: S,
}
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct ResponseFuture<D, F> {
decoder: Option<D>,
future: F,
}
/// Future mapping Response<B> into B.
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct BodyFuture<F> {
future: F,
}
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct Unary<F, D> where D: Codec {
body: Option<DecodingBody<D>>,
future: F,
head: Option<http::response::Parts>,
message: Option<D::Decode>,
}
/// A stream of a future Response's body items.
#[must_use = "streams do nothing unless polled"]
#[derive(Debug)]
pub struct Streaming<F, B> {
body: Option<B>,
future: F,
}
// ====== impl Client =====
impl<C, S> Client<C, S> {
/// Create a new `Client` over an h2 service.
pub fn new(codec: C, service: S) -> Self {
Client {
codec,
service,
}
}
}
impl<C, S, R> Service for Client<C, S>
where
C: Codec,
S: Service<Request=http::Request<EncodingBody<C, R>>, Response=http::Response<RecvBody>>,
R: Stream<Item=C::Encode>,
{
type Request = ::Request<R>;
type Response = ::Response<DecodingBody<C>>;
type Error = ::Error<S::Error>;
type Future = ResponseFuture<C, S::Future>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.service.poll_ready()
.map_err(::Error::Inner)
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let http = req.into_http();
let (mut head, body) = http.into_parts();
// gRPC headers
head.headers.insert(http::header::TE, HeaderValue::from_static("trailers"));
let content_type = HeaderValue::from_static(C::CONTENT_TYPE);
head.headers.insert(http::header::CONTENT_TYPE, content_type);
let encoded = EncodingBody::new(self.codec.clone(), body);
let req = http::Request::from_parts(head, encoded);
let fut = self.service.call(req);
ResponseFuture {
decoder: Some(self.codec.clone()),
future: fut,
}
}
}
// ====== impl ResponseFuture =====
impl<D, F> Future for ResponseFuture<D, F>
where
D: Codec,
F: Future<Item=http::Response<RecvBody>>,
{
type Item = ::Response<DecodingBody<D>>;
type Error = ::Error<F::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let res = try_ready!(self.future.poll().map_err(::Error::Inner));
let (head, body) = res.into_parts();
if let Some(status) = check_grpc_status(&head.headers) {
return Err(::Error::Grpc(status));
}
let decoded = DecodingBody::new(self.decoder.take().unwrap(), body);
let res = http::Response::from_parts(head, decoded);
let grpc = ::Response::from_http(res);
Ok(Async::Ready(grpc))
}
}
// ====== impl Unary =====
impl<F, D> Unary<F, D>
where
D: Codec,
{
pub fn map_future(future: F) -> Self {
Unary {
body: None,
future,
head: None,
message: None,
}
}
}
impl<F, D, E> Future for Unary<F, D>
where
F: Future<Item=::Response<DecodingBody<D>>, Error=::Error<E>>,
D: Codec,
{
type Item = ::Response<D::Decode>;
type Error = ::Error<E>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let ref mut body = match self.body {
Some(ref mut body) => body,
None => {
let resp = try_ready!(self.future.poll()).into_http();
let (head, body) = resp.into_parts();
self.head = Some(head);
self.body = Some(body);
self.body.as_mut().unwrap()
}
};
loop {
let message = try_ready!(body.poll()
.map_err(|e| match e {
::Error::Inner(h2) => ::Error::Grpc(::Status::from(h2)),
::Error::Grpc(err) => ::Error::Grpc(err),
}));
match (self.message.is_some(), message) {
(false, Some(msg)) => {
self.message = Some(msg);
continue;
},
(true, None) => {
let head = self.head.take().expect("polled more than once");
let body = self.message.take().expect("polled more than once");
let http = http::Response::from_parts(head, body);
let resp = ::Response::from_http(http);
return Ok(Async::Ready(resp));
}
(true, Some(_)) => {
debug!("Unary decoder found 2 messages");
return Err(::Error::Grpc(Status::UNKNOWN));
}
(false, None) => {
debug!("Unary decoder ended before any messages");
return Err(::Error::Grpc(Status::UNKNOWN));
}
}
}
}
}
// ====== impl Stream =====
impl<F, B> Streaming<F, B>
where
F: Future<Item=::Response<B>>,
B: Stream<Error=::Error<::h2::Error>>,
{
pub fn map_future(future: F) -> Self {
Streaming {
body: None,
future,
}
}
}
impl<F, B, E> Stream for Streaming<F, B>
where
F: Future<Item=::Response<B>, Error=::Error<E>>,
B: Stream<Error=::Error<::h2::Error>>,
{
type Item = B::Item;
type Error = ::Error<E>;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
loop {
if let Some(ref mut body) = self.body {
return body.poll().map_err(|e| match e {
::Error::Inner(h2) => ::Error::Grpc(::Status::from(h2)),
::Error::Grpc(err) => ::Error::Grpc(err),
});
} else {
let res = try_ready!(self.future.poll());
self.body = Some(res.into_http().into_parts().1);
}
}
}
}
// ====== impl BodyFuture =====
impl<F> BodyFuture<F> {
/// Wrap the future.
pub fn new(fut: F) -> Self {
BodyFuture {
future: fut,
}
}
}
impl<F, B> Future for BodyFuture<F>
where
F: Future<Item=::Response<B>>,
{
type Item = B;
type Error = F::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let res = try_ready!(self.future.poll());
Ok(Async::Ready(res.into_http().into_parts().1))
}
}
fn check_grpc_status(trailers: &HeaderMap) -> Option<Status> {
trailers.get("grpc-status").map(|s| {
Status::from_bytes(s.as_ref())
})
}

View File

@ -1,27 +0,0 @@
use h2;
#[derive(Debug)]
pub enum Error<T = ()> {
Grpc(::Status),
Inner(T),
}
impl<T> From<T> for Error<T> {
fn from(inner: T) -> Self {
Error::Inner(inner)
}
}
impl From<Error<()>> for h2::Error {
fn from(_err: Error<()>) -> Self {
// TODO: implement
h2::Reason::INTERNAL_ERROR.into()
}
}
impl From<h2::Error> for Error<()> {
fn from(_: h2::Error) -> Self {
// TODO: implement
Error::Inner(())
}
}

View File

@ -1,89 +0,0 @@
#![deny(warnings)]
//#![deny(missing_docs)]
//#![deny(missing_debug_implementations)]
extern crate bytes;
#[macro_use] extern crate futures;
extern crate http;
extern crate h2;
#[macro_use] extern crate log;
extern crate tower;
extern crate tower_h2;
#[cfg(feature = "protobuf")]
extern crate prost;
pub mod client;
pub mod server;
#[cfg(feature = "protobuf")]
pub mod protobuf;
mod error;
mod request;
mod response;
mod status;
pub use self::client::Client;
pub use self::error::Error;
pub use self::status::{Code, Status};
pub use self::request::Request;
pub use self::response::Response;
/// Type re-exports used by generated code
pub mod codegen {
/// Type re-exports used by generated server code
pub mod server {
/// Re-export types from this crate
pub mod grpc {
pub use ::{Request, Response, Error, Status};
pub use ::server::{
unary,
Unary,
ClientStreaming,
ServerStreaming,
NotImplemented,
};
pub use ::protobuf::server::{
Grpc,
GrpcService,
UnaryService,
ClientStreamingService,
ServerStreamingService,
Encode,
Decode,
};
}
/// Re-export types from the `bytes` crate.
pub mod bytes {
pub use ::bytes::Bytes;
}
/// Re-export types from the `future` crate.
pub mod futures {
pub use ::futures::{Future, Poll, Async};
pub use ::futures::future::{FutureResult, ok};
}
/// Re-exported types from the `http` crate.
pub mod http {
pub use ::http::{Request, Response, HeaderMap};
}
/// Re-exported types from the `h2` crate.
pub mod h2 {
pub use ::h2::Error;
}
/// Re-export types from the `tower_h2` crate
pub mod tower_h2 {
pub use ::tower_h2::{Body, RecvBody};
}
/// Re-exported types from the `tower` crate.
pub mod tower {
pub use ::tower::{Service, NewService};
}
}
}

View File

@ -1,433 +0,0 @@
use client::codec::{EncodeBuf, DecodeBuf};
use bytes::{BufMut};
use prost::Message;
use ::std::marker::PhantomData;
/// Protobuf codec
#[derive(Debug)]
pub struct Codec<T, U>(PhantomData<(T, U)>);
#[derive(Debug)]
pub struct Encoder<T>(PhantomData<T>);
#[derive(Debug)]
pub struct Decoder<T>(PhantomData<T>);
/// Protobuf gRPC type aliases
pub mod server {
use {Request, Response};
use futures::{Future, Stream, Poll};
use {h2, http};
use tower::Service;
/// A specialization of tower::Service.
///
/// Existing tower::Service implementations with the correct form will
/// automatically implement `GrpcService`.
///
/// TODO: Rename to StreamingService?
pub trait GrpcService: Clone {
/// Protobuf request message type
type Request;
/// Stream of inbound request messages
type RequestStream: Stream<Item = Self::Request, Error = ::Error>;
/// Protobuf response message type
type Response;
/// Stream of outbound response messages
type ResponseStream: Stream<Item = Self::Response, Error = ::Error>;
/// Response future
type Future: Future<Item = ::Response<Self::ResponseStream>, Error = ::Error>;
/// Returns `Ready` when the service can accept a request
fn poll_ready(&mut self) -> Poll<(), ::Error>;
/// Call the service
fn call(&mut self, request: Request<Self::RequestStream>) -> Self::Future;
}
impl<T, S1, S2> GrpcService for T
where T: Service<Request = Request<S1>,
Response = Response<S2>,
Error = ::Error> + Clone,
S1: Stream<Error = ::Error>,
S2: Stream<Error = ::Error>,
{
type Request = S1::Item;
type RequestStream = S1;
type Response = S2::Item;
type ResponseStream = S2;
type Future = T::Future;
fn poll_ready(&mut self) -> Poll<(), ::Error> {
Service::poll_ready(self)
}
fn call(&mut self, request: T::Request) -> Self::Future {
Service::call(self, request)
}
}
/// A specialization of tower::Service.
///
/// Existing tower::Service implementations with the correct form will
/// automatically implement `UnaryService`.
pub trait UnaryService: Clone {
/// Protobuf request message type
type Request;
/// Protobuf response message type
type Response;
/// Response future
type Future: Future<Item = ::Response<Self::Response>, Error = ::Error>;
/// Returns `Ready` when the service can accept a request
fn poll_ready(&mut self) -> Poll<(), ::Error>;
/// Call the service
fn call(&mut self, request: Request<Self::Request>) -> Self::Future;
}
impl<T, M1, M2> UnaryService for T
where T: Service<Request = Request<M1>,
Response = Response<M2>,
Error = ::Error> + Clone,
{
type Request = M1;
type Response = M2;
type Future = T::Future;
fn poll_ready(&mut self) -> Poll<(), ::Error> {
Service::poll_ready(self)
}
fn call(&mut self, request: T::Request) -> Self::Future {
Service::call(self, request)
}
}
/// A specialization of tower::Service.
///
/// Existing tower::Service implementations with the correct form will
/// automatically implement `UnaryService`.
pub trait ClientStreamingService: Clone {
/// Protobuf request message type
type Request;
/// Stream of inbound request messages
type RequestStream: Stream<Item = Self::Request, Error = ::Error>;
/// Protobuf response message type
type Response;
/// Response future
type Future: Future<Item = ::Response<Self::Response>, Error = ::Error>;
/// Returns `Ready` when the service can accept a request
fn poll_ready(&mut self) -> Poll<(), ::Error>;
/// Call the service
fn call(&mut self, request: Request<Self::RequestStream>) -> Self::Future;
}
impl<T, M, S> ClientStreamingService for T
where T: Service<Request = Request<S>,
Response = Response<M>,
Error = ::Error> + Clone,
S: Stream<Error = ::Error>,
{
type Request = S::Item;
type RequestStream = S;
type Response = M;
type Future = T::Future;
fn poll_ready(&mut self) -> Poll<(), ::Error> {
Service::poll_ready(self)
}
fn call(&mut self, request: T::Request) -> Self::Future {
Service::call(self, request)
}
}
/// A specialization of tower::Service.
///
/// Existing tower::Service implementations with the correct form will
/// automatically implement `UnaryService`.
pub trait ServerStreamingService: Clone {
/// Protobuf request message type
type Request;
/// Protobuf response message type
type Response;
/// Stream of outbound response messages
type ResponseStream: Stream<Item = Self::Response, Error = ::Error>;
/// Response future
type Future: Future<Item = ::Response<Self::ResponseStream>, Error = ::Error>;
/// Returns `Ready` when the service can accept a request
fn poll_ready(&mut self) -> Poll<(), ::Error>;
/// Call the service
fn call(&mut self, request: Request<Self::Request>) -> Self::Future;
}
impl<T, M, S> ServerStreamingService for T
where T: Service<Request = Request<M>,
Response = Response<S>,
Error = ::Error> + Clone,
S: Stream<Error = ::Error>,
{
type Request = M;
type Response = S::Item;
type ResponseStream = S;
type Future = T::Future;
fn poll_ready(&mut self) -> Poll<(), ::Error> {
Service::poll_ready(self)
}
fn call(&mut self, request: T::Request) -> Self::Future {
Service::call(self, request)
}
}
#[derive(Debug)]
pub struct Grpc<T>
where T: GrpcService,
{
inner: ::server::Grpc<Wrap<T>, ::protobuf::Codec<T::Response, T::Request>>,
}
#[derive(Debug)]
pub struct ResponseFuture<T>
where T: GrpcService,
{
inner: ::server::streaming::ResponseFuture<T::Future, ::protobuf::Encoder<T::Response>>,
}
/// A protobuf encoded gRPC request stream
#[derive(Debug)]
pub struct Decode<T> {
inner: ::server::Decode<::protobuf::Decoder<T>>,
}
/// A protobuf encoded gRPC response body
pub struct Encode<T>
where T: Stream,
{
inner: ::server::Encode<T, ::protobuf::Encoder<T::Item>>,
}
// ===== impl Grpc =====
impl<T, U> Grpc<T>
where T: GrpcService<Request = U, RequestStream = Decode<U>>,
T::Request: ::prost::Message + Default,
T::Response: ::prost::Message,
{
pub fn new(inner: T) -> Self {
let inner = ::server::Grpc::new(Wrap(inner), ::protobuf::Codec::new());
Grpc { inner }
}
}
impl<T, U> Service for Grpc<T>
where T: GrpcService<Request = U, RequestStream = Decode<U>>,
T::Request: ::prost::Message + Default,
T::Response: ::prost::Message,
{
type Request = ::http::Request<::tower_h2::RecvBody>;
type Response = ::http::Response<Encode<T::ResponseStream>>;
type Error = ::h2::Error;
type Future = ResponseFuture<T>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.inner.poll_ready()
.map_err(Into::into)
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let inner = self.inner.call(request);
ResponseFuture { inner }
}
}
impl<T> Clone for Grpc<T>
where T: GrpcService + Clone,
{
fn clone(&self) -> Self {
let inner = self.inner.clone();
Grpc { inner }
}
}
// ===== impl ResponseFuture =====
impl<T> Future for ResponseFuture<T>
where T: GrpcService,
T::Response: ::prost::Message,
{
type Item = ::http::Response<Encode<T::ResponseStream>>;
type Error = ::h2::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let response = try_ready!(self.inner.poll());
let (head, inner) = response.into_parts();
let body = Encode { inner };
let response = ::http::Response::from_parts(head, body);
Ok(response.into())
}
}
// ===== impl Encode =====
impl<T> ::tower_h2::Body for Encode<T>
where T: Stream<Error = ::Error>,
T::Item: ::prost::Message,
{
type Data = ::bytes::Bytes;
fn is_end_stream(&self) -> bool {
false
}
fn poll_data(&mut self) -> Poll<Option<Self::Data>, ::h2::Error> {
self.inner.poll_data()
}
fn poll_trailers(&mut self) -> Poll<Option<http::HeaderMap>, h2::Error> {
self.inner.poll_trailers()
}
}
// ===== impl Decode =====
impl<T> Stream for Decode<T>
where T: ::prost::Message + Default,
{
type Item = T;
type Error = ::Error;
fn poll(&mut self) -> Poll<Option<T>, ::Error> {
self.inner.poll()
}
}
// ===== impl Wrap =====
#[derive(Debug, Clone)]
struct Wrap<T>(T);
impl<T, U> Service for Wrap<T>
where T: GrpcService<Request = U, RequestStream = Decode<U>>,
T::Request: ::prost::Message + Default,
T::Response: ::prost::Message,
{
type Request = Request<::server::Decode<::protobuf::Decoder<T::Request>>>;
type Response = Response<T::ResponseStream>;
type Error = ::Error;
type Future = T::Future;
fn poll_ready(&mut self) -> Poll<(), ::Error> {
self.0.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let request = request.map(|inner| Decode { inner });
self.0.call(request)
}
}
}
// ===== impl Codec =====
impl<T, U> Codec<T, U>
where T: Message,
U: Message + Default,
{
/// Create a new protobuf codec
pub fn new() -> Self {
Codec(PhantomData)
}
}
impl<T, U> ::server::Codec for Codec<T, U>
where T: Message,
U: Message + Default,
{
/// Protocol buffer gRPC content type
const CONTENT_TYPE: &'static str = "application/grpc+proto";
type Encode = T;
type Encoder = Encoder<T>;
type Decode = U;
type Decoder = Decoder<U>;
fn encoder(&mut self) -> Self::Encoder {
Encoder(PhantomData)
}
fn decoder(&mut self) -> Self::Decoder {
Decoder(PhantomData)
}
}
impl<T, U> Clone for Codec<T, U> {
fn clone(&self) -> Self {
Codec(PhantomData)
}
}
// ===== impl Encoder =====
impl<T> ::server::Encoder for Encoder<T>
where T: Message,
{
type Item = T;
fn encode(&mut self, item: T, buf: &mut EncodeBuf) -> Result<(), ::Error> {
let len = item.encoded_len();
if buf.remaining_mut() < len {
buf.reserve(len);
}
item.encode(buf)
.map_err(|_| unreachable!("Message only errors if not enough space"))
}
}
impl<T> Clone for Encoder<T> {
fn clone(&self) -> Self {
Encoder(PhantomData)
}
}
// ===== impl Decoder =====
impl<T> ::server::Decoder for Decoder<T>
where T: Message + Default,
{
type Item = T;
fn decode(&mut self, buf: &mut DecodeBuf) -> Result<T, ::Error> {
Message::decode(buf)
.map_err(|_| unimplemented!())
}
}
impl<T> Clone for Decoder<T> {
fn clone(&self) -> Self {
Decoder(PhantomData)
}
}

View File

@ -1,67 +0,0 @@
use std::str::FromStr;
use http;
use ::client::codec::Unary;
#[derive(Debug)]
pub struct Request<T> {
http: http::Request<T>,
}
impl<T> Request<T> {
/// Create a new gRPC request
pub fn new(name: &str, message: T) -> Self {
let mut req = http::Request::new(message);
*req.version_mut() = http::Version::HTTP_2;
*req.method_mut() = http::Method::POST;
//TODO: specifically parse a `http::uri::PathAndQuery`
*req.uri_mut() = http::Uri::from_str(name)
.expect("user supplied illegal RPC name");
Request {
http: req,
}
}
/// Get a reference to the message
pub fn get_ref(&self) -> &T {
self.http.body()
}
/// Get a mutable reference to the message
pub fn get_mut(&mut self) -> &mut T {
self.http.body_mut()
}
/// Convert an HTTP request to a gRPC request
pub fn from_http(http: http::Request<T>) -> Self {
// TODO: validate
Request { http }
}
pub fn into_unary(self) -> Request<Unary<T>> {
let (head, body) = self.http.into_parts();
let http = http::Request::from_parts(head, Unary::new(body));
Request {
http,
}
}
pub fn into_http(self) -> http::Request<T> {
self.http
}
pub fn map<F, U>(self, f: F) -> Request<U>
where F: FnOnce(T) -> U,
{
let (head, body) = self.http.into_parts();
let body = f(body);
let http = http::Request::from_parts(head, body);
Request::from_http(http)
}
// pub fn metadata()
// pub fn metadata_bin()
}

View File

@ -1,39 +0,0 @@
use http;
#[derive(Debug)]
pub struct Response<T> {
http: http::Response<T>,
}
impl<T> Response<T> {
pub fn new(message: T) -> Self {
let mut res = http::Response::new(message);
*res.version_mut() = http::Version::HTTP_2;
Response {
http: res,
}
}
pub(crate) fn from_http(res: http::Response<T>) -> Self {
Response {
http: res,
}
}
pub fn into_http(self) -> http::Response<T> {
self.http
}
pub fn map<F, U>(self, f: F) -> Response<U>
where F: FnOnce(T) -> U,
{
let (head, body) = self.http.into_parts();
let body = f(body);
let http = http::Response::from_parts(head, body);
Response::from_http(http)
}
// pub fn metadata()
// pub fn metadata_bin()
}

View File

@ -1,59 +0,0 @@
use {Request, Response};
use super::unary::Once;
use protobuf::server::ClientStreamingService;
use futures::{Future, Poll};
use tower::Service;
/// Maps to a client streaming gRPC service.
#[derive(Debug, Clone)]
pub struct ClientStreaming<T> {
inner: T,
}
#[derive(Debug)]
pub struct ResponseFuture<T> {
inner: T,
}
// ===== impl ClientStreaming =====
impl<T> ClientStreaming<T>
where T: ClientStreamingService,
{
/// Return a new `ClientStreaming` gRPC service handler
pub fn new(inner: T) -> Self {
ClientStreaming { inner }
}
}
impl<T> Service for ClientStreaming<T>
where T: ClientStreamingService,
{
type Request = Request<T::RequestStream>;
type Response = Response<Once<T::Response>>;
type Error = ::Error;
type Future = ResponseFuture<T::Future>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.inner.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let inner = self.inner.call(request);
ResponseFuture { inner }
}
}
// ===== impl ResponseFuture ======
impl<T, U> Future for ResponseFuture<T>
where T: Future<Item = Response<U>, Error = ::Error> {
type Item = Response<Once<U>>;
type Error = ::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let response = try_ready!(self.inner.poll());
Ok(Once::map(response).into())
}
}

View File

@ -1,313 +0,0 @@
use Status;
// TODO: These types will most likely be moved back to the top level.
use client::codec::{DecodeBuf, EncodeBuf, BytesList};
use bytes::{Buf, BufMut, BytesMut, Bytes, BigEndian};
use futures::{Stream, Poll, Async};
use h2;
use http::HeaderMap;
use tower_h2::{self, Body};
use std::collections::VecDeque;
/// Encodes and decodes gRPC message types
pub trait Codec {
/// The content-type header for messages using this encoding.
///
/// Should be `application/grpc+yourencoding`.
const CONTENT_TYPE: &'static str;
/// The encode type
type Encode;
/// Encoder type
type Encoder: Encoder<Item = Self::Encode>;
/// The decode type
type Decode;
/// Decoder type
type Decoder: Decoder<Item = Self::Decode>;
/// Returns a new encoder
fn encoder(&mut self) -> Self::Encoder;
/// Returns a new decoder
fn decoder(&mut self) -> Self::Decoder;
}
/// Encodes gRPC message types
pub trait Encoder {
/// Type that is encoded
type Item;
/// Encode a message into the provided buffer.
fn encode(&mut self, item: Self::Item, buf: &mut EncodeBuf) -> Result<(), ::Error>;
}
/// Decodes gRPC message types
pub trait Decoder {
/// Type that is decoded
type Item;
/// Decode a message from the buffer.
///
/// The buffer will contain exactly the bytes of a full message. There
/// is no need to get the length from the bytes, gRPC framing is handled
/// for you.
fn decode(&mut self, buf: &mut DecodeBuf) -> Result<Self::Item, ::Error>;
}
/// Encodes gRPC message types
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct Encode<T, E> {
inner: EncodeInner<T, E>,
/// Destination buffer
buf: BytesMut,
}
#[derive(Debug)]
enum EncodeInner<T, E> {
Ok {
/// The source of messages to encode
inner: T,
/// The encoder
encoder: E,
},
Err(Status),
}
/// Decodes gRPC message types
#[must_use = "futures do nothing unless polled"]
#[derive(Debug)]
pub struct Decode<D> {
/// The source of encoded messages
inner: tower_h2::RecvBody,
/// The decoder
decoder: D,
/// buffer
bufs: BytesList,
/// Decoding state
state: State,
}
#[derive(Debug)]
enum State {
ReadHeader,
ReadBody {
compression: bool,
len: usize,
},
Done,
}
// ===== impl Encode =====
impl<T, E> Encode<T, E>
where T: Stream,
E: Encoder<Item = T::Item>,
{
pub(crate) fn new(inner: T, encoder: E) -> Self {
Encode {
inner: EncodeInner::Ok { inner, encoder },
buf: BytesMut::new(),
}
}
pub(crate) fn error(status: Status) -> Self {
Encode {
inner: EncodeInner::Err(status),
buf: BytesMut::new(),
}
}
}
impl<T, E> tower_h2::Body for Encode<T, E>
where T: Stream,
E: Encoder<Item = T::Item>,
{
type Data = Bytes;
fn is_end_stream(&self) -> bool {
false
}
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error> {
match self.inner {
EncodeInner::Ok { ref mut inner, ref mut encoder } => {
let item = try_ready!(inner.poll().map_err(|_| h2_err()));
if let Some(item) = item {
self.buf.reserve(5);
unsafe { self.buf.advance_mut(5); }
encoder.encode(item, &mut EncodeBuf {
bytes: &mut self.buf,
}).map_err(|_| h2_err())?;
// now that we know length, we can write the header
let len = self.buf.len() - 5;
assert!(len <= ::std::u32::MAX as usize);
{
let mut cursor = ::std::io::Cursor::new(&mut self.buf[..5]);
cursor.put_u8(0); // byte must be 0, reserve doesn't auto-zero
cursor.put_u32::<BigEndian>(len as u32);
}
Ok(Async::Ready(Some(self.buf.split_to(len + 5).freeze())))
} else {
Ok(Async::Ready(None))
}
}
_ => Ok(Async::Ready(None)),
}
}
fn poll_trailers(&mut self) -> Poll<Option<HeaderMap>, h2::Error> {
let mut map = HeaderMap::new();
let status = match self.inner {
EncodeInner::Ok { .. } => Status::OK.to_header_value(),
EncodeInner::Err(ref status) => status.to_header_value(),
};
// Success
map.insert("grpc-status", status);
Ok(Some(map).into())
}
}
// ===== impl Decode =====
impl<D> Decode<D>
where D: Decoder,
{
pub(crate) fn new(inner: tower_h2::RecvBody, decoder: D) -> Self {
Decode {
inner,
decoder,
bufs: BytesList {
bufs: VecDeque::new(),
},
state: State::ReadHeader,
}
}
fn decode(&mut self) -> Result<Option<D::Item>, Status> {
if let State::ReadHeader = self.state {
if self.bufs.remaining() < 5 {
return Ok(None);
}
let is_compressed = match self.bufs.get_u8() {
0 => false,
1 => {
trace!("message compressed, compression not supported yet");
return Err(Status::UNIMPLEMENTED);
},
_ => {
trace!("unexpected compression flag");
return Err(Status::UNKNOWN);
}
};
let len = self.bufs.get_u32::<BigEndian>() as usize;
self.state = State::ReadBody {
compression: is_compressed,
len,
}
}
if let State::ReadBody { len, .. } = self.state {
if self.bufs.remaining() < len {
return Ok(None);
}
match self.decoder.decode(&mut DecodeBuf {
bufs: &mut self.bufs,
len,
}) {
Ok(msg) => {
self.state = State::ReadHeader;
return Ok(Some(msg));
},
Err(_) => {
debug!("decoder error");
return Err(Status::UNKNOWN);
}
}
}
Ok(None)
}
}
impl<D> Stream for Decode<D>
where D: Decoder,
{
type Item = D::Item;
type Error = ::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
loop {
if let State::Done = self.state {
break;
}
match self.decode() {
Ok(Some(val)) => return Ok(Async::Ready(Some(val))),
Ok(None) => (),
Err(status) => return Err(::Error::Grpc(status)),
}
let chunk = try_ready!(self.inner.poll_data());
if let Some(data) = chunk {
self.bufs.bufs.push_back(data);
} else {
if self.bufs.has_remaining() {
trace!("unexpected EOF decoding stream");
return Err(::Error::Grpc(Status::UNKNOWN))
} else {
self.state = State::Done;
break;
}
}
}
if let Some(trailers) = try_ready!(self.inner.poll_trailers()) {
grpc_status(&trailers).map_err(::Error::Grpc)?;
Ok(Async::Ready(None))
} else {
trace!("receive body ended without trailers");
Err(::Error::Grpc(Status::UNKNOWN))
}
}
}
// ===== impl utils =====
fn h2_err() -> h2::Error {
unimplemented!("EncodingBody map_err")
}
fn grpc_status(trailers: &HeaderMap) -> Result<(), Status> {
if let Some(status) = trailers.get("grpc-status") {
let status = Status::from_bytes(status.as_ref());
if status.code() == ::Code::OK {
Ok(())
} else {
Err(status)
}
} else {
trace!("trailers missing grpc-status");
Err(Status::UNKNOWN)
}
}

View File

@ -1,57 +0,0 @@
mod codec;
pub mod client_streaming;
pub mod server_streaming;
pub mod streaming;
pub mod unary;
pub use self::codec::{Codec, Encoder, Decoder, Decode, Encode};
pub use self::streaming::Grpc;
pub use self::client_streaming::ClientStreaming;
pub use self::server_streaming::ServerStreaming;
pub use self::unary::Unary;
use {Request, Response};
use futures::{Poll};
use futures::future::{self, FutureResult};
use tower::Service;
/// A gRPC service that responds to all requests with not implemented
#[derive(Debug)]
pub struct NotImplemented<T, U> {
_p: ::std::marker::PhantomData<(T, U)>,
}
// ===== impl NotImplemented =====
impl<T, U> NotImplemented<T, U> {
pub fn new() -> Self {
NotImplemented {
_p: ::std::marker::PhantomData,
}
}
}
impl<T, U> Service for NotImplemented<T, U>
{
type Request = Request<T>;
type Response = Response<U>;
type Error = ::Error;
type Future = FutureResult<Self::Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(().into())
}
fn call(&mut self, _: Self::Request) -> Self::Future {
future::err(::Error::Grpc(::Status::UNIMPLEMENTED))
}
}
impl<T, U> Clone for NotImplemented<T, U> {
fn clone(&self) -> Self {
NotImplemented {
_p: ::std::marker::PhantomData,
}
}
}

View File

@ -1,144 +0,0 @@
use {Request, Response};
use protobuf::server::ServerStreamingService;
use futures::{Future, Stream, Poll};
use tower::Service;
/// Maps to a server streaming gRPC service.
#[derive(Debug)]
pub struct ServerStreaming<T, S> {
/// Inner service blueprint. This is used as the source to clone from.
inner: T,
/// The clone that will be used to handle the next request.
clone: T,
/// Making the rustc compiler happy since 2014.
_p: ::std::marker::PhantomData<S>,
}
pub struct ResponseFuture<T, S>
where T: ServerStreamingService,
S: Stream,
{
inner: T,
state: Option<State<T::Future, S>>,
}
enum State<T, S> {
/// Waiting for the request to be received
Requesting(Request<S>),
/// Waiting for the response future to resolve
Responding(T),
}
// ===== impl ServerStreaming =====
impl<T, S, U> ServerStreaming<T, S>
where T: ServerStreamingService<Request = S::Item, Response = U>,
S: Stream<Error = ::Error>,
{
/// Return a new `ServerStreaming` gRPC service handler
pub fn new(inner: T) -> Self {
let clone = inner.clone();
ServerStreaming {
inner,
clone,
_p: ::std::marker::PhantomData,
}
}
}
impl<T, S> Service for ServerStreaming<T, S>
where T: ServerStreamingService<Request = S::Item>,
S: Stream<Error = ::Error>,
{
type Request = Request<S>;
type Response = Response<T::ResponseStream>;
type Error = ::Error;
type Future = ResponseFuture<T, S>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
// Ensure that the clone is ready to process the request.
self.clone.poll_ready()
.map_err(|_| unimplemented!())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
use std::mem;
// Create a new clone to replace the old one.
let inner = mem::replace(&mut self.clone, self.inner.clone());
ResponseFuture {
inner,
state: Some(State::Requesting(request)),
}
}
}
impl<T, S> Clone for ServerStreaming<T, S>
where T: Clone,
{
fn clone(&self) -> Self {
let inner = self.inner.clone();
let clone = inner.clone();
ServerStreaming {
inner,
clone,
_p: ::std::marker::PhantomData,
}
}
}
// ===== impl ResponseFuture ======
impl<T, S> Future for ResponseFuture<T, S>
where T: ServerStreamingService<Request = S::Item>,
S: Stream<Error = ::Error>,
{
type Item = Response<T::ResponseStream>;
type Error = ::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
use self::State::*;
loop {
let msg = match *self.state.as_mut().unwrap() {
Requesting(ref mut request) => {
try_ready!(request.get_mut().poll())
}
Responding(ref mut fut) => {
return fut.poll();
}
};
match msg {
Some(msg) => {
match self.state.take().unwrap() {
Requesting(request) => {
// A bunch of junk to map the body type
let http = request.into_http();
let (head, _) = http.into_parts();
let http = ::http::Request::from_parts(head, msg);
let request = Request::from_http(http);
let response = self.inner.call(request);
self.state = Some(Responding(response));
}
_ => unreachable!(),
}
}
None => {
// TODO: Do something
return Err(::Error::Inner(()));
}
}
}
}
}

View File

@ -1,122 +0,0 @@
use {Request, Response};
use super::codec::{Codec, Encoder, Decode, Encode};
use {http, h2};
use futures::{Future, Stream, Poll, Async};
use tower::Service;
use tower_h2::RecvBody;
/// A bidirectional streaming gRPC service.
#[derive(Debug, Clone)]
pub struct Grpc<T, C> {
inner: T,
codec: C,
}
#[derive(Debug)]
pub struct ResponseFuture<T, E> {
inner: T,
encoder: Option<E>,
}
// ===== impl Grpc =====
impl<T, C, S> Grpc<T, C>
where T: Service<Request = Request<Decode<C::Decoder>>,
Response = Response<S>,
Error = ::Error>,
C: Codec,
S: Stream<Item = C::Encode>,
{
pub fn new(inner: T, codec: C) -> Self {
Grpc {
inner,
codec,
}
}
}
impl<T, C, S> Service for Grpc<T, C>
where T: Service<Request = Request<Decode<C::Decoder>>,
Response = Response<S>,
Error = ::Error>,
C: Codec,
S: Stream<Item = C::Encode>,
{
type Request = http::Request<RecvBody>;
type Response = http::Response<Encode<S, C::Encoder>>;
type Error = h2::Error;
type Future = ResponseFuture<T::Future, C::Encoder>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.inner.poll_ready().map_err(|_| unimplemented!())
}
fn call(&mut self, request: Self::Request) -> Self::Future {
// Map the request body
let (head, body) = request.into_parts();
// Wrap the body stream with a decoder
let body = Decode::new(body, self.codec.decoder());
// Reconstruct the HTTP request
let request = http::Request::from_parts(head, body);
// Convert the HTTP request to a gRPC request
let request = Request::from_http(request);
// Send the request to the inner service
let inner = self.inner.call(request);
// Return the response
ResponseFuture {
inner,
encoder: Some(self.codec.encoder()),
}
}
}
// ===== impl ResponseFuture =====
impl<T, E, S> Future for ResponseFuture<T, E>
where T: Future<Item = Response<S>,
Error = ::Error>,
E: Encoder,
S: Stream<Item = E::Item>,
{
type Item = http::Response<Encode<S, E>>;
type Error = h2::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
// Get the gRPC response
let response = match self.inner.poll() {
Ok(Async::Ready(response)) => response,
Ok(Async::NotReady) => return Ok(Async::NotReady),
Err(e) => {
match e {
::Error::Grpc(status) => {
let response = Response::new(Encode::error(status));
return Ok(response.into_http().into());
}
// TODO: Is this correct?
_ => return Err(h2::Reason::INTERNAL_ERROR.into()),
}
}
};
// Convert to an HTTP response
let response = response.into_http();
// Map the response body
let (head, body) = response.into_parts();
// Get the encoder
let encoder = self.encoder.take().expect("encoder consumed");
// Encode the body
let body = Encode::new(body, encoder);
// Success
Ok(http::Response::from_parts(head, body).into())
}
}

View File

@ -1,144 +0,0 @@
use super::server_streaming::{self, ServerStreaming};
use {Request, Response};
use protobuf::server::UnaryService;
use futures::{Future, Stream, Poll};
use tower::Service;
/// Maps to a unary gRPC service.
#[derive(Debug)]
pub struct Unary<T, S> {
inner: ServerStreaming<Inner<T>, S>,
}
pub struct ResponseFuture<T, S>
where T: UnaryService,
S: Stream,
{
inner: server_streaming::ResponseFuture<Inner<T>, S>,
}
#[derive(Debug)]
pub struct Once<T> {
inner: Option<T>,
}
/// Maps inbound requests
#[derive(Debug, Clone)]
struct Inner<T>(pub T);
struct InnerFuture<T>(T);
// ===== impl Unary =====
impl<T, S, U> Unary<T, S>
where T: UnaryService<Request = S::Item, Response = U>,
S: Stream<Error = ::Error>,
{
/// Return a new `Unary` gRPC service handler
pub fn new(inner: T) -> Self {
let inner = ServerStreaming::new(Inner(inner));
Unary { inner }
}
}
impl<T, S, U> Service for Unary<T, S>
where T: UnaryService<Request = S::Item, Response = U>,
S: Stream<Error = ::Error>,
{
type Request = Request<S>;
type Response = ::Response<Once<U>>;
type Error = ::Error;
type Future = ResponseFuture<T, S>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Service::poll_ready(&mut self.inner)
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let inner = Service::call(&mut self.inner, request);
ResponseFuture { inner }
}
}
impl<T, S> Clone for Unary<T, S>
where T: Clone,
{
fn clone(&self) -> Self {
Unary { inner: self.inner.clone() }
}
}
// ===== impl Inner =====
impl<T> Service for Inner<T>
where T: UnaryService,
{
type Request = Request<T::Request>;
type Response = Response<Once<T::Response>>;
type Error = ::Error;
type Future = InnerFuture<T::Future>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.0.poll_ready()
}
fn call(&mut self, request: Self::Request) -> Self::Future {
let inner = self.0.call(request);
InnerFuture(inner)
}
}
// ===== impl ResponseFuture ======
impl<T, S, U> Future for ResponseFuture<T, S>
where T: UnaryService<Request = S::Item, Response = U>,
S: Stream<Error = ::Error>,
{
type Item = Response<Once<U>>;
type Error = ::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
self.inner.poll()
}
}
// ===== impl InnerFuture ======
impl<T, U> Future for InnerFuture<T>
where T: Future<Item = Response<U>, Error = ::Error> {
type Item = Response<Once<U>>;
type Error = ::Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
let response = try_ready!(self.0.poll());
Ok(Once::map(response).into())
}
}
// ===== impl Once =====
impl<T> Once<T> {
/// Map a response to a response of a `Once` stream
pub(super) fn map(response: Response<T>) -> Response<Self> {
// A bunch of junk to map the body type
let http = response.into_http();
let (head, body) = http.into_parts();
// Wrap with `Once`
let body = Once { inner: Some(body) };
let http = ::http::Response::from_parts(head, body);
Response::from_http(http)
}
}
impl<T> Stream for Once<T> {
type Item = T;
type Error = ::Error;
fn poll(&mut self) -> Poll<Option<Self::Item>, Self::Error> {
Ok(self.inner.take().into())
}
}

View File

@ -1,204 +0,0 @@
use std::fmt;
use h2;
use http::header::HeaderValue;
#[derive(Debug, Clone)]
pub struct Status {
code: Code,
}
#[derive(Clone, Copy, PartialEq, Eq)]
pub struct Code(Code_);
impl Status {
#[inline]
pub fn code(&self) -> Code {
self.code
}
pub const OK: Status = Status {
code: Code(Code_::Ok),
};
pub const CANCELED: Status = Status {
code: Code(Code_::Canceled),
};
pub const UNKNOWN: Status = Status {
code: Code(Code_::Unknown),
};
pub const INVALID_ARGUMENT: Status = Status {
code: Code(Code_::InvalidArgument),
};
pub const DEADLINE_EXCEEDED: Status = Status {
code: Code(Code_::DeadlineExceeded),
};
pub const NOT_FOUND: Status = Status {
code: Code(Code_::NotFound),
};
pub const ALREADY_EXISTS: Status = Status {
code: Code(Code_::AlreadyExists),
};
pub const PERMISSION_DENIED: Status = Status {
code: Code(Code_::PermissionDenied),
};
pub const RESOURCE_EXHAUSTED: Status = Status {
code: Code(Code_::ResourceExhausted),
};
pub const FAILED_PRECONDITION: Status = Status {
code: Code(Code_::FailedPrecondition),
};
pub const ABORTED: Status = Status {
code: Code(Code_::Aborted),
};
pub const OUT_OF_RANGE: Status = Status {
code: Code(Code_::OutOfRange),
};
pub const UNIMPLEMENTED: Status = Status {
code: Code(Code_::Unimplemented),
};
pub const INTERNAL: Status = Status {
code: Code(Code_::Internal),
};
pub const UNAVAILABLE: Status = Status {
code: Code(Code_::Unavailable),
};
pub const DATA_LOSS: Status = Status {
code: Code(Code_::DataLoss),
};
pub const UNAUTHENTICATED: Status = Status {
code: Code(Code_::Unauthenticated),
};
pub(crate) fn from_bytes(bytes: &[u8]) -> Status {
let code = match bytes.len() {
1 => {
match bytes[0] {
b'0' => Code_::Ok,
b'1' => Code_::Canceled,
b'2' => Code_::Unknown,
b'3' => Code_::InvalidArgument,
b'4' => Code_::DeadlineExceeded,
b'5' => Code_::NotFound,
b'6' => Code_::AlreadyExists,
b'7' => Code_::PermissionDenied,
b'8' => Code_::ResourceExhausted,
b'9' => Code_::FailedPrecondition,
_ => return Status::parse_err(),
}
},
2 => {
match (bytes[0], bytes[1]) {
(b'1', b'0') => Code_::Aborted,
(b'1', b'1') => Code_::OutOfRange,
(b'1', b'2') => Code_::Unimplemented,
(b'1', b'3') => Code_::Internal,
(b'1', b'4') => Code_::Unavailable,
(b'1', b'5') => Code_::DataLoss,
(b'1', b'6') => Code_::Unauthenticated,
_ => return Status::parse_err(),
}
},
_ => return Status::parse_err(),
};
Status::new(Code(code))
}
// TODO: It would be nice for this not to be public
pub fn to_header_value(&self) -> HeaderValue {
use self::Code_::*;
match self.code.0 {
Ok => HeaderValue::from_static("0"),
Canceled => HeaderValue::from_static("1"),
Unknown => HeaderValue::from_static("2"),
InvalidArgument => HeaderValue::from_static("3"),
DeadlineExceeded => HeaderValue::from_static("4"),
NotFound => HeaderValue::from_static("5"),
AlreadyExists => HeaderValue::from_static("6"),
PermissionDenied => HeaderValue::from_static("7"),
ResourceExhausted => HeaderValue::from_static("8"),
FailedPrecondition => HeaderValue::from_static("9"),
Aborted => HeaderValue::from_static("10"),
OutOfRange => HeaderValue::from_static("11"),
Unimplemented => HeaderValue::from_static("12"),
Internal => HeaderValue::from_static("13"),
Unavailable => HeaderValue::from_static("14"),
DataLoss => HeaderValue::from_static("15"),
Unauthenticated => HeaderValue::from_static("16"),
}
}
fn new(code: Code) -> Status {
Status {
code,
}
}
fn parse_err() -> Status {
trace!("error parsing grpc-status");
Status::UNKNOWN
}
}
impl From<h2::Error> for Status {
fn from(_err: h2::Error) -> Self {
//TODO: https://grpc.io/docs/guides/wire.html#errors
Status::new(Code(Code_::Internal))
}
}
impl From<Status> for h2::Error {
fn from(_status: Status) -> Self {
// TODO: implement
h2::Reason::INTERNAL_ERROR.into()
}
}
impl Code {
pub const OK: Code = Code(Code_::Ok);
//TODO: the rest...
}
impl fmt::Debug for Code {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&self.0, f)
}
}
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
enum Code_ {
Ok = 0,
Canceled = 1,
Unknown = 2,
InvalidArgument = 3,
DeadlineExceeded = 4,
NotFound = 5,
AlreadyExists = 6,
PermissionDenied = 7,
ResourceExhausted = 8,
FailedPrecondition = 9,
Aborted = 10,
OutOfRange = 11,
Unimplemented = 12,
Internal = 13,
Unavailable = 14,
DataLoss = 15,
Unauthenticated = 16,
}

View File

@ -1,21 +0,0 @@
[package]
name = "tower-h2"
version = "0.2.0"
authors = ["Oliver Gould <ver@buoyant.io>"]
description = "Exploring tower + h2"
publish = false
[dependencies]
bytes = "0.4"
futures = "0.1"
h2 = { git = "https://github.com/carllerche/h2" }
http = "0.1"
log = "0.3"
tokio-core = "0.1"
tokio-connect = { git = "https://github.com/carllerche/tokio-connect" }
tokio-io = "0.1"
tower = { git = "https://github.com/tower-rs/tower" }
[dev-dependencies]
env_logger = "^0.4.3"
string = { git = "https://github.com/carllerche/string" }

View File

@ -1,143 +0,0 @@
extern crate env_logger;
extern crate futures;
extern crate bytes;
extern crate h2;
extern crate http;
extern crate string;
extern crate tokio_connect;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
use futures::*;
use bytes::Bytes;
use http::{Request, Response};
use std::net::SocketAddr;
use string::{String, TryFrom};
use tokio_connect::Connect;
use tokio_core::net::TcpStream;
use tokio_core::reactor::{Core, Handle};
use tower::{NewService, Service};
use tower_h2::{Body, Client, RecvBody};
use h2::Reason;
pub struct Conn(SocketAddr, Handle);
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let addr = "[::1]:8888".parse().unwrap();
impl Connect for Conn {
type Connected = TcpStream;
type Error = ::std::io::Error;
type Future = Box<Future<Item = TcpStream, Error = ::std::io::Error>>;
fn connect(&self) -> Self::Future {
let c = TcpStream::connect(&self.0, &self.1)
.and_then(|tcp| tcp.set_nodelay(true).map(move |_| tcp));
Box::new(c)
}
}
let conn = Conn(addr, reactor.clone());
let h2 = Client::<Conn, Handle, ()>::new(conn, Default::default(), reactor);
let done = h2.new_service()
.map_err(|_| Reason::REFUSED_STREAM.into())
.and_then(move |h2| {
Serial {
h2,
count: 500,
pending: None,
}
})
.map(|_| println!("done"))
.map_err(|e| println!("error: {:?}", e));
core.run(done).unwrap();
}
/// Avoids overflowing max concurrent streams
struct Serial {
count: usize,
h2: tower_h2::client::Service<Conn, Handle, ()>,
pending: Option<Box<Future<Item = (), Error = tower_h2::client::Error>>>,
}
impl Future for Serial {
type Item = ();
type Error = tower_h2::client::Error;
fn poll(&mut self) -> Poll<(), Self::Error> {
loop {
if let Some(mut fut) = self.pending.take() {
if fut.poll()?.is_not_ready() {
self.pending = Some(fut);
return Ok(Async::NotReady);
}
}
if self.count == 0 {
return Ok(Async::Ready(()));
}
let pfx = format!("{}", self.count);
self.count -= 1;
let mut fut = self.h2
.call(mkreq())
.and_then(move |rsp| read_response(&pfx, rsp).map_err(Into::into));
if fut.poll()?.is_not_ready() {
self.pending = Some(Box::new(fut));
return Ok(Async::NotReady);
}
}
}
}
fn mkreq() -> Request<()> {
Request::builder()
.method("GET")
.uri("http://[::1]:8888/")
.version(http::Version::HTTP_2)
.body(())
.unwrap()
}
fn read_response(pfx: &str, rsp: Response<RecvBody>) -> ReadResponse {
let (parts, body) = rsp.into_parts();
println!("{}: {}", pfx, parts.status);
let pfx = pfx.to_owned();
ReadResponse {
pfx,
body,
}
}
struct ReadResponse {
pfx: ::std::string::String,
body: RecvBody,
}
impl Future for ReadResponse {
type Item = ();
type Error = tower_h2::client::Error;
fn poll(&mut self) -> Poll<(), Self::Error> {
loop {
match try_ready!(self.body.poll_data()) {
None => return Ok(Async::Ready(())),
Some(b) => {
let b: Bytes = b.into();
{
let s = String::try_from(b).expect("decode utf8 string");
println!("{}: {}", self.pfx, &*s);
}
}
}
}
}
}

View File

@ -1,121 +0,0 @@
extern crate bytes;
extern crate env_logger;
extern crate futures;
extern crate h2;
extern crate http;
#[macro_use]
extern crate log;
extern crate tokio_core;
extern crate tower;
extern crate tower_h2;
use bytes::Bytes;
use futures::*;
use http::Request;
use tokio_core::net::TcpListener;
use tokio_core::reactor::Core;
use tower::{NewService, Service};
use tower_h2::{Body, Server, RecvBody};
type Response = http::Response<RspBody>;
struct RspBody(Option<Bytes>);
impl RspBody {
fn new(body: Bytes) -> Self {
RspBody(Some(body))
}
fn empty() -> Self {
RspBody(None)
}
}
impl Body for RspBody {
type Data = Bytes;
fn is_end_stream(&self) -> bool {
self.0.as_ref().map(|b| b.is_empty()).unwrap_or(false)
}
fn poll_data(&mut self) -> Poll<Option<Bytes>, h2::Error> {
let data = self.0
.take()
.and_then(|b| if b.is_empty() { None } else { Some(b) });
Ok(Async::Ready(data))
}
}
//const ROOT: &'static str = "/";
const ROOT: &'static str = "/helloworld.Greeter/SayHello";
#[derive(Debug)]
struct Svc;
impl Service for Svc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type Future = future::FutureResult<Response, Self::Error>;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
Ok(Async::Ready(()))
}
fn call(&mut self, req: Self::Request) -> Self::Future {
let mut rsp = http::Response::builder();
rsp.version(http::Version::HTTP_2);
let uri = req.uri();
if uri.path() != ROOT {
let body = RspBody::empty();
let rsp = rsp.status(404).body(body).unwrap();
return future::ok(rsp);
}
let body = RspBody::new("heyo!".into());
let rsp = rsp.status(200).body(body).unwrap();
future::ok(rsp)
}
}
#[derive(Debug)]
struct NewSvc;
impl NewService for NewSvc {
type Request = Request<RecvBody>;
type Response = Response;
type Error = h2::Error;
type InitError = ::std::io::Error;
type Service = Svc;
type Future = future::FutureResult<Svc, Self::InitError>;
fn new_service(&self) -> Self::Future {
future::ok(Svc)
}
}
fn main() {
drop(env_logger::init());
let mut core = Core::new().unwrap();
let reactor = core.handle();
let h2 = Server::new(NewSvc, Default::default(), reactor.clone());
let addr = "[::1]:8888".parse().unwrap();
let bind = TcpListener::bind(&addr, &reactor).expect("bind");
let serve = bind.incoming()
.fold((h2, reactor), |(h2, reactor), (sock, _)| {
if let Err(e) = sock.set_nodelay(true) {
return Err(e);
}
let serve = h2.serve(sock);
reactor.spawn(serve.map_err(|e| error!("h2 error: {:?}", e)));
Ok((h2, reactor))
});
core.run(serve).unwrap();
}

View File

@ -1,48 +0,0 @@
use h2;
use bytes::IntoBuf;
use futures::{Async, Poll};
use http::HeaderMap;
/// A generic h2 client/server request/response body.
pub trait Body {
/// The body chunk type
type Data: IntoBuf + 'static;
/// Returns `true` when the end of stream has been reached.
///
/// An end of stream means that both `poll_data` and `poll_trailers` will
/// return `None`.
///
/// A return value of `false` **does not** guarantee that a value will be
/// returend from `poll_stream` or `poll_trailers`.
fn is_end_stream(&self) -> bool {
false
}
/// Polls a stream of data.
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error>;
/// Returns possibly **one** `HeaderMap` for trailers.
fn poll_trailers(&mut self) -> Poll<Option<HeaderMap>, h2::Error> {
Ok(Async::Ready(None))
}
}
impl Body for () {
type Data = &'static [u8];
#[inline]
fn is_end_stream(&self) -> bool {
true
}
#[inline]
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error> {
Ok(Async::Ready(None))
}
#[inline]
fn poll_trailers(&mut self) -> Poll<Option<HeaderMap>, h2::Error> {
Ok(Async::Ready(None))
}
}

View File

@ -1,63 +0,0 @@
use Body;
use flush::Flush;
use futures::{Future, Poll};
use h2::client::Connection;
use tokio_connect::Connect;
/// Task that performs background tasks for a client.
///
/// This is not used directly by a user of this library.
pub struct Background<C, S>
where C: Connect,
S: Body,
{
task: Task<C, S>,
}
/// The specific task to execute
enum Task<C, S>
where C: Connect,
S: Body,
{
Connection(Connection<C::Connected, S::Data>),
Flush(Flush<S>),
}
// ===== impl Background =====
impl<C, S> Background<C, S>
where C: Connect,
S: Body,
{
pub(crate) fn connection(
connection: Connection<C::Connected, S::Data>)
-> Self
{
let task = Task::Connection(connection);
Background { task }
}
pub(crate) fn flush(flush: Flush<S>) -> Self {
let task = Task::Flush(flush);
Background { task }
}
}
impl<C, S> Future for Background<C, S>
where C: Connect,
S: Body,
{
type Item = ();
type Error = ();
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
use self::Task::*;
match self.task {
// TODO: Log error?
Connection(ref mut f) => f.poll().map_err(|_| ()),
Flush(ref mut f) => f.poll(),
}
}
}

View File

@ -1,7 +0,0 @@
mod background;
mod new_service;
mod service;
pub use self::background::Background;
pub use self::new_service::{Client, ConnectFuture, ConnectError};
pub use self::service::{Service, ResponseFuture, Error};

View File

@ -1,140 +0,0 @@
use {Body, RecvBody};
use super::{Service, Background};
use futures::{Future, Async, Poll};
use futures::future::Executor;
use h2;
use h2::client::Connection;
use http::{Request, Response};
use tokio_connect::Connect;
use std::boxed::Box;
use std::marker::PhantomData;
/// Establishes a Client on an H2 connection.
///
/// Has a builder-like API for configuring client connections. Currently this only allows
/// the configuration of TLS transport on new services created by this factory.
pub struct Client<C, E, S> {
/// Establish new session layer values (usually TCP sockets w/ TLS).
connect: C,
/// H2 client configuration
builder: h2::client::Builder,
/// Used to spawn connection management tasks and tasks to flush send
/// body streams.
executor: E,
/// The HTTP request body type.
_p: PhantomData<S>,
}
/// Completes with a Service when the H2 connection has been initialized.
pub struct ConnectFuture<C, E, S>
where C: Connect + 'static,
S: Body + 'static,
{
future: Box<Future<Item = Connected<S::Data, C::Connected>, Error = ConnectError<C::Error>>>,
executor: E,
_p: PhantomData<S>,
}
/// The type yielded by an h2 client handshake future
type Connected<S, C> = (h2::client::Client<S>, Connection<C, S>);
/// Error produced when establishing an H2 client connection.
#[derive(Debug)]
pub enum ConnectError<T> {
/// An error occurred when attempting to establish the underlying session
/// layer.
Connect(T),
/// An error occurred when attempting to perform the HTTP/2.0 handshake.
Proto(h2::Error),
/// An error occured when attempting to execute a worker task
Execute,
}
// ===== impl Client =====
impl<C, E, S> Client<C, E, S>
where
C: Connect,
E: Executor<Background<C, S>> + Clone,
S: Body,
{
/// Create a new `Client`.
///
/// The `connect` argument is used to obtain new session layer instances
/// (`AsyncRead` + `AsyncWrite`). For each new client service returned, a
/// task will be spawned onto `executor` that will be used to manage the H2
/// connection.
pub fn new(connect: C, builder: h2::client::Builder, executor: E) -> Self {
Client {
connect,
executor,
builder,
_p: PhantomData,
}
}
}
impl<C, E, S> ::tower::NewService for Client<C, E, S>
where
C: Connect + 'static,
E: Executor<Background<C, S>> + Clone,
S: Body + 'static,
{
type Request = Request<S>;
type Response = Response<RecvBody>;
type Error = super::Error;
type InitError = ConnectError<C::Error>;
type Service = Service<C, E, S>;
type Future = ConnectFuture<C, E, S>;
/// Obtains a Service on a single plaintext h2 connection to a remote.
fn new_service(&self) -> Self::Future {
let client = self.builder.clone();
let conn = self.connect.connect()
.map_err(ConnectError::Connect)
.and_then(move |io| {
client
.handshake(io)
.map_err(ConnectError::Proto)
});
ConnectFuture {
future: Box::new(conn),
executor: self.executor.clone(),
_p: PhantomData,
}
}
}
// ===== impl ConnectFuture =====
impl<C, E, S> Future for ConnectFuture<C, E, S>
where
C: Connect,
E: Executor<Background<C, S>> + Clone,
S: Body,
{
type Item = Service<C, E, S>;
type Error = ConnectError<C::Error>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
// Get the session layer instance
let (client, connection) = try_ready!(self.future.poll());
// Spawn the worker task
let task = Background::connection(connection);
self.executor.execute(task).map_err(|_| ConnectError::Execute)?;
// Create an instance of the service
let service = Service::new(client, self.executor.clone());
Ok(Async::Ready(service))
}
}

View File

@ -1,204 +0,0 @@
use {Body, RecvBody};
use super::Background;
use flush::Flush;
use bytes::IntoBuf;
use futures::{Future, Poll};
use futures::future::Executor;
use h2;
use h2::client::{self, Client};
use http::{self, Request, Response};
use tokio_connect::Connect;
use std::marker::PhantomData;
/// Exposes a request/response API on an h2 client connection..
pub struct Service<C, E, S>
where S: Body,
{
client: Client<S::Data>,
executor: E,
_p: PhantomData<(C, S)>,
}
/// Drives the sending of a request (and its body) until a response is received (i.e. the
/// initial HEADERS or RESET frames sent from the remote).
///
/// This is necessary because, for instance, the remote server may not respond until the
/// request body is fully sent.
pub struct ResponseFuture {
inner: Inner,
}
/// ResponseFuture inner
enum Inner {
/// Inner response future
Inner(client::ResponseFuture),
/// Failed to send the request
Error(Option<Error>),
}
/// Errors produced by client `Service` calls.
#[derive(Debug)]
pub struct Error {
kind: Kind,
}
#[derive(Debug)]
enum Kind {
Inner(h2::Error),
Spawn,
}
// ===== impl Service =====
impl<C, E, S> Service<C, E, S>
where S: Body,
S::Data: IntoBuf + 'static,
C: Connect,
E: Executor<Background<C, S>>,
{
/// Builds Service on an H2 client connection.
pub(super) fn new(client: Client<S::Data>, executor: E) -> Self {
let _p = PhantomData;
Service {
client,
executor,
_p,
}
}
}
impl<C, E, S> Service<C, E, S>
where S: Body,
S::Data: IntoBuf + 'static,
C: Connect,
E: Executor<Background<C, S>> + Clone,
{
pub fn clone_handle<S2>(&self) -> Service<C, E, S2>
where S2: Body<Data=S::Data>,
{
Service {
client: self.client.clone(),
executor: self.executor.clone(),
_p: PhantomData,
}
}
}
impl<C, E, S> Clone for Service<C, E, S>
where S: Body,
E: Clone,
{
fn clone(&self) -> Self {
Service {
client: self.client.clone(),
executor: self.executor.clone(),
_p: PhantomData,
}
}
}
impl<C, E, S> ::tower::Service for Service<C, E, S>
where S: Body + 'static,
S::Data: IntoBuf + 'static,
C: Connect,
E: Executor<Background<C, S>>,
{
type Request = Request<S>;
type Response = Response<RecvBody>;
type Error = Error;
type Future = ResponseFuture;
fn poll_ready(&mut self) -> Poll<(), Self::Error> {
self.client.poll_ready()
.map_err(Into::into)
}
fn call(&mut self, request: Self::Request) -> Self::Future {
trace!("request: {} {}", request.method(), request.uri());
// Split the request from the body
let (parts, body) = request.into_parts();
let request = http::Request::from_parts(parts, ());
// If there is no body, then there is no point spawning a task to flush
// it.
let end_of_stream = body.is_end_stream();
// Initiate the H2 request
let res = self.client.send_request(request, end_of_stream);
let (response, send_body) = match res {
Ok(success) => success,
Err(e) => {
let e = Error { kind: Kind::Inner(e) };
let inner = Inner::Error(Some(e));
return ResponseFuture { inner };
}
};
if !end_of_stream {
let flush = Flush::new(body, send_body);
let res = self.executor.execute(Background::flush(flush));
if let Err(_) = res {
let e = Error { kind: Kind::Spawn };
let inner = Inner::Error(Some(e));
return ResponseFuture { inner };
}
}
ResponseFuture { inner: Inner::Inner(response) }
}
}
// ===== impl ResponseFuture =====
impl Future for ResponseFuture {
type Item = Response<RecvBody>;
type Error = Error;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
use self::Inner::*;
match self.inner {
Inner(ref mut fut) => {
let response = try_ready!(fut.poll());
let (parts, body) = response.into_parts();
let body = RecvBody::new(body);
Ok(Response::from_parts(parts, body).into())
}
Error(ref mut e) => {
return Err(e.take().unwrap());
}
}
}
}
// ===== impl Error =====
impl Error {
pub fn reason(&self) -> Option<h2::Reason> {
match self.kind {
Kind::Inner(ref h2) => h2.reason(),
_ => None,
}
}
}
impl From<h2::Error> for Error {
fn from(src: h2::Error) -> Self {
Error { kind: Kind::Inner(src) }
}
}
impl From<h2::Reason> for Error {
fn from(src: h2::Reason) -> Self {
h2::Error::from(src).into()
}
}

View File

@ -1,103 +0,0 @@
use Body;
use futures::{Future, Poll, Async};
use h2::{self, SendStream};
use http::HeaderMap;
/// Flush a body to the HTTP/2.0 send stream
pub(crate) struct Flush<S>
where S: Body,
{
h2: SendStream<S::Data>,
body: S,
state: FlushState,
}
enum FlushState {
Data,
Trailers,
Done,
}
enum DataOrTrailers<B> {
Data(B),
Trailers(HeaderMap),
}
// ===== impl Flush =====
impl<S> Flush<S>
where S: Body,
{
pub fn new(src: S, dst: SendStream<S::Data>) -> Self {
Flush {
h2: dst,
body: src,
state: FlushState::Data,
}
}
/// Try to flush the body.
fn poll_complete(&mut self) -> Poll<(), h2::Error> {
let mut first = try_ready!(self.poll_body());
loop {
if let Some(DataOrTrailers::Data(buf)) = first {
let second = self.poll_body()?;
let eos = if let Async::Ready(None) = second {
true
} else {
false
};
self.h2.send_data(buf, eos)?;
if eos {
return Ok(Async::Ready(()));
} else if let Async::Ready(item) = second {
first = item;
} else {
return Ok(Async::NotReady);
}
} else if let Some(DataOrTrailers::Trailers(trailers)) = first {
self.h2.send_trailers(trailers)?;
return Ok(Async::Ready(()));
} else {
return Ok(Async::Ready(()));
}
}
}
/// Get the next message to write, either a data frame or trailers.
fn poll_body(&mut self) -> Poll<Option<DataOrTrailers<S::Data>>, h2::Error> {
loop {
match self.state {
FlushState::Data => {
if let Some(data) = try_ready!(self.body.poll_data()) {
return Ok(Async::Ready(Some(DataOrTrailers::Data(data))));
} else {
self.state = FlushState::Trailers;
}
}
FlushState::Trailers => {
let trailers = try_ready!(self.body.poll_trailers());
self.state = FlushState::Done;
if let Some(trailers) = trailers {
return Ok(Async::Ready(Some(DataOrTrailers::Trailers(trailers))));
}
}
FlushState::Done => return Ok(Async::Ready(None)),
}
}
}
}
impl<S> Future for Flush<S>
where S: Body,
{
type Item = ();
type Error = ();
fn poll(&mut self) -> Poll<(), ()> {
// TODO: Do something with the error
self.poll_complete().map_err(|_| ())
}
}

View File

@ -1,23 +0,0 @@
extern crate bytes;
#[macro_use]
extern crate futures;
extern crate h2;
extern crate http;
#[macro_use]
extern crate log;
extern crate tokio_connect;
extern crate tokio_core;
extern crate tokio_io;
extern crate tower;
pub mod client;
pub mod server;
mod body;
mod flush;
mod recv_body;
pub use body::Body;
pub use client::Client;
pub use recv_body::{RecvBody, Data};
pub use server::Server;

View File

@ -1,118 +0,0 @@
use Body;
use bytes::{Bytes, BytesMut, Buf};
use futures::{Poll, Stream};
use h2;
use http;
/// Allows a stream to be read from the remote.
#[derive(Debug, Default)]
pub struct RecvBody {
inner: Option<h2::RecvStream>,
}
#[derive(Debug)]
pub struct Data {
release_capacity: h2::ReleaseCapacity,
bytes: Bytes,
}
// ===== impl RecvBody =====
impl RecvBody {
/// Return a new `RecvBody`.
pub(crate) fn new(inner: h2::RecvStream) -> Self {
RecvBody { inner: Some(inner) }
}
}
impl Body for RecvBody {
type Data = Data;
#[inline]
fn is_end_stream(&self) -> bool {
match self.inner {
Some(ref inner) => inner.is_end_stream(),
None => true,
}
}
fn poll_data(&mut self) -> Poll<Option<Self::Data>, h2::Error> {
match self.inner {
Some(ref mut inner) => {
let data = try_ready!(inner.poll())
.map(|bytes| {
Data {
release_capacity: inner.release_capacity().clone(),
bytes,
}
});
Ok(data.into())
}
None => Ok(None.into()),
}
}
fn poll_trailers(&mut self) -> Poll<Option<http::HeaderMap>, h2::Error> {
match self.inner {
Some(ref mut inner) => inner.poll_trailers(),
None => Ok(None.into()),
}
}
}
// ===== impl Data =====
impl Buf for Data {
fn remaining(&self) -> usize {
self.bytes.len()
}
fn bytes(&self) -> &[u8] {
self.bytes.as_ref()
}
fn advance(&mut self, cnt: usize) {
if cnt > self.remaining() {
panic!("advanced past end of buffer");
}
trace!("releasing capacity: {} of {}", cnt, self.remaining());
let _ = self.bytes.split_to(cnt);
self.release_capacity.release_capacity(cnt)
.expect("flow control error")
}
}
impl Drop for Data {
fn drop(&mut self) {
let sz = self.remaining();
trace!("Data::drop: releasing capacity: {}", sz);
self.release_capacity
.release_capacity(sz)
.expect("flow control error");
}
}
impl From<Data> for Bytes {
fn from(mut src: Data) -> Self {
let bytes = ::std::mem::replace(&mut src.bytes, Bytes::new());
src.release_capacity.release_capacity(bytes.len())
.expect("flow control error");
bytes
}
}
impl From<Data> for BytesMut {
fn from(mut src: Data) -> Self {
let bytes = ::std::mem::replace(&mut src.bytes, Bytes::new());
src.release_capacity.release_capacity(bytes.len())
.expect("flow control error");
bytes.into()
}
}

View File

@ -1,357 +0,0 @@
use {flush, Body, RecvBody};
use futures::{Future, Poll, Stream};
use futures::future::{Executor, Either, Join, MapErr};
use h2::{self, Reason};
use h2::server::{Server as Accept, Handshake, Respond};
use http::{self, Request, Response};
use tokio_io::{AsyncRead, AsyncWrite};
use tower::{NewService, Service};
use std::marker::PhantomData;
/// Attaches service implementations to h2 connections.
pub struct Server<S, E, B>
where S: NewService,
B: Body,
{
new_service: S,
builder: h2::server::Builder,
executor: E,
_p: PhantomData<B>,
}
/// Drives connection-level I/O .
pub struct Connection<T, S, E, B, F>
where T: AsyncRead + AsyncWrite,
S: NewService,
B: Body,
{
state: State<T, S, B>,
executor: E,
modify: F,
}
/// Modify a received request
pub trait Modify {
/// Modify a request before calling the service.
fn modify(&mut self, request: &mut Request<()>);
}
enum State<T, S, B>
where T: AsyncRead + AsyncWrite,
S: NewService,
B: Body,
{
/// Establish the HTTP/2.0 connection and get a service to process inbound
/// requests.
Init(Init<T, B::Data, S::Future, S::InitError>),
/// Both the HTTP/2.0 connection and the service are ready.
Ready {
connection: Accept<T, B::Data>,
service: S::Service,
},
}
type Init<T, B, S, E> =
Join<
MapErr<Handshake<T, B>, MapErrA<E>>,
MapErr<S, MapErrB<E>>>;
type MapErrA<E> = fn(h2::Error) -> Either<h2::Error, E>;
type MapErrB<E> = fn(E) -> Either<h2::Error, E>;
/// Task used to process requests
pub struct Background<T, B>
where B: Body,
{
state: BackgroundState<T, B>,
}
enum BackgroundState<T, B>
where B: Body,
{
Respond {
respond: Respond<B::Data>,
response: T,
},
Flush(flush::Flush<B>),
}
/// Error produced by a `Connection`.
#[derive(Debug)]
pub enum Error<S>
where S: NewService,
{
/// Error produced during the HTTP/2.0 handshake.
Handshake(h2::Error),
/// Error produced by the HTTP/2.0 stream
Protocol(h2::Error),
/// Error produced when obtaining the service
NewService(S::InitError),
/// Error produced by the service
Service(S::Error),
/// Error produced when attempting to spawn a task
Execute,
}
// ===== impl Server =====
impl<S, E, B> Server<S, E, B>
where S: NewService<Request = Request<RecvBody>, Response = Response<B>>,
B: Body,
{
pub fn new(new_service: S, builder: h2::server::Builder, executor: E) -> Self {
Server {
new_service,
executor,
builder,
_p: PhantomData,
}
}
}
impl<S, E, B> Server<S, E, B>
where S: NewService<Request = http::Request<RecvBody>, Response = Response<B>>,
B: Body,
E: Clone,
{
/// Produces a future that is satisfied once the h2 connection has been initialized.
pub fn serve<T>(&self, io: T) -> Connection<T, S, E, B, ()>
where T: AsyncRead + AsyncWrite,
{
self.serve_modified(io, ())
}
pub fn serve_modified<T, F>(&self, io: T, modify: F) -> Connection<T, S, E, B, F>
where T: AsyncRead + AsyncWrite,
F: Modify,
{
// Clone a handle to the executor so that it can be moved into the
// connection handle
let executor = self.executor.clone();
let service = self.new_service.new_service()
.map_err(Either::B as MapErrB<S::InitError>);
// TODO we should specify initial settings here!
let handshake = self.builder.handshake(io)
.map_err(Either::A as MapErrA<S::InitError>);
Connection {
state: State::Init(handshake.join(service)),
executor,
modify,
}
}
}
// B doesn't need to be Clone, it's just a marker type.
impl<S, E, B> Clone for Server<S, E, B>
where
S: NewService + Clone,
E: Clone,
B: Body,
{
fn clone(&self) -> Self {
Server {
new_service: self.new_service.clone(),
executor: self.executor.clone(),
builder: self.builder.clone(),
_p: PhantomData,
}
}
}
// ===== impl Connection =====
impl<T, S, E, B, F> Connection<T, S, E, B, F>
where T: AsyncRead + AsyncWrite,
S: NewService<Request = http::Request<RecvBody>, Response = Response<B>>,
B: Body,
{
fn is_ready(&self) -> bool {
use self::State::*;
match self.state {
Ready { .. } => true,
_ => false,
}
}
fn try_ready(&mut self) -> Poll<(), Error<S>> {
use self::State::*;
let (connection, service) = match self.state {
Init(ref mut join) => try_ready!(join.poll().map_err(Error::from_init)),
_ => unreachable!(),
};
self.state = Ready { connection, service };
Ok(().into())
}
}
impl<T, S, E, B, F> Future for Connection<T, S, E, B, F>
where T: AsyncRead + AsyncWrite,
S: NewService<Request = http::Request<RecvBody>, Response = Response<B>>,
E: Executor<Background<<S::Service as Service>::Future, B>>,
B: Body + 'static,
F: Modify,
{
type Item = ();
type Error = Error<S>;
fn poll(&mut self) -> Poll<Self::Item, Self::Error> {
if !self.is_ready() {
// Advance the initialization of the service and HTTP/2.0 connection
try_ready!(self.try_ready());
}
let (connection, service) = match self.state {
State::Ready { ref mut connection, ref mut service } => {
(connection, service)
}
_ => unreachable!(),
};
loop {
// Make sure the service is ready
let ready = service.poll_ready()
// TODO: Don't dump the error
.map_err(Error::Service);
try_ready!(ready);
let next = connection.poll()
.map_err(Error::Protocol);
let (request, respond) = match try_ready!(next) {
Some(next) => next,
None => return Ok(().into()),
};
let (parts, body) = request.into_parts();
// This is really unfortunate, but the `http` currently lacks the
// APIs to do this better :(
let mut request = Request::from_parts(parts, ());
self.modify.modify(&mut request);
let (parts, _) = request.into_parts();
let request = Request::from_parts(parts, RecvBody::new(body));
// Dispatch the request to the service
let response = service.call(request);
// Spawn a new task to process the response future
if let Err(_) = self.executor.execute(Background::new(respond, response)) {
return Err(Error::Execute)
}
}
}
}
// ===== impl Modify =====
impl<T> Modify for T
where T: FnMut(&mut Request<()>)
{
fn modify(&mut self, request: &mut Request<()>) {
(*self)(request);
}
}
impl Modify for () {
fn modify(&mut self, _: &mut Request<()>) {
}
}
// ===== impl Background =====
impl<T, B> Background<T, B>
where T: Future,
B: Body,
{
fn new(respond: Respond<B::Data>, response: T) -> Self {
Background {
state: BackgroundState::Respond {
respond,
response,
},
}
}
}
impl<T, B> Future for Background<T, B>
where T: Future<Item = Response<B>>,
B: Body,
{
type Item = ();
type Error = ();
fn poll(&mut self) -> Poll<(), ()> {
use self::BackgroundState::*;
loop {
let flush = match self.state {
Respond { ref mut respond, ref mut response } => {
use flush::Flush;
let response = try_ready!(response.poll().map_err(|_| {
// TODO: do something better the error?
let reason = Reason::INTERNAL_ERROR;
respond.send_reset(reason);
}));
let (parts, body) = response.into_parts();
// Check if the response is immediately an end-of-stream.
let end_stream = body.is_end_stream();
trace!("send_response eos={} {:?}", end_stream, parts);
// Try sending the response.
let response = Response::from_parts(parts, ());
match respond.send_response(response, end_stream) {
Ok(stream) => {
if end_stream {
// Nothing more to do
return Ok(().into());
}
// Transition to flushing the body
Flush::new(body, stream)
}
Err(_) => {
// TODO: Do something with the error?
return Ok(().into());
}
}
}
Flush(ref mut flush) => return flush.poll(),
};
self.state = Flush(flush);
}
}
}
// ===== impl Error =====
impl<S> Error<S>
where S: NewService,
{
fn from_init(err: Either<h2::Error, S::InitError>) -> Self {
match err {
Either::A(err) => Error::Handshake(err),
Either::B(err) => Error::NewService(err),
}
}
}