0
Fork 0
mirror of https://github.com/ninenines/cowboy.git synced 2025-07-16 13:10:24 +00:00

Switch to Ranch for connection handling

This is the first of many API incompatible changes.

You have been warned.
This commit is contained in:
Loïc Hoguin 2012-08-27 11:50:35 +02:00
parent 50e5a616dc
commit e4124de2c7
35 changed files with 133 additions and 1066 deletions

View file

@ -14,12 +14,13 @@
{application, cowboy, [
{description, "Small, fast, modular HTTP server."},
{vsn, "0.6.1"},
{vsn, "0.7.0"},
{modules, []},
{registered, [cowboy_clock, cowboy_sup]},
{applications, [
kernel,
stdlib
stdlib,
ranch
]},
{mod, {cowboy_app, []}},
{env, []}

View file

@ -12,103 +12,28 @@
%% ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF
%% OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE.
%% @doc Cowboy API to start and stop listeners.
%% @doc Convenience API to start and stop HTTP/HTTPS listeners.
-module(cowboy).
-export([start_listener/6, stop_listener/1, child_spec/6, accept_ack/1,
get_protocol_options/1, set_protocol_options/2]).
-export([start_http/4]).
-export([start_https/4]).
-export([stop_listener/1]).
%% @doc Start a listener for the given transport and protocol.
%%
%% A listener is effectively a pool of <em>NbAcceptors</em> acceptors.
%% Acceptors accept connections on the given <em>Transport</em> and forward
%% requests to the given <em>Protocol</em> handler. Both transport and protocol
%% modules can be given options through the <em>TransOpts</em> and the
%% <em>ProtoOpts</em> arguments. Available options are documented in the
%% <em>listen</em> transport function and in the protocol module of your choice.
%%
%% All acceptor and request processes are supervised by the listener.
%%
%% It is recommended to set a large enough number of acceptors to improve
%% performance. The exact number depends of course on your hardware, on the
%% protocol used and on the number of expected simultaneous connections.
%%
%% The <em>Transport</em> option <em>max_connections</em> allows you to define
%% the maximum number of simultaneous connections for this listener. It defaults
%% to 1024. See <em>cowboy_listener</em> for more details on limiting the number
%% of connections.
%%
%% Although Cowboy includes a <em>cowboy_http_protocol</em> handler, other
%% handlers can be created for different protocols like IRC, FTP and more.
%%
%% <em>Ref</em> can be used to stop the listener later on.
-spec start_listener(any(), non_neg_integer(), module(), any(), module(), any())
-> {ok, pid()}.
start_listener(Ref, NbAcceptors, Transport, TransOpts, Protocol, ProtoOpts)
when is_integer(NbAcceptors) andalso is_atom(Transport)
andalso is_atom(Protocol) ->
supervisor:start_child(cowboy_sup, child_spec(Ref, NbAcceptors,
Transport, TransOpts, Protocol, ProtoOpts)).
%% @doc Start an HTTP listener.
-spec start_http(any(), non_neg_integer(), any(), any()) -> {ok, pid()}.
start_http(Ref, NbAcceptors, TransOpts, ProtoOpts)
when is_integer(NbAcceptors), NbAcceptors > 0 ->
ranch:start_listener(Ref, NbAcceptors,
ranch_tcp, TransOpts, cowboy_http_protocol, ProtoOpts).
%% @doc Stop a listener identified by <em>Ref</em>.
-spec stop_listener(any()) -> ok | {error, not_found}.
%% @doc Start an HTTPS listener.
-spec start_https(any(), non_neg_integer(), any(), any()) -> {ok, pid()}.
start_https(Ref, NbAcceptors, TransOpts, ProtoOpts)
when is_integer(NbAcceptors), NbAcceptors > 0 ->
ranch:start_listener(Ref, NbAcceptors,
ranch_ssl, TransOpts, cowboy_http_protocol, ProtoOpts).
%% @doc Stop a listener.
-spec stop_listener(any()) -> ok.
stop_listener(Ref) ->
case supervisor:terminate_child(cowboy_sup, {cowboy_listener_sup, Ref}) of
ok ->
supervisor:delete_child(cowboy_sup, {cowboy_listener_sup, Ref});
{error, Reason} ->
{error, Reason}
end.
%% @doc Return a child spec suitable for embedding.
%%
%% When you want to embed cowboy in another application, you can use this
%% function to create a <em>ChildSpec</em> suitable for use in a supervisor.
%% The parameters are the same as in <em>start_listener/6</em> but rather
%% than hooking the listener to the cowboy internal supervisor, it just returns
%% the spec.
-spec child_spec(any(), non_neg_integer(), module(), any(), module(), any())
-> supervisor:child_spec().
child_spec(Ref, NbAcceptors, Transport, TransOpts, Protocol, ProtoOpts)
when is_integer(NbAcceptors) andalso is_atom(Transport)
andalso is_atom(Protocol) ->
{{cowboy_listener_sup, Ref}, {cowboy_listener_sup, start_link, [
NbAcceptors, Transport, TransOpts, Protocol, ProtoOpts
]}, permanent, 5000, supervisor, [cowboy_listener_sup]}.
%% @doc Acknowledge the accepted connection.
%%
%% Effectively used to make sure the socket control has been given to
%% the protocol process before starting to use it.
-spec accept_ack(pid()) -> ok.
accept_ack(ListenerPid) ->
receive {shoot, ListenerPid} -> ok end.
%% @doc Return the current protocol options for the given listener.
-spec get_protocol_options(any()) -> any().
get_protocol_options(Ref) ->
ListenerPid = ref_to_listener_pid(Ref),
{ok, ProtoOpts} = cowboy_listener:get_protocol_options(ListenerPid),
ProtoOpts.
%% @doc Upgrade the protocol options for the given listener.
%%
%% The upgrade takes place at the acceptor level, meaning that only the
%% newly accepted connections receive the new protocol options. This has
%% no effect on the currently opened connections.
-spec set_protocol_options(any(), any()) -> ok.
set_protocol_options(Ref, ProtoOpts) ->
ListenerPid = ref_to_listener_pid(Ref),
ok = cowboy_listener:set_protocol_options(ListenerPid, ProtoOpts).
%% Internal.
-spec ref_to_listener_pid(any()) -> pid().
ref_to_listener_pid(Ref) ->
Children = supervisor:which_children(cowboy_sup),
{_, ListenerSupPid, _, _} = lists:keyfind(
{cowboy_listener_sup, Ref}, 1, Children),
ListenerSupChildren = supervisor:which_children(ListenerSupPid),
{_, ListenerPid, _, _} = lists:keyfind(
cowboy_listener, 1, ListenerSupChildren),
ListenerPid.
ranch:stop_listener(Ref).

View file

@ -1,57 +0,0 @@
%% Copyright (c) 2011-2012, 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.
%% @private
-module(cowboy_acceptor).
-export([start_link/6]). %% API.
-export([acceptor/7]). %% Internal.
%% API.
-spec start_link(inet:socket(), module(), module(), any(),
pid(), pid()) -> {ok, pid()}.
start_link(LSocket, Transport, Protocol, Opts,
ListenerPid, ReqsSup) ->
Pid = spawn_link(?MODULE, acceptor,
[LSocket, Transport, Protocol, Opts, 1, ListenerPid, ReqsSup]),
{ok, Pid}.
%% Internal.
-spec acceptor(inet:socket(), module(), module(), any(),
non_neg_integer(), pid(), pid()) -> no_return().
acceptor(LSocket, Transport, Protocol, Opts, OptsVsn, ListenerPid, ReqsSup) ->
Res = case Transport:accept(LSocket, 2000) of
{ok, CSocket} ->
{ok, Pid} = supervisor:start_child(ReqsSup,
[ListenerPid, CSocket, Transport, Protocol, Opts]),
Transport:controlling_process(CSocket, Pid),
cowboy_listener:add_connection(ListenerPid,
default, Pid, OptsVsn);
{error, timeout} ->
cowboy_listener:check_upgrades(ListenerPid, OptsVsn);
{error, _Reason} ->
%% @todo Probably do something here. If the socket was closed,
%% we may want to try and listen again on the port?
ok
end,
case Res of
ok ->
?MODULE:acceptor(LSocket, Transport, Protocol,
Opts, OptsVsn, ListenerPid, ReqsSup);
{upgrade, Opts2, OptsVsn2} ->
?MODULE:acceptor(LSocket, Transport, Protocol,
Opts2, OptsVsn2, ListenerPid, ReqsSup)
end.

View file

@ -1,48 +0,0 @@
%% Copyright (c) 2011-2012, 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.
%% @private
-module(cowboy_acceptors_sup).
-behaviour(supervisor).
-export([start_link/7]). %% API.
-export([init/1]). %% supervisor.
%% API.
-spec start_link(non_neg_integer(), module(), any(),
module(), any(), pid(), pid()) -> {ok, pid()}.
start_link(NbAcceptors, Transport, TransOpts,
Protocol, ProtoOpts, ListenerPid, ReqsPid) ->
supervisor:start_link(?MODULE, [NbAcceptors, Transport, TransOpts,
Protocol, ProtoOpts, ListenerPid, ReqsPid]).
%% supervisor.
-spec init([any()]) -> {'ok', {{'one_for_one', 10, 10}, [{
any(), {atom() | tuple(), atom(), 'undefined' | [any()]},
'permanent' | 'temporary' | 'transient',
'brutal_kill' | 'infinity' | non_neg_integer(),
'supervisor' | 'worker',
'dynamic' | [atom() | tuple()]}]
}}.
init([NbAcceptors, Transport, TransOpts,
Protocol, ProtoOpts, ListenerPid, ReqsPid]) ->
{ok, LSocket} = Transport:listen(TransOpts),
Procs = [{{acceptor, self(), N}, {cowboy_acceptor, start_link, [
LSocket, Transport, Protocol, ProtoOpts,
ListenerPid, ReqsPid
]}, permanent, brutal_kill, worker, []}
|| N <- lists:seq(1, NbAcceptors)],
{ok, {{one_for_one, 10, 10}, Procs}}.

View file

@ -16,7 +16,7 @@
-module(cowboy_app).
-behaviour(application).
-export([start/2, stop/1, profile_output/0]). %% API.
-export([start/2, stop/1]). %% API.
-type application_start_type() :: normal
| {takeover, node()} | {failover, node()}.
@ -25,29 +25,8 @@
-spec start(application_start_type(), any()) -> {ok, pid()}.
start(_Type, _Args) ->
consider_profiling(),
cowboy_sup:start_link().
-spec stop(any()) -> ok.
stop(_State) ->
ok.
-spec profile_output() -> ok.
profile_output() ->
eprof:stop_profiling(),
eprof:log("procs.profile"),
eprof:analyze(procs),
eprof:log("total.profile"),
eprof:analyze(total).
%% Internal.
-spec consider_profiling() -> profiling | not_profiling.
consider_profiling() ->
case application:get_env(profile) of
{ok, true} ->
{ok, _Pid} = eprof:start(),
eprof:start_profiling([self()]);
_ ->
not_profiling
end.

View file

@ -108,11 +108,11 @@ request(Method, URL, Headers, Body, Client=#client{
raw_request(Data, Client2).
parse_url(<< "https://", Rest/binary >>) ->
parse_url(Rest, cowboy_ssl_transport);
parse_url(Rest, ranch_ssl);
parse_url(<< "http://", Rest/binary >>) ->
parse_url(Rest, cowboy_tcp_transport);
parse_url(Rest, ranch_tcp);
parse_url(URL) ->
parse_url(URL, cowboy_tcp_transport).
parse_url(URL, ranch_tcp).
parse_url(URL, Transport) ->
case binary:split(URL, <<"/">>) of
@ -126,9 +126,9 @@ parse_url(URL, Transport) ->
parse_peer(Peer, Transport) ->
case binary:split(Peer, <<":">>) of
[Host] when Transport =:= cowboy_tcp_transport ->
[Host] when Transport =:= ranch_tcp ->
{binary_to_list(Host), 80};
[Host] when Transport =:= cowboy_ssl_transport ->
[Host] when Transport =:= ranch_ssl ->
{binary_to_list(Host), 443};
[Host, Port] ->
{binary_to_list(Host), list_to_integer(binary_to_list(Port))}

View file

@ -33,7 +33,6 @@
%% @see cowboy_dispatcher
%% @see cowboy_http_handler
-module(cowboy_http_protocol).
-behaviour(cowboy_protocol).
-export([start_link/4]). %% API.
-export([init/4, parse_request/1, handler_loop/3]). %% FSM.
@ -85,7 +84,7 @@ init(ListenerPid, Socket, Transport, Opts) ->
Timeout = proplists:get_value(timeout, Opts, 5000),
URLDecDefault = {fun cowboy_http:urldecode/2, crash},
URLDec = proplists:get_value(urldecode, Opts, URLDecDefault),
ok = cowboy:accept_ack(ListenerPid),
ok = ranch:accept_ack(ListenerPid),
wait_request(#state{listener=ListenerPid, socket=Socket, transport=Transport,
dispatch=Dispatch, max_empty_lines=MaxEmptyLines,
max_keepalive=MaxKeepalive, max_line_length=MaxLineLength,

View file

@ -326,13 +326,14 @@ file_contents(Req, #state{filepath=Filepath,
-spec content_function(module(), inet:socket(), binary()) ->
fun(() -> {sent, non_neg_integer()}).
content_function(Transport, Socket, Filepath) ->
%% `file:sendfile/2' will only work with the `cowboy_tcp_transport'
%% `file:sendfile/2' will only work with the `ranch_tcp'
%% transport module. SSL or future SPDY transports that require the
%% content to be encrypted or framed as the content is sent.
%% content to be encrypted or framed as the content is sent
%% will use the fallback mechanism.
case erlang:function_exported(file, sendfile, 2) of
false ->
fun() -> sfallback(Transport, Socket, Filepath) end;
_ when Transport =/= cowboy_tcp_transport ->
_ when Transport =/= ranch_tcp ->
fun() -> sfallback(Transport, Socket, Filepath) end;
true ->
fun() -> sendfile(Socket, Filepath) end

View file

@ -57,7 +57,7 @@
%% in your <em>cowboy_http_handler:init/3</em> handler function.
-spec upgrade(pid(), module(), any(), #http_req{}) -> closed.
upgrade(ListenerPid, Handler, Opts, Req) ->
cowboy_listener:move_connection(ListenerPid, websocket, self()),
ranch_listener:remove_connection(ListenerPid),
case catch websocket_upgrade(#state{handler=Handler, opts=Opts}, Req) of
{ok, State, Req2} -> handler_init(State, Req2);
{'EXIT', _Reason} -> upgrade_error(Req)

View file

@ -1,224 +0,0 @@
%% Copyright (c) 2011-2012, 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 Public API for managing listeners.
-module(cowboy_listener).
-behaviour(gen_server).
-export([start_link/2, stop/1,
add_connection/4, move_connection/3, remove_connection/2, check_upgrades/2,
get_protocol_options/1, set_protocol_options/2]). %% API.
-export([init/1, handle_call/3, handle_cast/2,
handle_info/2, terminate/2, code_change/3]). %% gen_server.
-type pools() :: [{atom(), non_neg_integer()}].
-record(state, {
req_pools = [] :: pools(),
reqs_table :: ets:tid(),
queue = undefined :: queue(),
max_conns = undefined :: non_neg_integer(),
proto_opts :: any(),
proto_opts_vsn = 1 :: non_neg_integer()
}).
%% API.
%% @private
%%
%% We set the process priority to high because cowboy_listener is the central
%% gen_server in Cowboy and is used to manage all the incoming connections.
%% Setting the process priority to high ensures the connection-related code
%% will always be executed when a connection needs it, allowing Cowboy to
%% scale far beyond what it would with a normal priority.
-spec start_link(non_neg_integer(), any()) -> {ok, pid()}.
start_link(MaxConns, ProtoOpts) ->
gen_server:start_link(?MODULE, [MaxConns, ProtoOpts],
[{spawn_opt, [{priority, high}]}]).
%% @private
-spec stop(pid()) -> stopped.
stop(ServerPid) ->
gen_server:call(ServerPid, stop).
%% @doc Add a connection to the given pool in the listener.
%%
%% Pools of connections are used to restrict the maximum number of connections
%% depending on their type. By default, Cowboy add all connections to the
%% pool <em>default</em>. It also checks for the maximum number of connections
%% in that pool before accepting again. This function only returns when there
%% is free space in the pool.
%%
%% When a process managing a connection dies, the process is removed from the
%% pool. If the socket has been sent to another process, it is up to the
%% protocol code to inform the listener of the new <em>ConnPid</em> by removing
%% the previous and adding the new one.
%%
%% This function also returns whether the protocol options have been modified.
%% If so, then an {upgrade, ProtoOpts, OptsVsn} will be returned instead of
%% the atom 'ok'. The acceptor can then continue with the new protocol options.
-spec add_connection(pid(), atom(), pid(), non_neg_integer())
-> ok | {upgrade, any(), non_neg_integer()}.
add_connection(ServerPid, Pool, ConnPid, OptsVsn) ->
gen_server:call(ServerPid, {add_connection, Pool, ConnPid, OptsVsn},
infinity).
%% @doc Move a connection from one pool to another.
-spec move_connection(pid(), atom(), pid()) -> ok.
move_connection(ServerPid, DestPool, ConnPid) ->
gen_server:cast(ServerPid, {move_connection, DestPool, ConnPid}).
%% @doc Remove the given connection from its pool.
-spec remove_connection(pid(), pid()) -> ok.
remove_connection(ServerPid, ConnPid) ->
gen_server:cast(ServerPid, {remove_connection, ConnPid}).
%% @doc Return whether a protocol upgrade is required.
-spec check_upgrades(pid(), non_neg_integer())
-> ok | {upgrade, any(), non_neg_integer()}.
check_upgrades(ServerPid, OptsVsn) ->
gen_server:call(ServerPid, {check_upgrades, OptsVsn}).
%% @doc Return the current protocol options.
-spec get_protocol_options(pid()) -> {ok, any()}.
get_protocol_options(ServerPid) ->
gen_server:call(ServerPid, get_protocol_options).
%% @doc Upgrade the protocol options.
-spec set_protocol_options(pid(), any()) -> ok.
set_protocol_options(ServerPid, ProtoOpts) ->
gen_server:call(ServerPid, {set_protocol_options, ProtoOpts}).
%% gen_server.
%% @private
-spec init(list()) -> {ok, #state{}}.
init([MaxConns, ProtoOpts]) ->
ReqsTable = ets:new(requests_table, [set, private]),
Queue = queue:new(),
{ok, #state{reqs_table=ReqsTable, max_conns=MaxConns,
proto_opts=ProtoOpts, queue=Queue}}.
%% @private
-spec handle_call(_, _, State)
-> {reply, ignored, State} | {stop, normal, stopped, State}.
handle_call({add_connection, Pool, ConnPid, AccOptsVsn}, From, State=#state{
req_pools=Pools, reqs_table=ReqsTable,
queue=Queue, max_conns=MaxConns,
proto_opts=ProtoOpts, proto_opts_vsn=LisOptsVsn}) ->
{NbConns, Pools2} = add_pid(ConnPid, Pool, Pools, ReqsTable),
State2 = State#state{req_pools=Pools2},
if AccOptsVsn =/= LisOptsVsn ->
{reply, {upgrade, ProtoOpts, LisOptsVsn}, State2};
NbConns > MaxConns ->
Queue2 = queue:in(From, Queue),
{noreply, State2#state{queue=Queue2}};
true ->
{reply, ok, State2}
end;
handle_call({check_upgrades, AccOptsVsn}, _From, State=#state{
proto_opts=ProtoOpts, proto_opts_vsn=LisOptsVsn}) ->
if AccOptsVsn =/= LisOptsVsn ->
{reply, {upgrade, ProtoOpts, LisOptsVsn}, State};
true ->
{reply, ok, State}
end;
handle_call(get_protocol_options, _From, State=#state{proto_opts=ProtoOpts}) ->
{reply, {ok, ProtoOpts}, State};
handle_call({set_protocol_options, ProtoOpts}, _From,
State=#state{proto_opts_vsn=OptsVsn}) ->
{reply, ok, State#state{proto_opts=ProtoOpts, proto_opts_vsn=OptsVsn + 1}};
handle_call(stop, _From, State) ->
{stop, normal, stopped, State};
handle_call(_Request, _From, State) ->
{reply, ignored, State}.
%% @private
-spec handle_cast(_, State) -> {noreply, State}.
handle_cast({move_connection, DestPool, ConnPid}, State=#state{
req_pools=Pools, reqs_table=ReqsTable}) ->
Pools2 = move_pid(ConnPid, DestPool, Pools, ReqsTable),
{noreply, State#state{req_pools=Pools2}};
handle_cast({remove_connection, ConnPid}, State=#state{
req_pools=Pools, reqs_table=ReqsTable, queue=Queue}) ->
{Pools2, Queue2} = remove_pid(ConnPid, Pools, ReqsTable, Queue),
{noreply, State#state{req_pools=Pools2, queue=Queue2}};
handle_cast(_Msg, State) ->
{noreply, State}.
%% @private
-spec handle_info(_, State) -> {noreply, State}.
handle_info({'DOWN', _Ref, process, Pid, _Info}, State=#state{
req_pools=Pools, reqs_table=ReqsTable, queue=Queue}) ->
{Pools2, Queue2} = remove_pid(Pid, Pools, ReqsTable, Queue),
{noreply, State#state{req_pools=Pools2, queue=Queue2}};
handle_info(_Info, State) ->
{noreply, State}.
%% @private
-spec terminate(_, _) -> ok.
terminate(_Reason, _State) ->
ok.
%% @private
-spec code_change(_, State, _) -> {ok, State}.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%% Internal.
%% @private
-spec add_pid(pid(), atom(), pools(), ets:tid())
-> {non_neg_integer(), pools()}.
add_pid(ConnPid, Pool, Pools, ReqsTable) ->
MonitorRef = erlang:monitor(process, ConnPid),
ConnPid ! {shoot, self()},
{NbConnsRet, Pools2} = case lists:keyfind(Pool, 1, Pools) of
false ->
{1, [{Pool, 1}|Pools]};
{Pool, NbConns} ->
NbConns2 = NbConns + 1,
{NbConns2, [{Pool, NbConns2}|lists:keydelete(Pool, 1, Pools)]}
end,
ets:insert(ReqsTable, {ConnPid, {MonitorRef, Pool}}),
{NbConnsRet, Pools2}.
%% @private
-spec move_pid(pid(), atom(), pools(), ets:tid()) -> pools().
move_pid(ConnPid, DestPool, Pools, ReqsTable) ->
{MonitorRef, SrcPool} = ets:lookup_element(ReqsTable, ConnPid, 2),
ets:insert(ReqsTable, {ConnPid, {MonitorRef, DestPool}}),
{SrcPool, SrcNbConns} = lists:keyfind(SrcPool, 1, Pools),
DestNbConns = case lists:keyfind(DestPool, 1, Pools) of
false -> 1;
{DestPool, NbConns} -> NbConns + 1
end,
Pools2 = lists:keydelete(SrcPool, 1, lists:keydelete(DestPool, 1, Pools)),
[{SrcPool, SrcNbConns - 1}, {DestPool, DestNbConns}|Pools2].
%% @private
-spec remove_pid(pid(), pools(), ets:tid(), queue()) -> {pools(), queue()}.
remove_pid(Pid, Pools, ReqsTable, Queue) ->
{MonitorRef, Pool} = ets:lookup_element(ReqsTable, Pid, 2),
erlang:demonitor(MonitorRef, [flush]),
{Pool, NbConns} = lists:keyfind(Pool, 1, Pools),
Pools2 = [{Pool, NbConns - 1}|lists:keydelete(Pool, 1, Pools)],
ets:delete(ReqsTable, Pid),
case queue:out(Queue) of
{{value, Client}, Queue2} ->
gen_server:reply(Client, ok),
{Pools2, Queue2};
_ ->
{Pools2, Queue}
end.

View file

@ -1,46 +0,0 @@
%% Copyright (c) 2011-2012, 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.
%% @private
-module(cowboy_listener_sup).
-behaviour(supervisor).
-export([start_link/5]). %% API.
-export([init/1]). %% supervisor.
%% API.
-spec start_link(non_neg_integer(), module(), any(), module(), any())
-> {ok, pid()}.
start_link(NbAcceptors, Transport, TransOpts, Protocol, ProtoOpts) ->
MaxConns = proplists:get_value(max_connections, TransOpts, 1024),
{ok, SupPid} = supervisor:start_link(?MODULE, []),
{ok, ListenerPid} = supervisor:start_child(SupPid,
{cowboy_listener, {cowboy_listener, start_link, [MaxConns, ProtoOpts]},
permanent, 5000, worker, [cowboy_listener]}),
{ok, ReqsPid} = supervisor:start_child(SupPid,
{cowboy_requests_sup, {cowboy_requests_sup, start_link, []},
permanent, 5000, supervisor, [cowboy_requests_sup]}),
{ok, _PoolPid} = supervisor:start_child(SupPid,
{cowboy_acceptors_sup, {cowboy_acceptors_sup, start_link, [
NbAcceptors, Transport, TransOpts,
Protocol, ProtoOpts, ListenerPid, ReqsPid
]}, permanent, 5000, supervisor, [cowboy_acceptors_sup]}),
{ok, SupPid}.
%% supervisor.
-spec init([]) -> {ok, {{one_for_all, 10, 10}, []}}.
init([]) ->
{ok, {{one_for_all, 10, 10}, []}}.

View file

@ -1,61 +0,0 @@
%% Copyright (c) 2011-2012, Loïc Hoguin <essen@ninenines.eu>
%% Copyright (c) 2011, Michiel Hakvoort <michiel@hakvoort.it>
%%
%% 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 Cowboy protocol.
%%
%% A Cowboy protocol must implement one callback: <em>start_link/4</em>.
%%
%% <em>start_link/4</em> is meant for the initialization of the
%% protocol process.
%% It receives the pid to the listener's gen_server, the client socket,
%% the module name of the chosen transport and the options defined when
%% starting the listener. The <em>start_link/4</em> function must follow
%% the supervisor start function specification.
%%
%% After initializing your protocol, it is recommended to call the
%% function cowboy:accept_ack/1 with the ListenerPid as argument,
%% as it will ensure Cowboy has been able to fully initialize the socket.
%% Anything you do past this point is up to you!
%%
%% If you need to change some socket options, like enabling raw mode
%% for example, you can call the <em>Transport:setopts/2</em> function.
%% It is the protocol's responsability to manage the socket usage,
%% there should be no need for an user to specify that kind of options
%% while starting a listener.
%%
%% You should definitely look at the cowboy_http_protocol module for
%% a great example of fast request handling if you need to.
%% Otherwise it's probably safe to use <code>{active, once}</code> mode
%% and handle everything as it comes.
%%
%% Note that while you technically can run a protocol handler directly
%% as a gen_server or a gen_fsm, it's probably not a good idea,
%% as the only call you'll ever receive from Cowboy is the
%% <em>start_link/4</em> call. On the other hand, feel free to write
%% a very basic protocol handler which then forwards requests to a
%% gen_server or gen_fsm. By doing so however you must take care to
%% supervise their processes as Cowboy only knows about the protocol
%% handler itself.
-module(cowboy_protocol).
-export([behaviour_info/1]).
%% @private
-spec behaviour_info(_)
-> undefined | [{start_link, 4}, ...].
behaviour_info(callbacks) ->
[{start_link, 4}];
behaviour_info(_Other) ->
undefined.

View file

@ -1,44 +0,0 @@
%% Copyright (c) 2011-2012, 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.
%% @private
-module(cowboy_requests_sup).
-behaviour(supervisor).
-export([start_link/0, start_request/5]). %% API.
-export([init/1]). %% supervisor.
%% API.
-spec start_link() -> {ok, pid()}.
start_link() ->
supervisor:start_link(?MODULE, []).
-spec start_request(pid(), inet:socket(), module(), module(), any())
-> {ok, pid()}.
start_request(ListenerPid, Socket, Transport, Protocol, Opts) ->
Protocol:start_link(ListenerPid, Socket, Transport, Opts).
%% supervisor.
-spec init([]) -> {'ok', {{'simple_one_for_one', 0, 1}, [{
any(), {atom() | tuple(), atom(), 'undefined' | [any()]},
'permanent' | 'temporary' | 'transient',
'brutal_kill' | 'infinity' | non_neg_integer(),
'supervisor' | 'worker',
'dynamic' | [atom() | tuple()]}]
}}.
init([]) ->
{ok, {{simple_one_for_one, 0, 1}, [{?MODULE, {?MODULE, start_request, []},
temporary, brutal_kill, worker, [?MODULE]}]}}.

View file

@ -1,177 +0,0 @@
%% Copyright (c) 2011-2012, 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 SSL transport API.
%%
%% Wrapper around <em>ssl</em> implementing the Cowboy transport API.
%%
%% This transport requires the <em>crypto</em>, <em>public_key</em>
%% and <em>ssl</em> applications to be started. If they aren't started,
%% it will try to start them itself before opening a port to listen.
%% Applications aren't stopped when the listening socket is closed, though.
%%
%% @see ssl
-module(cowboy_ssl_transport).
-export([name/0, messages/0, listen/1, accept/2, recv/3, send/2, setopts/2,
controlling_process/2, peername/1, close/1, sockname/1]).
-export([connect/3]).
%% @doc Name of this transport API, <em>ssl</em>.
-spec name() -> ssl.
name() -> ssl.
%% @doc Atoms used in the process messages sent by this API.
%%
%% They identify incoming data, closed connection and errors when receiving
%% data in active mode.
-spec messages() -> {ssl, ssl_closed, ssl_error}.
messages() -> {ssl, ssl_closed, ssl_error}.
%% @private
%% @todo Probably filter Opts?
connect(Host, Port, Opts) when is_list(Host), is_integer(Port) ->
ssl:connect(Host, Port,
Opts ++ [binary, {active, false}, {packet, raw}]).
%% @doc Setup a socket to listen on the given port on the local host.
%%
%% The available options are:
%% <dl>
%% <dt>port</dt><dd>Mandatory. TCP port number to open.</dd>
%% <dt>backlog</dt><dd>Maximum length of the pending connections queue.
%% Defaults to 1024.</dd>
%% <dt>ip</dt><dd>Interface to listen on. Listen on all interfaces
%% by default.</dd>
%% <dt>certfile</dt><dd>Mandatory. Path to a file containing the user's
%% certificate.</dd>
%% <dt>keyfile</dt><dd>Optional. Path to the file containing the user's
%% private PEM encoded key.</dd>
%% <dt>cacertfile</dt><dd>Optional. Path to file containing PEM encoded
%% CA certificates (trusted certificates used for verifying a peer
%% certificate).</dd>
%% <dt>password</dt><dd>Optional. String containing the user's password.
%% All private keyfiles must be password protected currently.</dd>
%% <dt>ciphers</dt><dd>Optional. The cipher suites that should be supported.
%% The function ssl:cipher_suites/0 can be used to find all available
%% ciphers.</dd>
%% </dl>
%%
%% @see ssl:listen/2
-spec listen([{port, inet:port_number()} | {certfile, string()}
| {keyfile, string()} | {password, string()}
| {cacertfile, string()} | {ip, inet:ip_address()}])
-> {ok, ssl:sslsocket()} | {error, atom()}.
listen(Opts) ->
require([crypto, public_key, ssl]),
{port, Port} = lists:keyfind(port, 1, Opts),
Backlog = proplists:get_value(backlog, Opts, 1024),
{certfile, CertFile} = lists:keyfind(certfile, 1, Opts),
ListenOpts0 = [binary, {active, false},
{backlog, Backlog}, {packet, raw}, {reuseaddr, true},
{certfile, CertFile}],
ListenOpts = lists:foldl(fun
({ip, _} = Ip, Acc) -> [Ip | Acc];
({keyfile, _} = KeyFile, Acc) -> [KeyFile | Acc];
({cacertfile, _} = CACertFile, Acc) -> [CACertFile | Acc];
({password, _} = Password, Acc) -> [Password | Acc];
({ciphers, _} = Ciphers, Acc) -> [Ciphers | Acc];
(_, Acc) -> Acc
end, ListenOpts0, Opts),
ssl:listen(Port, ListenOpts).
%% @doc Accept an incoming connection on a listen socket.
%%
%% Note that this function does both the transport accept and
%% the SSL handshake.
%%
%% @see ssl:transport_accept/2
%% @see ssl:ssl_accept/2
-spec accept(ssl:sslsocket(), timeout())
-> {ok, ssl:sslsocket()} | {error, closed | timeout | atom()}.
accept(LSocket, Timeout) ->
case ssl:transport_accept(LSocket, Timeout) of
{ok, CSocket} ->
ssl_accept(CSocket, Timeout);
{error, Reason} ->
{error, Reason}
end.
%% @doc Receive a packet from a socket in passive mode.
%% @see ssl:recv/3
-spec recv(ssl:sslsocket(), non_neg_integer(), timeout())
-> {ok, any()} | {error, closed | atom()}.
recv(Socket, Length, Timeout) ->
ssl:recv(Socket, Length, Timeout).
%% @doc Send a packet on a socket.
%% @see ssl:send/2
-spec send(ssl:sslsocket(), iolist()) -> ok | {error, atom()}.
send(Socket, Packet) ->
ssl:send(Socket, Packet).
%% @doc Set one or more options for a socket.
%% @see ssl:setopts/2
-spec setopts(ssl:sslsocket(), list()) -> ok | {error, atom()}.
setopts(Socket, Opts) ->
ssl:setopts(Socket, Opts).
%% @doc Assign a new controlling process <em>Pid</em> to <em>Socket</em>.
%% @see ssl:controlling_process/2
-spec controlling_process(ssl:sslsocket(), pid())
-> ok | {error, closed | not_owner | atom()}.
controlling_process(Socket, Pid) ->
ssl:controlling_process(Socket, Pid).
%% @doc Return the address and port for the other end of a connection.
%% @see ssl:peername/1
-spec peername(ssl:sslsocket())
-> {ok, {inet:ip_address(), inet:port_number()}} | {error, atom()}.
peername(Socket) ->
ssl:peername(Socket).
%% @doc Close a TCP socket.
%% @see ssl:close/1
-spec close(ssl:sslsocket()) -> ok.
close(Socket) ->
ssl:close(Socket).
%% @doc Get the local address and port of a socket
%% @see ssl:sockname/1
-spec sockname(ssl:sslsocket())
-> {ok, {inet:ip_address(), inet:port_number()}} | {error, atom()}.
sockname(Socket) ->
ssl:sockname(Socket).
%% Internal.
-spec require(list(module())) -> ok.
require([]) ->
ok;
require([App|Tail]) ->
case application:start(App) of
ok -> ok;
{error, {already_started, App}} -> ok
end,
require(Tail).
-spec ssl_accept(ssl:sslsocket(), timeout())
-> {ok, ssl:sslsocket()} | {error, closed | timeout | atom()}.
ssl_accept(Socket, Timeout) ->
case ssl:ssl_accept(Socket, Timeout) of
ok ->
{ok, Socket};
{error, Reason} ->
{error, Reason}
end.

View file

@ -1,119 +0,0 @@
%% Copyright (c) 2011-2012, 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 TCP transport API.
%%
%% Wrapper around <em>gen_tcp</em> implementing the Cowboy transport API.
%%
%% @see gen_tcp
-module(cowboy_tcp_transport).
-export([name/0, messages/0, listen/1, accept/2, recv/3, send/2, setopts/2,
controlling_process/2, peername/1, close/1, sockname/1]).
-export([connect/3]).
%% @doc Name of this transport API, <em>tcp</em>.
-spec name() -> tcp.
name() -> tcp.
%% @doc Atoms used in the process messages sent by this API.
%%
%% They identify incoming data, closed connection and errors when receiving
%% data in active mode.
-spec messages() -> {tcp, tcp_closed, tcp_error}.
messages() -> {tcp, tcp_closed, tcp_error}.
%% @private
connect(Host, Port, Opts) when is_list(Host), is_integer(Port) ->
gen_tcp:connect(Host, Port,
Opts ++ [binary, {active, false}, {packet, raw}]).
%% @doc Setup a socket to listen on the given port on the local host.
%%
%% The available options are:
%% <dl>
%% <dt>port</dt><dd>Mandatory. TCP port number to open.</dd>
%% <dt>backlog</dt><dd>Maximum length of the pending connections queue.
%% Defaults to 1024.</dd>
%% <dt>ip</dt><dd>Interface to listen on. Listen on all interfaces
%% by default.</dd>
%% </dl>
%%
%% @see gen_tcp:listen/2
-spec listen([{port, inet:port_number()} | {ip, inet:ip_address()}])
-> {ok, inet:socket()} | {error, atom()}.
listen(Opts) ->
{port, Port} = lists:keyfind(port, 1, Opts),
Backlog = proplists:get_value(backlog, Opts, 1024),
ListenOpts0 = [binary, {active, false},
{backlog, Backlog}, {packet, raw}, {reuseaddr, true}],
ListenOpts =
case lists:keyfind(ip, 1, Opts) of
false -> ListenOpts0;
Ip -> [Ip|ListenOpts0]
end,
gen_tcp:listen(Port, ListenOpts).
%% @doc Accept an incoming connection on a listen socket.
%% @see gen_tcp:accept/2
-spec accept(inet:socket(), timeout())
-> {ok, inet:socket()} | {error, closed | timeout | atom()}.
accept(LSocket, Timeout) ->
gen_tcp:accept(LSocket, Timeout).
%% @doc Receive a packet from a socket in passive mode.
%% @see gen_tcp:recv/3
-spec recv(inet:socket(), non_neg_integer(), timeout())
-> {ok, any()} | {error, closed | atom()}.
recv(Socket, Length, Timeout) ->
gen_tcp:recv(Socket, Length, Timeout).
%% @doc Send a packet on a socket.
%% @see gen_tcp:send/2
-spec send(inet:socket(), iolist()) -> ok | {error, atom()}.
send(Socket, Packet) ->
gen_tcp:send(Socket, Packet).
%% @doc Set one or more options for a socket.
%% @see inet:setopts/2
-spec setopts(inet:socket(), list()) -> ok | {error, atom()}.
setopts(Socket, Opts) ->
inet:setopts(Socket, Opts).
%% @doc Assign a new controlling process <em>Pid</em> to <em>Socket</em>.
%% @see gen_tcp:controlling_process/2
-spec controlling_process(inet:socket(), pid())
-> ok | {error, closed | not_owner | atom()}.
controlling_process(Socket, Pid) ->
gen_tcp:controlling_process(Socket, Pid).
%% @doc Return the address and port for the other end of a connection.
%% @see inet:peername/1
-spec peername(inet:socket())
-> {ok, {inet:ip_address(), inet:port_number()}} | {error, atom()}.
peername(Socket) ->
inet:peername(Socket).
%% @doc Close a TCP socket.
%% @see gen_tcp:close/1
-spec close(inet:socket()) -> ok.
close(Socket) ->
gen_tcp:close(Socket).
%% @doc Get the local address and port of a socket
%% @see inet:sockname/1
-spec sockname(inet:socket())
-> {ok, {inet:ip_address(), inet:port_number()}} | {error, atom()}.
sockname(Socket) ->
inet:sockname(Socket).