0
Fork 0
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:
Loïc Hoguin 2013-05-30 20:21:01 +02:00
parent c7f0834dc3
commit 9a2d35c2e8
7 changed files with 980 additions and 35 deletions

View file

@ -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.

View file

@ -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) ->

View file

@ -54,6 +54,7 @@
%% Internal.
-export([init/4]).
-export([parse_request/3]).
-export([parse_host/2]).
-export([resume/6]).
-type opts() :: [{compress, boolean()}

View file

@ -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
View 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
View 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
View 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].