mirror of
https://github.com/ninenines/cowboy.git
synced 2025-07-14 12:20:24 +00:00
Add experimental and incomplete SPDY support
The SPDY connection processes are also supervisors. Missing: * sendfile support * request body reading support
This commit is contained in:
parent
c7f0834dc3
commit
9a2d35c2e8
7 changed files with 980 additions and 35 deletions
2
Makefile
2
Makefile
|
@ -5,7 +5,7 @@ PROJECT = cowboy
|
|||
# Options.
|
||||
|
||||
COMPILE_FIRST = cowboy_middleware cowboy_sub_protocol
|
||||
CT_SUITES = eunit http ws
|
||||
CT_SUITES = eunit http spdy ws
|
||||
PLT_APPS = crypto public_key ssl
|
||||
|
||||
# Dependencies.
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
|
||||
-export([start_http/4]).
|
||||
-export([start_https/4]).
|
||||
-export([start_spdy/4]).
|
||||
-export([stop_listener/1]).
|
||||
-export([set_env/3]).
|
||||
|
||||
|
@ -52,6 +53,18 @@ start_https(Ref, NbAcceptors, TransOpts, ProtoOpts)
|
|||
ranch:start_listener(Ref, NbAcceptors,
|
||||
ranch_ssl, TransOpts, cowboy_protocol, ProtoOpts).
|
||||
|
||||
%% @doc Start a SPDY listener.
|
||||
-spec start_spdy(any(), non_neg_integer(), any(), any()) -> {ok, pid()}.
|
||||
start_spdy(Ref, NbAcceptors, TransOpts, ProtoOpts)
|
||||
when is_integer(NbAcceptors), NbAcceptors > 0 ->
|
||||
TransOpts2 = [
|
||||
{connection_type, supervisor},
|
||||
{next_protocols_advertised,
|
||||
[<<"spdy/3">>, <<"http/1.1">>, <<"http/1.0">>]}
|
||||
|TransOpts],
|
||||
ranch:start_listener(Ref, NbAcceptors,
|
||||
ranch_ssl, TransOpts2, cowboy_spdy, ProtoOpts).
|
||||
|
||||
%% @doc Stop a listener.
|
||||
-spec stop_listener(ranch:ref()) -> ok.
|
||||
stop_listener(Ref) ->
|
||||
|
|
|
@ -54,6 +54,7 @@
|
|||
%% Internal.
|
||||
-export([init/4]).
|
||||
-export([parse_request/3]).
|
||||
-export([parse_host/2]).
|
||||
-export([resume/6]).
|
||||
|
||||
-type opts() :: [{compress, boolean()}
|
||||
|
|
|
@ -130,13 +130,13 @@
|
|||
| {done, binary(), non_neg_integer(), binary()}
|
||||
| {error, atom()}).
|
||||
|
||||
-type resp_body_fun() :: fun((inet:socket(), module()) -> ok).
|
||||
-type resp_body_fun() :: fun((any(), module()) -> ok).
|
||||
-type send_chunk_fun() :: fun((iodata()) -> ok | {error, atom()}).
|
||||
-type resp_chunked_fun() :: fun((send_chunk_fun()) -> ok).
|
||||
|
||||
-record(http_req, {
|
||||
%% Transport.
|
||||
socket = undefined :: undefined | inet:socket(),
|
||||
socket = undefined :: any(),
|
||||
transport = undefined :: undefined | module(),
|
||||
connection = keepalive :: keepalive | close,
|
||||
|
||||
|
@ -189,7 +189,7 @@
|
|||
%%
|
||||
%% Since we always need to parse the Connection header, we do it
|
||||
%% in an optimized way and add the parsed value to p_headers' cache.
|
||||
-spec new(inet:socket(), module(),
|
||||
-spec new(any(), module(),
|
||||
undefined | {inet:ip_address(), inet:port_number()},
|
||||
binary(), binary(), binary(),
|
||||
cowboy:http_version(), cowboy:http_headers(), binary(),
|
||||
|
@ -917,7 +917,7 @@ has_resp_body(#http_req{resp_body={Length, _}}) ->
|
|||
has_resp_body(#http_req{resp_body=RespBody}) ->
|
||||
iolist_size(RespBody) > 0.
|
||||
|
||||
%% Remove a header previously set for the response.
|
||||
%% @doc Remove a header previously set for the response.
|
||||
-spec delete_resp_header(binary(), Req)
|
||||
-> Req when Req::req().
|
||||
delete_resp_header(Name, Req=#http_req{resp_headers=RespHeaders}) ->
|
||||
|
@ -944,20 +944,30 @@ reply(Status, Headers, Body, Req=#http_req{
|
|||
version=Version, connection=Connection,
|
||||
method=Method, resp_compress=Compress,
|
||||
resp_state=waiting, resp_headers=RespHeaders}) ->
|
||||
HTTP11Headers = case Version of
|
||||
'HTTP/1.1' -> [{<<"connection">>, atom_to_connection(Connection)}];
|
||||
_ -> []
|
||||
HTTP11Headers = if
|
||||
Transport =/= cowboy_spdy, Version =:= 'HTTP/1.1' ->
|
||||
[{<<"connection">>, atom_to_connection(Connection)}];
|
||||
true ->
|
||||
[]
|
||||
end,
|
||||
Req3 = case Body of
|
||||
BodyFun when is_function(BodyFun) ->
|
||||
%% We stream the response body until we close the connection.
|
||||
RespConn = close,
|
||||
{RespType, Req2} = response(Status, Headers, RespHeaders, [
|
||||
{RespType, Req2} = if
|
||||
Transport =:= cowboy_spdy ->
|
||||
response(Status, Headers, RespHeaders, [
|
||||
{<<"date">>, cowboy_clock:rfc1123()},
|
||||
{<<"server">>, <<"Cowboy">>}
|
||||
], stream, Req);
|
||||
true ->
|
||||
response(Status, Headers, RespHeaders, [
|
||||
{<<"connection">>, <<"close">>},
|
||||
{<<"date">>, cowboy_clock:rfc1123()},
|
||||
{<<"server">>, <<"Cowboy">>},
|
||||
{<<"transfer-encoding">>, <<"identity">>}
|
||||
], <<>>, Req),
|
||||
], <<>>, Req)
|
||||
end,
|
||||
if RespType =/= hook, Method =/= <<"HEAD">> ->
|
||||
BodyFun(Socket, Transport);
|
||||
true -> ok
|
||||
|
@ -970,13 +980,12 @@ reply(Status, Headers, Body, Req=#http_req{
|
|||
ChunkFun = fun(IoData) -> chunk(IoData, Req2) end,
|
||||
BodyFun(ChunkFun),
|
||||
%% Terminate the chunked body for HTTP/1.1 only.
|
||||
_ = case Version of
|
||||
'HTTP/1.0' -> ok;
|
||||
_ -> Transport:send(Socket, <<"0\r\n\r\n">>)
|
||||
case Version of
|
||||
'HTTP/1.0' -> Req2;
|
||||
_ -> last_chunk(Req2)
|
||||
end;
|
||||
true -> Req2
|
||||
end;
|
||||
true -> ok
|
||||
end,
|
||||
Req2;
|
||||
{ContentLength, BodyFun} ->
|
||||
%% We stream the response body for ContentLength bytes.
|
||||
RespConn = response_connection(Headers, Connection),
|
||||
|
@ -984,7 +993,7 @@ reply(Status, Headers, Body, Req=#http_req{
|
|||
{<<"content-length">>, integer_to_list(ContentLength)},
|
||||
{<<"date">>, cowboy_clock:rfc1123()},
|
||||
{<<"server">>, <<"Cowboy">>}
|
||||
|HTTP11Headers], <<>>, Req),
|
||||
|HTTP11Headers], stream, Req),
|
||||
if RespType =/= hook, Method =/= <<"HEAD">> ->
|
||||
BodyFun(Socket, Transport);
|
||||
true -> ok
|
||||
|
@ -1001,7 +1010,7 @@ reply(Status, Headers, Body, Req=#http_req{
|
|||
RespHeaders, HTTP11Headers, Method, iolist_size(Body)),
|
||||
Req2#http_req{connection=RespConn}
|
||||
end,
|
||||
{ok, Req3#http_req{resp_state=done,resp_headers=[], resp_body= <<>>}}.
|
||||
{ok, Req3#http_req{resp_state=done, resp_headers=[], resp_body= <<>>}}.
|
||||
|
||||
reply_may_compress(Status, Headers, Body, Req,
|
||||
RespHeaders, HTTP11Headers, Method) ->
|
||||
|
@ -1065,18 +1074,34 @@ chunked_reply(Status, Headers, Req) ->
|
|||
-spec chunk(iodata(), req()) -> ok | {error, atom()}.
|
||||
chunk(_Data, #http_req{method= <<"HEAD">>}) ->
|
||||
ok;
|
||||
chunk(Data, #http_req{socket=Socket, transport=Transport, version='HTTP/1.0'}) ->
|
||||
chunk(Data, #http_req{socket=Socket, transport=cowboy_spdy,
|
||||
resp_state=chunks}) ->
|
||||
cowboy_spdy:stream_data(Socket, Data);
|
||||
chunk(Data, #http_req{socket=Socket, transport=Transport,
|
||||
resp_state=chunks, version='HTTP/1.0'}) ->
|
||||
Transport:send(Socket, Data);
|
||||
chunk(Data, #http_req{socket=Socket, transport=Transport, resp_state=chunks}) ->
|
||||
chunk(Data, #http_req{socket=Socket, transport=Transport,
|
||||
resp_state=chunks}) ->
|
||||
Transport:send(Socket, [integer_to_list(iolist_size(Data), 16),
|
||||
<<"\r\n">>, Data, <<"\r\n">>]).
|
||||
|
||||
%% @doc Finish the chunked reply.
|
||||
%% @todo If ever made public, need to send nothing if HEAD.
|
||||
-spec last_chunk(Req) -> Req when Req::req().
|
||||
last_chunk(Req=#http_req{socket=Socket, transport=cowboy_spdy}) ->
|
||||
_ = cowboy_spdy:stream_close(Socket),
|
||||
Req#http_req{resp_state=done};
|
||||
last_chunk(Req=#http_req{socket=Socket, transport=Transport}) ->
|
||||
_ = Transport:send(Socket, <<"0\r\n\r\n">>),
|
||||
Req#http_req{resp_state=done}.
|
||||
|
||||
%% @doc Send an upgrade reply.
|
||||
%% @private
|
||||
-spec upgrade_reply(cowboy:http_status(), cowboy:http_headers(), Req)
|
||||
-> {ok, Req} when Req::req().
|
||||
upgrade_reply(Status, Headers, Req=#http_req{
|
||||
resp_state=waiting, resp_headers=RespHeaders}) ->
|
||||
upgrade_reply(Status, Headers, Req=#http_req{transport=Transport,
|
||||
resp_state=waiting, resp_headers=RespHeaders})
|
||||
when Transport =/= cowboy_spdy ->
|
||||
{_, Req2} = response(Status, Headers, RespHeaders, [
|
||||
{<<"connection">>, <<"Upgrade">>}
|
||||
], <<>>, Req),
|
||||
|
@ -1098,9 +1123,8 @@ ensure_response(#http_req{method= <<"HEAD">>, resp_state=chunks}, _) ->
|
|||
ok;
|
||||
ensure_response(#http_req{version='HTTP/1.0', resp_state=chunks}, _) ->
|
||||
ok;
|
||||
ensure_response(#http_req{socket=Socket, transport=Transport,
|
||||
resp_state=chunks}, _) ->
|
||||
Transport:send(Socket, <<"0\r\n\r\n">>),
|
||||
ensure_response(Req=#http_req{resp_state=chunks}, _) ->
|
||||
_ = last_chunk(Req),
|
||||
ok.
|
||||
|
||||
%% Private setter/getter API.
|
||||
|
@ -1212,6 +1236,15 @@ to_list(Req) ->
|
|||
|
||||
-spec chunked_response(cowboy:http_status(), cowboy:http_headers(), Req) ->
|
||||
{normal | hook, Req} when Req::req().
|
||||
chunked_response(Status, Headers, Req=#http_req{
|
||||
transport=cowboy_spdy, resp_state=waiting,
|
||||
resp_headers=RespHeaders}) ->
|
||||
{RespType, Req2} = response(Status, Headers, RespHeaders, [
|
||||
{<<"date">>, cowboy_clock:rfc1123()},
|
||||
{<<"server">>, <<"Cowboy">>}
|
||||
], stream, Req),
|
||||
{RespType, Req2#http_req{resp_state=chunks,
|
||||
resp_headers=[], resp_body= <<>>}};
|
||||
chunked_response(Status, Headers, Req=#http_req{
|
||||
version=Version, connection=Connection,
|
||||
resp_state=waiting, resp_headers=RespHeaders}) ->
|
||||
|
@ -1230,7 +1263,7 @@ chunked_response(Status, Headers, Req=#http_req{
|
|||
resp_headers=[], resp_body= <<>>}}.
|
||||
|
||||
-spec response(cowboy:http_status(), cowboy:http_headers(),
|
||||
cowboy:http_headers(), cowboy:http_headers(), iodata(), Req)
|
||||
cowboy:http_headers(), cowboy:http_headers(), stream | iodata(), Req)
|
||||
-> {normal | hook, Req} when Req::req().
|
||||
response(Status, Headers, RespHeaders, DefaultHeaders, Body, Req=#http_req{
|
||||
socket=Socket, transport=Transport, version=Version,
|
||||
|
@ -1239,22 +1272,32 @@ response(Status, Headers, RespHeaders, DefaultHeaders, Body, Req=#http_req{
|
|||
already_called -> Headers;
|
||||
_ -> response_merge_headers(Headers, RespHeaders, DefaultHeaders)
|
||||
end,
|
||||
Body2 = case Body of stream -> <<>>; _ -> Body end,
|
||||
Req2 = case OnResponse of
|
||||
already_called -> Req;
|
||||
undefined -> Req;
|
||||
OnResponse -> OnResponse(Status, FullHeaders, Body,
|
||||
OnResponse ->
|
||||
OnResponse(Status, FullHeaders, Body2,
|
||||
%% Don't call 'onresponse' from the hook itself.
|
||||
Req#http_req{resp_headers=[], resp_body= <<>>,
|
||||
onresponse=already_called})
|
||||
end,
|
||||
ReplyType = case Req2#http_req.resp_state of
|
||||
waiting when Transport =:= cowboy_spdy, Body =:= stream ->
|
||||
cowboy_spdy:stream_reply(Socket, status(Status), FullHeaders),
|
||||
ReqPid ! {?MODULE, resp_sent},
|
||||
normal;
|
||||
waiting when Transport =:= cowboy_spdy ->
|
||||
cowboy_spdy:reply(Socket, status(Status), FullHeaders, Body),
|
||||
ReqPid ! {?MODULE, resp_sent},
|
||||
normal;
|
||||
waiting ->
|
||||
HTTPVer = atom_to_binary(Version, latin1),
|
||||
StatusLine = << HTTPVer/binary, " ",
|
||||
(status(Status))/binary, "\r\n" >>,
|
||||
HeaderLines = [[Key, <<": ">>, Value, <<"\r\n">>]
|
||||
|| {Key, Value} <- FullHeaders],
|
||||
Transport:send(Socket, [StatusLine, HeaderLines, <<"\r\n">>, Body]),
|
||||
Transport:send(Socket, [StatusLine, HeaderLines, <<"\r\n">>, Body2]),
|
||||
ReqPid ! {?MODULE, resp_sent},
|
||||
normal;
|
||||
_ ->
|
||||
|
|
544
src/cowboy_spdy.erl
Normal file
544
src/cowboy_spdy.erl
Normal file
|
@ -0,0 +1,544 @@
|
|||
%% Copyright (c) 2013, Loïc Hoguin <essen@ninenines.eu>
|
||||
%%
|
||||
%% Permission to use, copy, modify, and/or distribute this software for any
|
||||
%% purpose with or without fee is hereby granted, provided that the above
|
||||
%% copyright notice and this permission notice appear in all copies.
|
||||
%%
|
||||
%% THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
%% WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
%% MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
%% ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
%% WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
%% ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
%% OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
|
||||
%% @doc SPDY protocol handler.
|
||||
%%
|
||||
%% The available options are:
|
||||
%% <dl>
|
||||
%% </dl>
|
||||
%%
|
||||
%% Note that there is no need to monitor these processes when using Cowboy as
|
||||
%% an application as it already supervises them under the listener supervisor.
|
||||
-module(cowboy_spdy).
|
||||
|
||||
%% API.
|
||||
-export([start_link/4]).
|
||||
|
||||
%% Internal.
|
||||
-export([init/5]).
|
||||
-export([system_continue/3]).
|
||||
-export([system_terminate/4]).
|
||||
-export([system_code_change/4]).
|
||||
|
||||
%% Internal request process.
|
||||
-export([request_init/9]).
|
||||
-export([resume/5]).
|
||||
-export([reply/4]).
|
||||
-export([stream_reply/3]).
|
||||
-export([stream_data/2]).
|
||||
-export([stream_close/1]).
|
||||
|
||||
%% Internal transport functions.
|
||||
-export([name/0]).
|
||||
-export([send/2]).
|
||||
|
||||
-record(child, {
|
||||
streamid :: non_neg_integer(),
|
||||
pid :: pid(),
|
||||
input = nofin :: fin | nofin,
|
||||
output = nofin :: fin | nofin
|
||||
}).
|
||||
|
||||
-record(state, {
|
||||
parent = undefined :: pid(),
|
||||
socket,
|
||||
transport,
|
||||
buffer = <<>> :: binary(),
|
||||
middlewares,
|
||||
env,
|
||||
onrequest,
|
||||
onresponse,
|
||||
peer,
|
||||
zdef,
|
||||
zinf,
|
||||
last_streamid = 0 :: non_neg_integer(),
|
||||
children = [] :: [#child{}]
|
||||
}).
|
||||
|
||||
-record(special_headers, {
|
||||
method,
|
||||
path,
|
||||
version,
|
||||
host,
|
||||
scheme %% @todo We don't use it.
|
||||
}).
|
||||
|
||||
-type opts() :: [].
|
||||
-export_type([opts/0]).
|
||||
|
||||
-include("cowboy_spdy.hrl").
|
||||
|
||||
%% API.
|
||||
|
||||
%% @doc Start a SPDY protocol process.
|
||||
-spec start_link(any(), inet:socket(), module(), any()) -> {ok, pid()}.
|
||||
start_link(Ref, Socket, Transport, Opts) ->
|
||||
proc_lib:start_link(?MODULE, init,
|
||||
[self(), Ref, Socket, Transport, Opts]).
|
||||
|
||||
%% Internal.
|
||||
|
||||
%% @doc Faster alternative to proplists:get_value/3.
|
||||
%% @private
|
||||
get_value(Key, Opts, Default) ->
|
||||
case lists:keyfind(Key, 1, Opts) of
|
||||
{_, Value} -> Value;
|
||||
_ -> Default
|
||||
end.
|
||||
|
||||
%% @private
|
||||
-spec init(pid(), ranch:ref(), inet:socket(), module(), opts()) -> ok.
|
||||
init(Parent, Ref, Socket, Transport, Opts) ->
|
||||
process_flag(trap_exit, true),
|
||||
ok = proc_lib:init_ack(Parent, {ok, self()}),
|
||||
{ok, Peer} = Transport:peername(Socket),
|
||||
Middlewares = get_value(middlewares, Opts, [cowboy_router, cowboy_handler]),
|
||||
Env = [{listener, Ref}|get_value(env, Opts, [])],
|
||||
OnRequest = get_value(onrequest, Opts, undefined),
|
||||
OnResponse = get_value(onresponse, Opts, undefined),
|
||||
Zdef = zlib:open(),
|
||||
ok = zlib:deflateInit(Zdef),
|
||||
_ = zlib:deflateSetDictionary(Zdef, ?ZDICT),
|
||||
Zinf = zlib:open(),
|
||||
ok = zlib:inflateInit(Zinf),
|
||||
ok = ranch:accept_ack(Ref),
|
||||
loop(#state{parent=Parent, socket=Socket, transport=Transport,
|
||||
middlewares=Middlewares, env=Env, onrequest=OnRequest,
|
||||
onresponse=OnResponse, peer=Peer, zdef=Zdef, zinf=Zinf}).
|
||||
|
||||
loop(State=#state{parent=Parent, socket=Socket, transport=Transport,
|
||||
buffer=Buffer, children=Children}) ->
|
||||
{OK, Closed, Error} = Transport:messages(),
|
||||
Transport:setopts(Socket, [{active, once}]),
|
||||
receive
|
||||
{OK, Socket, Data} ->
|
||||
Data2 = << Buffer/binary, Data/binary >>,
|
||||
case Data2 of
|
||||
<< _:40, Length:24, _/bits >>
|
||||
when byte_size(Data2) >= Length + 8 ->
|
||||
Length2 = Length + 8,
|
||||
<< Frame:Length2/binary, Rest/bits >> = Data2,
|
||||
control_frame(State#state{buffer=Rest}, Frame);
|
||||
Rest ->
|
||||
loop(State#state{buffer=Rest})
|
||||
end;
|
||||
{Closed, Socket} ->
|
||||
terminate(State);
|
||||
{Error, Socket, _Reason} ->
|
||||
terminate(State);
|
||||
{reply, {Pid, StreamID}, Status, Headers}
|
||||
when Pid =:= self() ->
|
||||
Child = #child{output=nofin} = lists:keyfind(StreamID,
|
||||
#child.streamid, Children),
|
||||
syn_reply(State, fin, StreamID, Status, Headers),
|
||||
Children2 = lists:keyreplace(StreamID,
|
||||
#child.streamid, Children, Child#child{output=fin}),
|
||||
loop(State#state{children=Children2});
|
||||
{reply, {Pid, StreamID}, Status, Headers, Body}
|
||||
when Pid =:= self() ->
|
||||
Child = #child{output=nofin} = lists:keyfind(StreamID,
|
||||
#child.streamid, Children),
|
||||
syn_reply(State, nofin, StreamID, Status, Headers),
|
||||
data(State, fin, StreamID, Body),
|
||||
Children2 = lists:keyreplace(StreamID,
|
||||
#child.streamid, Children, Child#child{output=fin}),
|
||||
loop(State#state{children=Children2});
|
||||
{stream_reply, {Pid, StreamID}, Status, Headers}
|
||||
when Pid =:= self() ->
|
||||
#child{output=nofin} = lists:keyfind(StreamID,
|
||||
#child.streamid, Children),
|
||||
syn_reply(State, nofin, StreamID, Status, Headers),
|
||||
loop(State);
|
||||
{stream_data, {Pid, StreamID}, Data}
|
||||
when Pid =:= self() ->
|
||||
#child{output=nofin} = lists:keyfind(StreamID,
|
||||
#child.streamid, Children),
|
||||
data(State, nofin, StreamID, Data),
|
||||
loop(State);
|
||||
{stream_close, {Pid, StreamID}}
|
||||
when Pid =:= self() ->
|
||||
Child = #child{output=nofin} = lists:keyfind(StreamID,
|
||||
#child.streamid, Children),
|
||||
data(State, fin, StreamID),
|
||||
Children2 = lists:keyreplace(StreamID,
|
||||
#child.streamid, Children, Child#child{output=fin}),
|
||||
loop(State#state{children=Children2});
|
||||
{'EXIT', Parent, Reason} ->
|
||||
exit(Reason);
|
||||
{'EXIT', Pid, _} ->
|
||||
Children2 = lists:keydelete(Pid, #child.pid, Children),
|
||||
loop(State#state{children=Children2});
|
||||
{system, From, Request} ->
|
||||
sys:handle_system_msg(Request, From, Parent, ?MODULE, [], State);
|
||||
%% Calls from the supervisor module.
|
||||
{'$gen_call', {To, Tag}, which_children} ->
|
||||
Children = [{?MODULE, Pid, worker, [?MODULE]}
|
||||
|| #child{pid=Pid} <- Children],
|
||||
To ! {Tag, Children},
|
||||
loop(State);
|
||||
{'$gen_call', {To, Tag}, count_children} ->
|
||||
NbChildren = length(Children),
|
||||
Counts = [{specs, 1}, {active, NbChildren},
|
||||
{supervisors, 0}, {workers, NbChildren}],
|
||||
To ! {Tag, Counts},
|
||||
loop(State);
|
||||
{'$gen_call', {To, Tag}, _} ->
|
||||
To ! {Tag, {error, ?MODULE}},
|
||||
loop(State)
|
||||
after 60000 ->
|
||||
goaway(State, ok),
|
||||
terminate(State)
|
||||
end.
|
||||
|
||||
system_continue(_, _, State) ->
|
||||
loop(State).
|
||||
|
||||
-spec system_terminate(any(), _, _, _) -> no_return().
|
||||
system_terminate(Reason, _, _, _) ->
|
||||
exit(Reason).
|
||||
|
||||
system_code_change(Misc, _, _, _) ->
|
||||
{ok, Misc}.
|
||||
|
||||
%% We do not support SYN_STREAM with FLAG_UNIDIRECTIONAL set.
|
||||
control_frame(State, << _:38, 1:1, _:26, StreamID:31, _/bits >>) ->
|
||||
rst_stream(State, StreamID, internal_error),
|
||||
loop(State);
|
||||
%% We do not support Associated-To-Stream-ID and CREDENTIAL Slot.
|
||||
control_frame(State, << _:65, StreamID:31, _:1, AssocToStreamID:31,
|
||||
_:8, Slot:8, _/bits >>) when AssocToStreamID =/= 0; Slot =/= 0 ->
|
||||
rst_stream(State, StreamID, internal_error),
|
||||
loop(State);
|
||||
%% SYN_STREAM
|
||||
%%
|
||||
%% Erlang does not allow us to control the priority of processes
|
||||
%% so we ignore that value entirely.
|
||||
control_frame(State=#state{middlewares=Middlewares, env=Env,
|
||||
onrequest=OnRequest, onresponse=OnResponse, peer=Peer,
|
||||
zinf=Zinf, children=Children},
|
||||
<< 1:1, 3:15, 1:16, Flags:8, _:25, StreamID:31,
|
||||
_:32, _Priority:3, _:13, Rest/bits >>) ->
|
||||
IsFin = case Flags of
|
||||
1 -> fin;
|
||||
0 -> nofin
|
||||
end,
|
||||
[<< NbHeaders:32, Rest2/bits >>] = try
|
||||
zlib:inflate(Zinf, Rest)
|
||||
catch _:_ ->
|
||||
ok = zlib:inflateSetDictionary(Zinf, ?ZDICT),
|
||||
zlib:inflate(Zinf, <<>>)
|
||||
end,
|
||||
case syn_stream_headers(Rest2, NbHeaders, [], #special_headers{}) of
|
||||
{ok, Headers, Special} ->
|
||||
Pid = spawn_link(?MODULE, request_init,
|
||||
[self(), StreamID, Peer, Headers,
|
||||
OnRequest, OnResponse, Env, Middlewares, Special]),
|
||||
loop(State#state{last_streamid=StreamID,
|
||||
children=[#child{streamid=StreamID, pid=Pid,
|
||||
input=IsFin, output=nofin}|Children]});
|
||||
{error, special} ->
|
||||
rst_stream(State, StreamID, protocol_error),
|
||||
loop(State#state{last_streamid=StreamID})
|
||||
end;
|
||||
%% SYN_REPLY
|
||||
control_frame(State, << 1:1, 3:15, 2:16, _/bits >>) ->
|
||||
error_logger:error_msg("Ignored SYN_REPLY control frame~n"),
|
||||
loop(State);
|
||||
%% RST_STREAM
|
||||
control_frame(State, << 1:1, 3:15, 3:16, _Flags:8, _Length:24,
|
||||
_:1, _StreamID:31, StatusCode:32 >>) ->
|
||||
Status = case StatusCode of
|
||||
1 -> protocol_error;
|
||||
2 -> invalid_stream;
|
||||
3 -> refused_stream;
|
||||
4 -> unsupported_version;
|
||||
5 -> cancel;
|
||||
6 -> internal_error;
|
||||
7 -> flow_control_error;
|
||||
8 -> stream_in_use;
|
||||
9 -> stream_already_closed;
|
||||
10 -> invalid_credentials;
|
||||
11 -> frame_too_large
|
||||
end,
|
||||
error_logger:error_msg("Received RST_STREAM control frame: ~p~n", [Status]),
|
||||
%% @todo Stop StreamID.
|
||||
loop(State);
|
||||
%% SETTINGS
|
||||
control_frame(State, << 1:1, 3:15, 4:16, 0:8, _:24,
|
||||
NbEntries:32, Rest/bits >>) ->
|
||||
Settings = [begin
|
||||
Name = case ID of
|
||||
1 -> upload_bandwidth;
|
||||
2 -> download_bandwidth;
|
||||
3 -> round_trip_time;
|
||||
4 -> max_concurrent_streams;
|
||||
5 -> current_cwnd;
|
||||
6 -> download_retrans_rate;
|
||||
7 -> initial_window_size;
|
||||
8 -> client_certificate_vector_size
|
||||
end,
|
||||
{Flags, Name, Value}
|
||||
end || << Flags:8, ID:24, Value:32 >> <= Rest],
|
||||
if
|
||||
NbEntries =/= length(Settings) ->
|
||||
goaway(State, protocol_error),
|
||||
terminate(State);
|
||||
true ->
|
||||
error_logger:error_msg("Ignored SETTINGS control frame: ~p~n",
|
||||
[Settings]),
|
||||
loop(State)
|
||||
end;
|
||||
%% PING initiated by the server; ignore, we don't send any
|
||||
control_frame(State, << 1:1, 3:15, 6:16, 0:8, 4:24, PingID:32 >>)
|
||||
when PingID rem 2 =:= 0 ->
|
||||
error_logger:error_msg("Ignored PING control frame: ~p~n", [PingID]),
|
||||
loop(State);
|
||||
%% PING initiated by the client; send it back
|
||||
control_frame(State=#state{socket=Socket, transport=Transport},
|
||||
Data = << 1:1, 3:15, 6:16, 0:8, 4:24, _:32 >>) ->
|
||||
Transport:send(Socket, Data),
|
||||
loop(State);
|
||||
%% GOAWAY
|
||||
control_frame(State, << 1:1, 3:15, 7:16, _/bits >>) ->
|
||||
error_logger:error_msg("Ignored GOAWAY control frame~n"),
|
||||
loop(State);
|
||||
%% HEADERS
|
||||
control_frame(State, << 1:1, 3:15, 8:16, _/bits >>) ->
|
||||
error_logger:error_msg("Ignored HEADERS control frame~n"),
|
||||
loop(State);
|
||||
%% WINDOW_UPDATE
|
||||
control_frame(State, << 1:1, 3:15, 9:16, 0:8, _/bits >>) ->
|
||||
error_logger:error_msg("Ignored WINDOW_UPDATE control frame~n"),
|
||||
loop(State);
|
||||
%% CREDENTIAL
|
||||
control_frame(State, << 1:1, 3:15, 10:16, _/bits >>) ->
|
||||
error_logger:error_msg("Ignored CREDENTIAL control frame~n"),
|
||||
loop(State);
|
||||
%% ???
|
||||
control_frame(State, _) ->
|
||||
goaway(State, protocol_error),
|
||||
terminate(State).
|
||||
|
||||
%% @todo We must wait for the children to finish here,
|
||||
%% but only up to N milliseconds. Then we shutdown.
|
||||
terminate(_State) ->
|
||||
ok.
|
||||
|
||||
syn_stream_headers(<<>>, 0, Acc, Special=#special_headers{
|
||||
method=Method, path=Path, version=Version, host=Host, scheme=Scheme}) ->
|
||||
if
|
||||
Method =:= undefined; Path =:= undefined; Version =:= undefined;
|
||||
Host =:= undefined; Scheme =:= undefined ->
|
||||
{error, special};
|
||||
true ->
|
||||
{ok, lists:reverse(Acc), Special}
|
||||
end;
|
||||
syn_stream_headers(<< NameLen:32, Rest/bits >>, NbHeaders, Acc, Special) ->
|
||||
<< Name:NameLen/binary, ValueLen:32, Rest2/bits >> = Rest,
|
||||
<< Value:ValueLen/binary, Rest3/bits >> = Rest2,
|
||||
case Name of
|
||||
<<":host">> ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1,
|
||||
[{<<"host">>, Value}|Acc],
|
||||
Special#special_headers{host=Value});
|
||||
<<":method">> ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1, Acc,
|
||||
Special#special_headers{method=Value});
|
||||
<<":path">> ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1, Acc,
|
||||
Special#special_headers{path=Value});
|
||||
<<":version">> ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1, Acc,
|
||||
Special#special_headers{version=Value});
|
||||
<<":scheme">> ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1, Acc,
|
||||
Special#special_headers{scheme=Value});
|
||||
_ ->
|
||||
syn_stream_headers(Rest3, NbHeaders - 1,
|
||||
[{Name, Value}|Acc], Special)
|
||||
end.
|
||||
|
||||
syn_reply(#state{socket=Socket, transport=Transport, zdef=Zdef},
|
||||
IsFin, StreamID, Status, Headers) ->
|
||||
Headers2 = [{<<":status">>, Status},
|
||||
{<<":version">>, <<"HTTP/1.1">>}|Headers],
|
||||
NbHeaders = length(Headers2),
|
||||
HeaderBlock = [begin
|
||||
NameLen = byte_size(Name),
|
||||
ValueLen = iolist_size(Value),
|
||||
[<< NameLen:32, Name/binary, ValueLen:32 >>, Value]
|
||||
end || {Name, Value} <- Headers2],
|
||||
HeaderBlock2 = [<< NbHeaders:32 >>, HeaderBlock],
|
||||
HeaderBlock3 = zlib:deflate(Zdef, HeaderBlock2, full),
|
||||
Flags = case IsFin of
|
||||
fin -> 1;
|
||||
nofin -> 0
|
||||
end,
|
||||
Len = 4 + iolist_size(HeaderBlock3),
|
||||
Transport:send(Socket, [
|
||||
<< 1:1, 3:15, 2:16, Flags:8, Len:24, 0:1, StreamID:31 >>,
|
||||
HeaderBlock3]).
|
||||
|
||||
rst_stream(#state{socket=Socket, transport=Transport}, StreamID, Status) ->
|
||||
StatusCode = case Status of
|
||||
protocol_error -> 1;
|
||||
%% invalid_stream -> 2;
|
||||
%% refused_stream -> 3;
|
||||
%% unsupported_version -> 4;
|
||||
%% cancel -> 5;
|
||||
internal_error -> 6
|
||||
%% flow_control_error -> 7;
|
||||
%% stream_in_use -> 8;
|
||||
%% stream_already_closed -> 9;
|
||||
%% invalid_credentials -> 10;
|
||||
%% frame_too_large -> 11
|
||||
end,
|
||||
Transport:send(Socket, << 1:1, 3:15, 3:16, 0:8, 8:24,
|
||||
0:1, StreamID:31, StatusCode:32 >>).
|
||||
|
||||
goaway(#state{socket=Socket, transport=Transport, last_streamid=LastStreamID},
|
||||
Status) ->
|
||||
StatusCode = case Status of
|
||||
ok -> 0;
|
||||
protocol_error -> 1
|
||||
%% internal_error -> 2
|
||||
end,
|
||||
Transport:send(Socket, << 1:1, 3:15, 7:16, 0:8, 8:24,
|
||||
0:1, LastStreamID:31, StatusCode:32 >>).
|
||||
|
||||
data(#state{socket=Socket, transport=Transport}, fin, StreamID) ->
|
||||
Transport:send(Socket, << 0:1, StreamID:31, 1:8, 0:24 >>).
|
||||
|
||||
data(#state{socket=Socket, transport=Transport}, IsFin, StreamID, Data) ->
|
||||
Flags = case IsFin of
|
||||
fin -> 1;
|
||||
nofin -> 0
|
||||
end,
|
||||
Len = iolist_size(Data),
|
||||
Transport:send(Socket, [
|
||||
<< 0:1, StreamID:31, Flags:8, Len:24 >>,
|
||||
Data]).
|
||||
|
||||
%% Request process.
|
||||
|
||||
request_init(Parent, StreamID, Peer,
|
||||
Headers, OnRequest, OnResponse, Env, Middlewares,
|
||||
#special_headers{method=Method, path=Path, version=Version,
|
||||
host=Host}) ->
|
||||
Version2 = parse_version(Version),
|
||||
{Host2, Port} = cowboy_protocol:parse_host(Host, <<>>),
|
||||
{Path2, Query} = parse_path(Path, <<>>),
|
||||
Req = cowboy_req:new({Parent, StreamID}, ?MODULE, Peer,
|
||||
Method, Path2, Query, Version2, Headers,
|
||||
Host2, Port, <<>>, true, false, OnResponse),
|
||||
case OnRequest of
|
||||
undefined ->
|
||||
execute(Req, Env, Middlewares);
|
||||
_ ->
|
||||
Req2 = OnRequest(Req),
|
||||
case cowboy_req:get(resp_state, Req2) of
|
||||
waiting -> execute(Req2, Env, Middlewares);
|
||||
_ -> ok
|
||||
end
|
||||
end.
|
||||
|
||||
parse_version(<<"HTTP/1.1">>) ->
|
||||
'HTTP/1.1';
|
||||
parse_version(<<"HTTP/1.0">>) ->
|
||||
'HTTP/1.0'.
|
||||
|
||||
parse_path(<<>>, Path) ->
|
||||
{Path, <<>>};
|
||||
parse_path(<< $?, Rest/binary >>, Path) ->
|
||||
parse_query(Rest, Path, <<>>);
|
||||
parse_path(<< C, Rest/binary >>, SoFar) ->
|
||||
parse_path(Rest, << SoFar/binary, C >>).
|
||||
|
||||
parse_query(<<>>, Path, Query) ->
|
||||
{Path, Query};
|
||||
parse_query(<< C, Rest/binary >>, Path, SoFar) ->
|
||||
parse_query(Rest, Path, << SoFar/binary, C >>).
|
||||
|
||||
-spec execute(cowboy_req:req(), cowboy_middleware:env(), [module()])
|
||||
-> ok.
|
||||
execute(Req, _, []) ->
|
||||
cowboy_req:ensure_response(Req, 204);
|
||||
execute(Req, Env, [Middleware|Tail]) ->
|
||||
case Middleware:execute(Req, Env) of
|
||||
{ok, Req2, Env2} ->
|
||||
execute(Req2, Env2, Tail);
|
||||
{suspend, Module, Function, Args} ->
|
||||
erlang:hibernate(?MODULE, resume,
|
||||
[Env, Tail, Module, Function, Args]);
|
||||
{halt, Req2} ->
|
||||
cowboy_req:ensure_response(Req2, 204);
|
||||
{error, Code, Req2} ->
|
||||
error_terminate(Code, Req2)
|
||||
end.
|
||||
|
||||
%% @private
|
||||
-spec resume(cowboy_middleware:env(), [module()],
|
||||
module(), module(), [any()]) -> ok.
|
||||
resume(Env, Tail, Module, Function, Args) ->
|
||||
case apply(Module, Function, Args) of
|
||||
{ok, Req2, Env2} ->
|
||||
execute(Req2, Env2, Tail);
|
||||
{suspend, Module2, Function2, Args2} ->
|
||||
erlang:hibernate(?MODULE, resume,
|
||||
[Env, Tail, Module2, Function2, Args2]);
|
||||
{halt, Req2} ->
|
||||
cowboy_req:ensure_response(Req2, 204);
|
||||
{error, Code, Req2} ->
|
||||
error_terminate(Code, Req2)
|
||||
end.
|
||||
|
||||
%% Only send an error reply if there is no resp_sent message.
|
||||
-spec error_terminate(cowboy:http_status(), cowboy_req:req()) -> ok.
|
||||
error_terminate(Code, Req) ->
|
||||
receive
|
||||
{cowboy_req, resp_sent} -> ok
|
||||
after 0 ->
|
||||
_ = cowboy_req:reply(Code, Req),
|
||||
ok
|
||||
end.
|
||||
|
||||
%% Reply functions used by cowboy_req.
|
||||
|
||||
reply(Socket = {Pid, _}, Status, Headers, Body) ->
|
||||
_ = case iolist_size(Body) of
|
||||
0 -> Pid ! {reply, Socket, Status, Headers};
|
||||
_ -> Pid ! {reply, Socket, Status, Headers, Body}
|
||||
end,
|
||||
ok.
|
||||
|
||||
stream_reply(Socket = {Pid, _}, Status, Headers) ->
|
||||
_ = Pid ! {stream_reply, Socket, Status, Headers},
|
||||
ok.
|
||||
|
||||
stream_data(Socket = {Pid, _}, Data) ->
|
||||
_ = Pid ! {stream_data, Socket, Data},
|
||||
ok.
|
||||
|
||||
stream_close(Socket = {Pid, _}) ->
|
||||
_ = Pid ! {stream_close, Socket},
|
||||
ok.
|
||||
|
||||
%% Internal transport functions.
|
||||
%% @todo recv, sendfile
|
||||
|
||||
name() ->
|
||||
spdy.
|
||||
|
||||
send(Socket, Data) ->
|
||||
stream_data(Socket, Data).
|
181
src/cowboy_spdy.hrl
Normal file
181
src/cowboy_spdy.hrl
Normal file
|
@ -0,0 +1,181 @@
|
|||
%% Zlib dictionary.
|
||||
|
||||
-define(ZDICT, <<
|
||||
16#00, 16#00, 16#00, 16#07, 16#6f, 16#70, 16#74, 16#69,
|
||||
16#6f, 16#6e, 16#73, 16#00, 16#00, 16#00, 16#04, 16#68,
|
||||
16#65, 16#61, 16#64, 16#00, 16#00, 16#00, 16#04, 16#70,
|
||||
16#6f, 16#73, 16#74, 16#00, 16#00, 16#00, 16#03, 16#70,
|
||||
16#75, 16#74, 16#00, 16#00, 16#00, 16#06, 16#64, 16#65,
|
||||
16#6c, 16#65, 16#74, 16#65, 16#00, 16#00, 16#00, 16#05,
|
||||
16#74, 16#72, 16#61, 16#63, 16#65, 16#00, 16#00, 16#00,
|
||||
16#06, 16#61, 16#63, 16#63, 16#65, 16#70, 16#74, 16#00,
|
||||
16#00, 16#00, 16#0e, 16#61, 16#63, 16#63, 16#65, 16#70,
|
||||
16#74, 16#2d, 16#63, 16#68, 16#61, 16#72, 16#73, 16#65,
|
||||
16#74, 16#00, 16#00, 16#00, 16#0f, 16#61, 16#63, 16#63,
|
||||
16#65, 16#70, 16#74, 16#2d, 16#65, 16#6e, 16#63, 16#6f,
|
||||
16#64, 16#69, 16#6e, 16#67, 16#00, 16#00, 16#00, 16#0f,
|
||||
16#61, 16#63, 16#63, 16#65, 16#70, 16#74, 16#2d, 16#6c,
|
||||
16#61, 16#6e, 16#67, 16#75, 16#61, 16#67, 16#65, 16#00,
|
||||
16#00, 16#00, 16#0d, 16#61, 16#63, 16#63, 16#65, 16#70,
|
||||
16#74, 16#2d, 16#72, 16#61, 16#6e, 16#67, 16#65, 16#73,
|
||||
16#00, 16#00, 16#00, 16#03, 16#61, 16#67, 16#65, 16#00,
|
||||
16#00, 16#00, 16#05, 16#61, 16#6c, 16#6c, 16#6f, 16#77,
|
||||
16#00, 16#00, 16#00, 16#0d, 16#61, 16#75, 16#74, 16#68,
|
||||
16#6f, 16#72, 16#69, 16#7a, 16#61, 16#74, 16#69, 16#6f,
|
||||
16#6e, 16#00, 16#00, 16#00, 16#0d, 16#63, 16#61, 16#63,
|
||||
16#68, 16#65, 16#2d, 16#63, 16#6f, 16#6e, 16#74, 16#72,
|
||||
16#6f, 16#6c, 16#00, 16#00, 16#00, 16#0a, 16#63, 16#6f,
|
||||
16#6e, 16#6e, 16#65, 16#63, 16#74, 16#69, 16#6f, 16#6e,
|
||||
16#00, 16#00, 16#00, 16#0c, 16#63, 16#6f, 16#6e, 16#74,
|
||||
16#65, 16#6e, 16#74, 16#2d, 16#62, 16#61, 16#73, 16#65,
|
||||
16#00, 16#00, 16#00, 16#10, 16#63, 16#6f, 16#6e, 16#74,
|
||||
16#65, 16#6e, 16#74, 16#2d, 16#65, 16#6e, 16#63, 16#6f,
|
||||
16#64, 16#69, 16#6e, 16#67, 16#00, 16#00, 16#00, 16#10,
|
||||
16#63, 16#6f, 16#6e, 16#74, 16#65, 16#6e, 16#74, 16#2d,
|
||||
16#6c, 16#61, 16#6e, 16#67, 16#75, 16#61, 16#67, 16#65,
|
||||
16#00, 16#00, 16#00, 16#0e, 16#63, 16#6f, 16#6e, 16#74,
|
||||
16#65, 16#6e, 16#74, 16#2d, 16#6c, 16#65, 16#6e, 16#67,
|
||||
16#74, 16#68, 16#00, 16#00, 16#00, 16#10, 16#63, 16#6f,
|
||||
16#6e, 16#74, 16#65, 16#6e, 16#74, 16#2d, 16#6c, 16#6f,
|
||||
16#63, 16#61, 16#74, 16#69, 16#6f, 16#6e, 16#00, 16#00,
|
||||
16#00, 16#0b, 16#63, 16#6f, 16#6e, 16#74, 16#65, 16#6e,
|
||||
16#74, 16#2d, 16#6d, 16#64, 16#35, 16#00, 16#00, 16#00,
|
||||
16#0d, 16#63, 16#6f, 16#6e, 16#74, 16#65, 16#6e, 16#74,
|
||||
16#2d, 16#72, 16#61, 16#6e, 16#67, 16#65, 16#00, 16#00,
|
||||
16#00, 16#0c, 16#63, 16#6f, 16#6e, 16#74, 16#65, 16#6e,
|
||||
16#74, 16#2d, 16#74, 16#79, 16#70, 16#65, 16#00, 16#00,
|
||||
16#00, 16#04, 16#64, 16#61, 16#74, 16#65, 16#00, 16#00,
|
||||
16#00, 16#04, 16#65, 16#74, 16#61, 16#67, 16#00, 16#00,
|
||||
16#00, 16#06, 16#65, 16#78, 16#70, 16#65, 16#63, 16#74,
|
||||
16#00, 16#00, 16#00, 16#07, 16#65, 16#78, 16#70, 16#69,
|
||||
16#72, 16#65, 16#73, 16#00, 16#00, 16#00, 16#04, 16#66,
|
||||
16#72, 16#6f, 16#6d, 16#00, 16#00, 16#00, 16#04, 16#68,
|
||||
16#6f, 16#73, 16#74, 16#00, 16#00, 16#00, 16#08, 16#69,
|
||||
16#66, 16#2d, 16#6d, 16#61, 16#74, 16#63, 16#68, 16#00,
|
||||
16#00, 16#00, 16#11, 16#69, 16#66, 16#2d, 16#6d, 16#6f,
|
||||
16#64, 16#69, 16#66, 16#69, 16#65, 16#64, 16#2d, 16#73,
|
||||
16#69, 16#6e, 16#63, 16#65, 16#00, 16#00, 16#00, 16#0d,
|
||||
16#69, 16#66, 16#2d, 16#6e, 16#6f, 16#6e, 16#65, 16#2d,
|
||||
16#6d, 16#61, 16#74, 16#63, 16#68, 16#00, 16#00, 16#00,
|
||||
16#08, 16#69, 16#66, 16#2d, 16#72, 16#61, 16#6e, 16#67,
|
||||
16#65, 16#00, 16#00, 16#00, 16#13, 16#69, 16#66, 16#2d,
|
||||
16#75, 16#6e, 16#6d, 16#6f, 16#64, 16#69, 16#66, 16#69,
|
||||
16#65, 16#64, 16#2d, 16#73, 16#69, 16#6e, 16#63, 16#65,
|
||||
16#00, 16#00, 16#00, 16#0d, 16#6c, 16#61, 16#73, 16#74,
|
||||
16#2d, 16#6d, 16#6f, 16#64, 16#69, 16#66, 16#69, 16#65,
|
||||
16#64, 16#00, 16#00, 16#00, 16#08, 16#6c, 16#6f, 16#63,
|
||||
16#61, 16#74, 16#69, 16#6f, 16#6e, 16#00, 16#00, 16#00,
|
||||
16#0c, 16#6d, 16#61, 16#78, 16#2d, 16#66, 16#6f, 16#72,
|
||||
16#77, 16#61, 16#72, 16#64, 16#73, 16#00, 16#00, 16#00,
|
||||
16#06, 16#70, 16#72, 16#61, 16#67, 16#6d, 16#61, 16#00,
|
||||
16#00, 16#00, 16#12, 16#70, 16#72, 16#6f, 16#78, 16#79,
|
||||
16#2d, 16#61, 16#75, 16#74, 16#68, 16#65, 16#6e, 16#74,
|
||||
16#69, 16#63, 16#61, 16#74, 16#65, 16#00, 16#00, 16#00,
|
||||
16#13, 16#70, 16#72, 16#6f, 16#78, 16#79, 16#2d, 16#61,
|
||||
16#75, 16#74, 16#68, 16#6f, 16#72, 16#69, 16#7a, 16#61,
|
||||
16#74, 16#69, 16#6f, 16#6e, 16#00, 16#00, 16#00, 16#05,
|
||||
16#72, 16#61, 16#6e, 16#67, 16#65, 16#00, 16#00, 16#00,
|
||||
16#07, 16#72, 16#65, 16#66, 16#65, 16#72, 16#65, 16#72,
|
||||
16#00, 16#00, 16#00, 16#0b, 16#72, 16#65, 16#74, 16#72,
|
||||
16#79, 16#2d, 16#61, 16#66, 16#74, 16#65, 16#72, 16#00,
|
||||
16#00, 16#00, 16#06, 16#73, 16#65, 16#72, 16#76, 16#65,
|
||||
16#72, 16#00, 16#00, 16#00, 16#02, 16#74, 16#65, 16#00,
|
||||
16#00, 16#00, 16#07, 16#74, 16#72, 16#61, 16#69, 16#6c,
|
||||
16#65, 16#72, 16#00, 16#00, 16#00, 16#11, 16#74, 16#72,
|
||||
16#61, 16#6e, 16#73, 16#66, 16#65, 16#72, 16#2d, 16#65,
|
||||
16#6e, 16#63, 16#6f, 16#64, 16#69, 16#6e, 16#67, 16#00,
|
||||
16#00, 16#00, 16#07, 16#75, 16#70, 16#67, 16#72, 16#61,
|
||||
16#64, 16#65, 16#00, 16#00, 16#00, 16#0a, 16#75, 16#73,
|
||||
16#65, 16#72, 16#2d, 16#61, 16#67, 16#65, 16#6e, 16#74,
|
||||
16#00, 16#00, 16#00, 16#04, 16#76, 16#61, 16#72, 16#79,
|
||||
16#00, 16#00, 16#00, 16#03, 16#76, 16#69, 16#61, 16#00,
|
||||
16#00, 16#00, 16#07, 16#77, 16#61, 16#72, 16#6e, 16#69,
|
||||
16#6e, 16#67, 16#00, 16#00, 16#00, 16#10, 16#77, 16#77,
|
||||
16#77, 16#2d, 16#61, 16#75, 16#74, 16#68, 16#65, 16#6e,
|
||||
16#74, 16#69, 16#63, 16#61, 16#74, 16#65, 16#00, 16#00,
|
||||
16#00, 16#06, 16#6d, 16#65, 16#74, 16#68, 16#6f, 16#64,
|
||||
16#00, 16#00, 16#00, 16#03, 16#67, 16#65, 16#74, 16#00,
|
||||
16#00, 16#00, 16#06, 16#73, 16#74, 16#61, 16#74, 16#75,
|
||||
16#73, 16#00, 16#00, 16#00, 16#06, 16#32, 16#30, 16#30,
|
||||
16#20, 16#4f, 16#4b, 16#00, 16#00, 16#00, 16#07, 16#76,
|
||||
16#65, 16#72, 16#73, 16#69, 16#6f, 16#6e, 16#00, 16#00,
|
||||
16#00, 16#08, 16#48, 16#54, 16#54, 16#50, 16#2f, 16#31,
|
||||
16#2e, 16#31, 16#00, 16#00, 16#00, 16#03, 16#75, 16#72,
|
||||
16#6c, 16#00, 16#00, 16#00, 16#06, 16#70, 16#75, 16#62,
|
||||
16#6c, 16#69, 16#63, 16#00, 16#00, 16#00, 16#0a, 16#73,
|
||||
16#65, 16#74, 16#2d, 16#63, 16#6f, 16#6f, 16#6b, 16#69,
|
||||
16#65, 16#00, 16#00, 16#00, 16#0a, 16#6b, 16#65, 16#65,
|
||||
16#70, 16#2d, 16#61, 16#6c, 16#69, 16#76, 16#65, 16#00,
|
||||
16#00, 16#00, 16#06, 16#6f, 16#72, 16#69, 16#67, 16#69,
|
||||
16#6e, 16#31, 16#30, 16#30, 16#31, 16#30, 16#31, 16#32,
|
||||
16#30, 16#31, 16#32, 16#30, 16#32, 16#32, 16#30, 16#35,
|
||||
16#32, 16#30, 16#36, 16#33, 16#30, 16#30, 16#33, 16#30,
|
||||
16#32, 16#33, 16#30, 16#33, 16#33, 16#30, 16#34, 16#33,
|
||||
16#30, 16#35, 16#33, 16#30, 16#36, 16#33, 16#30, 16#37,
|
||||
16#34, 16#30, 16#32, 16#34, 16#30, 16#35, 16#34, 16#30,
|
||||
16#36, 16#34, 16#30, 16#37, 16#34, 16#30, 16#38, 16#34,
|
||||
16#30, 16#39, 16#34, 16#31, 16#30, 16#34, 16#31, 16#31,
|
||||
16#34, 16#31, 16#32, 16#34, 16#31, 16#33, 16#34, 16#31,
|
||||
16#34, 16#34, 16#31, 16#35, 16#34, 16#31, 16#36, 16#34,
|
||||
16#31, 16#37, 16#35, 16#30, 16#32, 16#35, 16#30, 16#34,
|
||||
16#35, 16#30, 16#35, 16#32, 16#30, 16#33, 16#20, 16#4e,
|
||||
16#6f, 16#6e, 16#2d, 16#41, 16#75, 16#74, 16#68, 16#6f,
|
||||
16#72, 16#69, 16#74, 16#61, 16#74, 16#69, 16#76, 16#65,
|
||||
16#20, 16#49, 16#6e, 16#66, 16#6f, 16#72, 16#6d, 16#61,
|
||||
16#74, 16#69, 16#6f, 16#6e, 16#32, 16#30, 16#34, 16#20,
|
||||
16#4e, 16#6f, 16#20, 16#43, 16#6f, 16#6e, 16#74, 16#65,
|
||||
16#6e, 16#74, 16#33, 16#30, 16#31, 16#20, 16#4d, 16#6f,
|
||||
16#76, 16#65, 16#64, 16#20, 16#50, 16#65, 16#72, 16#6d,
|
||||
16#61, 16#6e, 16#65, 16#6e, 16#74, 16#6c, 16#79, 16#34,
|
||||
16#30, 16#30, 16#20, 16#42, 16#61, 16#64, 16#20, 16#52,
|
||||
16#65, 16#71, 16#75, 16#65, 16#73, 16#74, 16#34, 16#30,
|
||||
16#31, 16#20, 16#55, 16#6e, 16#61, 16#75, 16#74, 16#68,
|
||||
16#6f, 16#72, 16#69, 16#7a, 16#65, 16#64, 16#34, 16#30,
|
||||
16#33, 16#20, 16#46, 16#6f, 16#72, 16#62, 16#69, 16#64,
|
||||
16#64, 16#65, 16#6e, 16#34, 16#30, 16#34, 16#20, 16#4e,
|
||||
16#6f, 16#74, 16#20, 16#46, 16#6f, 16#75, 16#6e, 16#64,
|
||||
16#35, 16#30, 16#30, 16#20, 16#49, 16#6e, 16#74, 16#65,
|
||||
16#72, 16#6e, 16#61, 16#6c, 16#20, 16#53, 16#65, 16#72,
|
||||
16#76, 16#65, 16#72, 16#20, 16#45, 16#72, 16#72, 16#6f,
|
||||
16#72, 16#35, 16#30, 16#31, 16#20, 16#4e, 16#6f, 16#74,
|
||||
16#20, 16#49, 16#6d, 16#70, 16#6c, 16#65, 16#6d, 16#65,
|
||||
16#6e, 16#74, 16#65, 16#64, 16#35, 16#30, 16#33, 16#20,
|
||||
16#53, 16#65, 16#72, 16#76, 16#69, 16#63, 16#65, 16#20,
|
||||
16#55, 16#6e, 16#61, 16#76, 16#61, 16#69, 16#6c, 16#61,
|
||||
16#62, 16#6c, 16#65, 16#4a, 16#61, 16#6e, 16#20, 16#46,
|
||||
16#65, 16#62, 16#20, 16#4d, 16#61, 16#72, 16#20, 16#41,
|
||||
16#70, 16#72, 16#20, 16#4d, 16#61, 16#79, 16#20, 16#4a,
|
||||
16#75, 16#6e, 16#20, 16#4a, 16#75, 16#6c, 16#20, 16#41,
|
||||
16#75, 16#67, 16#20, 16#53, 16#65, 16#70, 16#74, 16#20,
|
||||
16#4f, 16#63, 16#74, 16#20, 16#4e, 16#6f, 16#76, 16#20,
|
||||
16#44, 16#65, 16#63, 16#20, 16#30, 16#30, 16#3a, 16#30,
|
||||
16#30, 16#3a, 16#30, 16#30, 16#20, 16#4d, 16#6f, 16#6e,
|
||||
16#2c, 16#20, 16#54, 16#75, 16#65, 16#2c, 16#20, 16#57,
|
||||
16#65, 16#64, 16#2c, 16#20, 16#54, 16#68, 16#75, 16#2c,
|
||||
16#20, 16#46, 16#72, 16#69, 16#2c, 16#20, 16#53, 16#61,
|
||||
16#74, 16#2c, 16#20, 16#53, 16#75, 16#6e, 16#2c, 16#20,
|
||||
16#47, 16#4d, 16#54, 16#63, 16#68, 16#75, 16#6e, 16#6b,
|
||||
16#65, 16#64, 16#2c, 16#74, 16#65, 16#78, 16#74, 16#2f,
|
||||
16#68, 16#74, 16#6d, 16#6c, 16#2c, 16#69, 16#6d, 16#61,
|
||||
16#67, 16#65, 16#2f, 16#70, 16#6e, 16#67, 16#2c, 16#69,
|
||||
16#6d, 16#61, 16#67, 16#65, 16#2f, 16#6a, 16#70, 16#67,
|
||||
16#2c, 16#69, 16#6d, 16#61, 16#67, 16#65, 16#2f, 16#67,
|
||||
16#69, 16#66, 16#2c, 16#61, 16#70, 16#70, 16#6c, 16#69,
|
||||
16#63, 16#61, 16#74, 16#69, 16#6f, 16#6e, 16#2f, 16#78,
|
||||
16#6d, 16#6c, 16#2c, 16#61, 16#70, 16#70, 16#6c, 16#69,
|
||||
16#63, 16#61, 16#74, 16#69, 16#6f, 16#6e, 16#2f, 16#78,
|
||||
16#68, 16#74, 16#6d, 16#6c, 16#2b, 16#78, 16#6d, 16#6c,
|
||||
16#2c, 16#74, 16#65, 16#78, 16#74, 16#2f, 16#70, 16#6c,
|
||||
16#61, 16#69, 16#6e, 16#2c, 16#74, 16#65, 16#78, 16#74,
|
||||
16#2f, 16#6a, 16#61, 16#76, 16#61, 16#73, 16#63, 16#72,
|
||||
16#69, 16#70, 16#74, 16#2c, 16#70, 16#75, 16#62, 16#6c,
|
||||
16#69, 16#63, 16#70, 16#72, 16#69, 16#76, 16#61, 16#74,
|
||||
16#65, 16#6d, 16#61, 16#78, 16#2d, 16#61, 16#67, 16#65,
|
||||
16#3d, 16#67, 16#7a, 16#69, 16#70, 16#2c, 16#64, 16#65,
|
||||
16#66, 16#6c, 16#61, 16#74, 16#65, 16#2c, 16#73, 16#64,
|
||||
16#63, 16#68, 16#63, 16#68, 16#61, 16#72, 16#73, 16#65,
|
||||
16#74, 16#3d, 16#75, 16#74, 16#66, 16#2d, 16#38, 16#63,
|
||||
16#68, 16#61, 16#72, 16#73, 16#65, 16#74, 16#3d, 16#69,
|
||||
16#73, 16#6f, 16#2d, 16#38, 16#38, 16#35, 16#39, 16#2d,
|
||||
16#31, 16#2c, 16#75, 16#74, 16#66, 16#2d, 16#2c, 16#2a,
|
||||
16#2c, 16#65, 16#6e, 16#71, 16#3d, 16#30, 16#2e >>).
|
163
test/spdy_SUITE.erl
Normal file
163
test/spdy_SUITE.erl
Normal file
|
@ -0,0 +1,163 @@
|
|||
%% Copyright (c) 2013, Loïc Hoguin <essen@ninenines.eu>
|
||||
%%
|
||||
%% Permission to use, copy, modify, and/or distribute this software for any
|
||||
%% purpose with or without fee is hereby granted, provided that the above
|
||||
%% copyright notice and this permission notice appear in all copies.
|
||||
%%
|
||||
%% THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES
|
||||
%% WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF
|
||||
%% MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR
|
||||
%% ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES
|
||||
%% WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN
|
||||
%% ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
|
||||
%% OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
|
||||
|
||||
-module(spdy_SUITE).
|
||||
|
||||
-include_lib("common_test/include/ct.hrl").
|
||||
-include("../src/cowboy_spdy.hrl").
|
||||
|
||||
%% ct.
|
||||
-export([all/0]).
|
||||
-export([groups/0]).
|
||||
-export([init_per_suite/1]).
|
||||
-export([end_per_suite/1]).
|
||||
-export([init_per_group/2]).
|
||||
-export([end_per_group/2]).
|
||||
|
||||
%% Tests.
|
||||
-export([check_status/1]).
|
||||
|
||||
%% ct.
|
||||
|
||||
all() ->
|
||||
[{group, spdy}].
|
||||
|
||||
groups() ->
|
||||
[{spdy, [], [
|
||||
check_status
|
||||
]}].
|
||||
|
||||
init_per_suite(Config) ->
|
||||
application:start(crypto),
|
||||
application:start(ranch),
|
||||
application:start(cowboy),
|
||||
application:start(public_key),
|
||||
application:start(ssl),
|
||||
Config.
|
||||
|
||||
end_per_suite(_Config) ->
|
||||
application:stop(ssl),
|
||||
application:stop(public_key),
|
||||
application:stop(cowboy),
|
||||
application:stop(ranch),
|
||||
application:stop(crypto),
|
||||
ok.
|
||||
|
||||
init_per_group(Name, Config) ->
|
||||
{_, Cert, Key} = ct_helper:make_certs(),
|
||||
Opts = [{cert, Cert}, {key, Key}],
|
||||
{ok, _} = cowboy:start_spdy(Name, 100, Opts ++ [{port, 0}], [
|
||||
{env, [{dispatch, init_dispatch(Config)}]}
|
||||
]),
|
||||
Port = ranch:get_port(Name),
|
||||
[{port, Port}|Config].
|
||||
|
||||
end_per_group(Name, _) ->
|
||||
cowboy:stop_listener(Name),
|
||||
ok.
|
||||
|
||||
%% Dispatch configuration.
|
||||
|
||||
init_dispatch(_) ->
|
||||
cowboy_router:compile([
|
||||
{"localhost", [
|
||||
{"/chunked", http_chunked, []},
|
||||
{"/", http_handler, []}
|
||||
]}
|
||||
]).
|
||||
|
||||
%% Convenience functions.
|
||||
|
||||
quick_get(Host, Path, ExpectedFlags, Config) ->
|
||||
{_, Port} = lists:keyfind(port, 1, Config),
|
||||
{ok, Socket} = ssl:connect("localhost", Port, [
|
||||
binary, {active, false},
|
||||
{client_preferred_next_protocols, client, [<<"spdy/3">>]}
|
||||
]),
|
||||
{Zdef, Zinf} = zlib_init(),
|
||||
ReqHeaders = headers_encode(Zdef, [
|
||||
{<<":method">>, <<"GET">>},
|
||||
{<<":path">>, list_to_binary(Path)},
|
||||
{<<":version">>, <<"HTTP/1.1">>},
|
||||
{<<":host">>, list_to_binary(Host)},
|
||||
{<<":scheme">>, <<"https">>}
|
||||
]),
|
||||
ReqLength = 10 + byte_size(ReqHeaders),
|
||||
StreamID = 1,
|
||||
ok = ssl:send(Socket, << 1:1, 3:15, 1:16, 0:8, ReqLength:24,
|
||||
0:1, StreamID:31, 0:1, 0:31, 0:3, 0:5, 0:8, ReqHeaders/binary >>),
|
||||
{ok, Packet} = ssl:recv(Socket, 0, 1000),
|
||||
<< 1:1, 3:15, 2:16, Flags:8, RespLength:24,
|
||||
_:1, StreamID:31, RespHeaders/bits >> = Packet,
|
||||
Flags = ExpectedFlags,
|
||||
RespLength = 4 + byte_size(RespHeaders),
|
||||
[<< NbHeaders:32, Rest/bits >>] = try
|
||||
zlib:inflate(Zinf, RespHeaders)
|
||||
catch _:_ ->
|
||||
ok = zlib:inflateSetDictionary(Zinf, ?ZDICT),
|
||||
zlib:inflate(Zinf, <<>>)
|
||||
end,
|
||||
RespHeaders2 = headers_decode(Zinf, Rest, []),
|
||||
NbHeaders = length(RespHeaders2),
|
||||
{_, << Status:3/binary, _/bits >>}
|
||||
= lists:keyfind(<<":status">>, 1, RespHeaders2),
|
||||
StatusCode = list_to_integer(binary_to_list(Status)),
|
||||
ok = ssl:close(Socket),
|
||||
zlib_terminate(Zdef, Zinf),
|
||||
{StatusCode, RespHeaders2}.
|
||||
|
||||
zlib_init() ->
|
||||
Zdef = zlib:open(),
|
||||
ok = zlib:deflateInit(Zdef),
|
||||
_ = zlib:deflateSetDictionary(Zdef, ?ZDICT),
|
||||
Zinf = zlib:open(),
|
||||
ok = zlib:inflateInit(Zinf),
|
||||
{Zdef, Zinf}.
|
||||
|
||||
zlib_terminate(Zdef, Zinf) ->
|
||||
zlib:close(Zdef),
|
||||
zlib:close(Zinf).
|
||||
|
||||
headers_encode(Zdef, Headers) ->
|
||||
NbHeaders = length(Headers),
|
||||
Headers2 = << << (begin
|
||||
SizeN = byte_size(N),
|
||||
SizeV = byte_size(V),
|
||||
<< SizeN:32, N/binary, SizeV:32, V/binary >>
|
||||
end)/binary >> || {N, V} <- Headers >>,
|
||||
Headers3 = << NbHeaders:32, Headers2/binary >>,
|
||||
iolist_to_binary(zlib:deflate(Zdef, Headers3, full)).
|
||||
|
||||
headers_decode(_, <<>>, Acc) ->
|
||||
lists:reverse(Acc);
|
||||
headers_decode(Zinf, << SizeN:32, Rest/bits >>, Acc) ->
|
||||
<< Name:SizeN/binary, SizeV:32, Rest2/bits >> = Rest,
|
||||
<< Value:SizeV/binary, Rest3/bits >> = Rest2,
|
||||
headers_decode(Zinf, Rest3, [{Name, Value}|Acc]).
|
||||
|
||||
%% Tests.
|
||||
|
||||
check_status(Config) ->
|
||||
Tests = [
|
||||
{200, nofin, "localhost", "/"},
|
||||
{200, nofin, "localhost", "/chunked"},
|
||||
{400, fin, "bad-host", "/"},
|
||||
{400, fin, "localhost", "bad-path"},
|
||||
{404, fin, "localhost", "/this/path/does/not/exist"}
|
||||
],
|
||||
_ = [{Status, Fin, Host, Path} = begin
|
||||
RespFlags = case Fin of fin -> 1; nofin -> 0 end,
|
||||
{Ret, _} = quick_get(Host, Path, RespFlags, Config),
|
||||
{Ret, Fin, Host, Path}
|
||||
end || {Status, Fin, Host, Path} <- Tests].
|
Loading…
Add table
Add a link
Reference in a new issue