Use Cowboy instead of Webmachine/Mochiweb for Web plugins

As part of a push to reduce the number of dependencies from the
RabbitMQ project, the management plugin needed to be converted
from Webmachine to Cowboy REST. This plugin provides a convenient
interface to listeners, which had to be converted from Mochiweb's
to Cowboy's also.

Because Cowboy does not have any logging built-in we keep
Webmachine's access logger for now. It has been adapted to use
data coming from Cowboy, as opposed to Mochiweb. The event handler
for logging has also been moved to web dispatch.

The main breaking change in the interface is that contexts do not
take a fun for the main loop anymore, but instead take a dispatch
list of the routes to be served by that context. Routes must be
compiled using cowboy_router:compile(Routes), and will be served
with the context prefix appended.

Some contexts do not require any code change, including the static
contexts and port redirect contexts.

To serve many applications, we use a Cowboy middleware specific
to RabbitMQ which, when receiving a request, checks in the registry
if there is a matching application, and use its dispatch list for
any subsequent operations.

(cherry picked from commit b8b8c06dd2af6034a47372c4b53215d54017c546)
This commit is contained in:
Loïc Hoguin 2015-10-02 16:05:33 +02:00
parent 163d3d96b3
commit 8368cfa701
15 changed files with 547 additions and 247 deletions

View File

@ -1,6 +1,8 @@
PROJECT = rabbitmq_web_dispatch
DEPS = mochiweb webmachine
DEPS = cowboy
dep_cowboy_commit = 1.0.3
TEST_DEPS = rabbit amqp_client
DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk

View File

@ -1,25 +1,16 @@
rabbitmq-web-dispatch
---------------------
rabbitmq-web-dispatch is a thin veneer around mochiweb that provides the
ability for multiple applications to co-exist on mochiweb
rabbitmq-web-dispatch is a thin veneer around Cowboy that provides the
ability for multiple applications to co-exist on Cowboy
listeners. Applications can register static docroots or dynamic
handlers to be executed, dispatched by URL path prefix.
See http://www.rabbitmq.com/mochiweb.html for information on
See http://www.rabbitmq.com/web-dispatch.html for information on
configuring web plugins.
The most general registration procedure is
`rabbit_web_dispatch:register_context_handler/5`. This takes a callback
procedure of the form
`rabbit_web_dispatch:register_context_handler/5`.
loop(Request) ->
...
The module `rabbit_webmachine` provides a means of running more than
one webmachine in a VM, and understands rabbitmq-web-dispatch contexts. To
use it, supply a dispatch table term of the kind usually given to
webmachine in the file `priv/dispatch.conf`.
`setup/{1,2}` in the same module allows some global configuration of
webmachine logging and error handling.
This takes a dispatch list of the kind usually given to Cowboy, in compiled
form.

View File

@ -0,0 +1,52 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2010-2015 GoPivotal, Inc. All rights reserved.
%%
-module(rabbit_cowboy_middleware).
-behavior(cowboy_middleware).
-export([execute/2]).
-export([onresponse/4]).
execute(Req, Env) ->
%% Pre-parse the query string.
{_, Req1} = cowboy_req:qs_vals(Req),
%% Find the correct dispatch list for this path.
{_, Listener} = lists:keyfind(rabbit_listener, 1, Env),
case rabbit_web_dispatch_registry:lookup(Listener, Req1) of
{ok, Dispatch} ->
{ok, Req1, [{dispatch, Dispatch}|Env]};
{error, Reason} ->
{ok, Req2} = cowboy_req:reply(500,
[{<<"content-type">>, <<"text/plain">>}],
"Registry Error: " ++ io_lib:format("~p", [Reason]), Req1),
{halt, Req2}
end.
onresponse(Status = 404, Headers0, Body = <<>>, Req0) ->
log_access(Status, Body, Req0),
Headers = [{<<"content-type">>, <<"application/json">>}|Headers0],
Json = {struct,
[{error, list_to_binary(httpd_util:reason_phrase(Status))},
{reason, <<"Not Found">>}]},
{ok, Req} = cowboy_req:reply(Status, Headers, mochijson2:encode(Json), Req0),
Req;
onresponse(Status, _, Body, Req) ->
log_access(Status, Body, Req),
Req.
log_access(Status, Body, Req) ->
webmachine_log:log_access({Status, Body, Req}).

View File

@ -0,0 +1,34 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2010-2015 GoPivotal, Inc. All rights reserved.
%%
-module(rabbit_cowboy_redirect).
-export([init/3]).
-export([handle/2]).
-export([terminate/3]).
init(_, Req, RedirectPort) ->
{ok, Req, RedirectPort}.
handle(Req0, RedirectPort) ->
%% Use a small trick to get a URL with the updated port.
RedReq = cowboy_req:set([{port, RedirectPort}], Req0),
{URL, _} = cowboy_req:url(RedReq),
{ok, Req} = cowboy_req:reply(301, [{<<"location">>, URL}], Req0),
{ok, Req, RedirectPort}.
terminate(_, _, _) ->
ok.

View File

@ -50,27 +50,16 @@ register_static_context(Name, Listener, Prefix, Module, FSPath, LinkText) ->
register_port_redirect(Name, Listener, Prefix, RedirectPort) ->
register_context_handler(
Name, Listener, Prefix,
fun (Req) ->
Host = case Req:get_header_value("host") of
undefined -> {ok, {IP, _Port}} = rabbit_net:sockname(
Req:get(socket)),
rabbit_misc:ntoa(IP);
Header -> hd(string:tokens(Header, ":"))
end,
URL = rabbit_misc:format(
"~s://~s:~B~s",
[Req:get(scheme), Host, RedirectPort, Req:get(raw_path)]),
Req:respond({301, [{"Location", URL}], ""})
end,
cowboy_router:compile([{'_', [{'_', rabbit_cowboy_redirect, RedirectPort}]}]),
rabbit_misc:format("Redirect to port ~B", [RedirectPort])).
context_selector("") ->
fun(_Req) -> true end;
context_selector(Prefix) ->
Prefix1 = "/" ++ Prefix,
Prefix1 = list_to_binary("/" ++ Prefix),
fun(Req) ->
Path = Req:get(raw_path),
(Path == Prefix1) orelse (string:str(Path, Prefix1 ++ "/") == 1)
{Path, _} = cowboy_req:path(Req),
(Path == Prefix1) orelse (binary:match(Path, << Prefix1/binary, $/ >>) =/= nomatch)
end.
%% Produces a handler for use with register_handler that serves up
@ -81,32 +70,10 @@ static_context_handler(Prefix, Module, FSPath) ->
{file, Here} = code:is_loaded(Module),
ModuleRoot = filename:dirname(filename:dirname(Here)),
LocalPath = filename:join(ModuleRoot, FSPath),
static_context_handler(Prefix, LocalPath).
%% Produces a handler for use with register_handler that serves up
%% static content from a specified directory.
static_context_handler("", LocalPath) ->
fun(Req) ->
"/" ++ Path = Req:get(path),
serve_file(Req, Path, LocalPath)
end;
static_context_handler(Prefix, LocalPath) ->
fun(Req) ->
"/" ++ Path = Req:get(path),
case string:substr(Path, length(Prefix) + 1) of
"" -> Req:respond({301, [{"Location", "/" ++ Prefix ++ "/"}], ""});
"/" ++ P -> serve_file(Req, P, LocalPath)
end
end.
serve_file(Req, Path, LocalPath) ->
case Req:get(method) of
Method when Method =:= 'GET'; Method =:= 'HEAD' ->
Req:serve_file(Path, LocalPath);
_ ->
Req:respond({405, [{"Allow", "GET, HEAD"}],
"Only GET or HEAD supported for static content"})
end.
cowboy_router:compile([{'_', [
{"/" ++ Prefix, cowboy_static, {file, LocalPath ++ "/index.html"}},
{"/" ++ Prefix ++ "/[...]", cowboy_static, {dir, LocalPath}}
]}]).
%% The opposite of all those register_* functions.
unregister_context(Name) ->

View File

@ -0,0 +1,44 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2010-2014 GoPivotal, Inc. All rights reserved.
%%
-module(rabbit_web_dispatch_listing_handler).
-export([init/3]).
-export([handle/2]).
-export([terminate/3]).
init(_, Req, Listener) ->
{ok, Req, Listener}.
handle(Req0, Listener) ->
HTMLPrefix =
"<html xmlns=\"http://www.w3.org/1999/xhtml\" xml:lang=\"en\">"
"<head><title>RabbitMQ Web Server</title></head>"
"<body><h1>RabbitMQ Web Server</h1><p>Contexts available:</p><ul>",
HTMLSuffix = "</ul></body></html>",
List =
case rabbit_web_dispatch_registry:list(Listener) of
[] ->
"<li>No contexts installed</li>";
Contexts ->
[["<li><a href=\"/", Path, "/\">", Desc, "</a></li>"]
|| {Path, Desc} <- Contexts]
end,
{ok, Req} = cowboy_req:reply(200, [], [HTMLPrefix, List, HTMLSuffix], Req0),
{ok, Req, Listener}.
terminate(_, _, _) ->
ok.

View File

@ -22,6 +22,7 @@
-export([add/5, remove/1, set_fallback/2, lookup/2, list_all/0]).
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
code_change/3]).
-export([list/1]).
-define(ETS, rabbitmq_web_dispatch).
@ -38,6 +39,8 @@ add(Name, Listener, Selector, Handler, Link) ->
remove(Name) ->
gen_server:call(?MODULE, {remove, Name}, infinity).
%% @todo This needs to be dispatch instead of a fun too.
%% But I'm not sure what code is using this.
set_fallback(Listener, FallbackHandler) ->
gen_server:call(?MODULE, {set_fallback, Listener, FallbackHandler},
infinity).
@ -46,9 +49,9 @@ lookup(Listener, Req) ->
case lookup_dispatch(Listener) of
{ok, {Selectors, Fallback}} ->
case catch match_request(Selectors, Req) of
{'EXIT', Reason} -> {lookup_failure, Reason};
no_handler -> {handler, Fallback};
Handler -> {handler, Handler}
{'EXIT', Reason} -> {error, {lookup_failure, Reason}};
not_found -> {ok, Fallback};
Dispatch -> {ok, Dispatch}
end;
Err ->
Err
@ -147,10 +150,10 @@ set_dispatch(Listener, Selectors, Fallback) ->
ets:insert(?ETS, {port(Listener), Listener, Selectors, Fallback}).
match_request([], _) ->
no_handler;
match_request([{_Name, Selector, Handler, _Link}|Rest], Req) ->
not_found;
match_request([{_Name, Selector, Dispatch, _Link}|Rest], Req) ->
case Selector(Req) of
true -> Handler;
true -> Dispatch;
false -> match_request(Rest, Req)
end.
@ -175,25 +178,6 @@ list(Listener) ->
%%---------------------------------------------------------------------------
listing_fallback_handler(Listener) ->
fun(Req) ->
HTMLPrefix =
"<html xmlns=\"http://www.w3.org/1999/xhtml\" xml:lang=\"en\">"
"<head><title>RabbitMQ Web Server</title></head>"
"<body><h1>RabbitMQ Web Server</h1><p>Contexts available:</p><ul>",
HTMLSuffix = "</ul></body></html>",
{ReqPath, _, _} = mochiweb_util:urlsplit_path(Req:get(raw_path)),
List =
case list(Listener) of
[] ->
"<li>No contexts installed</li>";
Contexts ->
[handler_listing(Path, ReqPath, Desc)
|| {Path, Desc} <- Contexts]
end,
Req:respond({200, [], HTMLPrefix ++ List ++ HTMLSuffix})
end.
handler_listing(Path, ReqPath, Desc) ->
io_lib:format(
"<li><a href=\"~s\">~s</a></li>",
[rabbit_web_dispatch_util:relativise(ReqPath, "/" ++ Path), Desc]).
cowboy_router:compile([{'_', [
{"/", rabbit_web_dispatch_listing_handler, Listener}
]}]).

View File

@ -36,9 +36,14 @@ ensure_listener(Listener) ->
undefined ->
{error, {no_port_given, Listener}};
_ ->
Child = {{rabbit_web_dispatch_web, name(Listener)},
{mochiweb_http, start, [mochi_options(Listener)]},
transient, 5000, worker, dynamic},
{Transport, TransportOpts} = preprocess_config(Listener),
Child = ranch:child_spec(name(Listener), 100,
Transport, TransportOpts,
cowboy_protocol, [
{env, [{rabbit_listener, Listener}]},
{middlewares, [rabbit_cowboy_middleware, cowboy_router, cowboy_handler]},
{onresponse, fun rabbit_cowboy_middleware:onresponse/4}
]),
case supervisor:start_child(?SUP, Child) of
{ok, _} -> new;
{error, {already_started, _}} -> existing;
@ -48,8 +53,8 @@ ensure_listener(Listener) ->
stop_listener(Listener) ->
Name = name(Listener),
ok = supervisor:terminate_child(?SUP, {rabbit_web_dispatch_web, Name}),
ok = supervisor:delete_child(?SUP, {rabbit_web_dispatch_web, Name}).
ok = supervisor:terminate_child(?SUP, {ranch_listener_sup, Name}),
ok = supervisor:delete_child(?SUP, {ranch_listener_sup, Name}).
%% @spec init([[instance()]]) -> SupervisorTree
%% @doc supervisor callback.
@ -57,43 +62,28 @@ init([]) ->
Registry = {rabbit_web_dispatch_registry,
{rabbit_web_dispatch_registry, start_link, []},
transient, 5000, worker, dynamic},
{ok, {{one_for_one, 10, 10}, [Registry]}}.
Log = {rabbit_mgmt_access_logger, {gen_event, start_link,
[{local, webmachine_log_event}]},
permanent, 5000, worker, [dynamic]},
{ok, {{one_for_one, 10, 10}, [Registry, Log]}}.
%% ----------------------------------------------------------------------
mochi_options(Listener) ->
[{name, name(Listener)},
{loop, loopfun(Listener)} |
ssl_config(proplists:delete(
name, proplists:delete(ignore_in_use, Listener)))].
loopfun(Listener) ->
fun (Req) ->
case rabbit_web_dispatch_registry:lookup(Listener, Req) of
no_handler ->
Req:not_found();
{error, Reason} ->
Req:respond({500, [], "Registry Error: " ++ Reason});
{handler, Handler} ->
Handler(Req)
end
end.
name(Listener) ->
Port = proplists:get_value(port, Listener),
list_to_atom(atom_to_list(?MODULE) ++ "_" ++ integer_to_list(Port)).
ssl_config(Options) ->
preprocess_config(Options) ->
case proplists:get_value(ssl, Options) of
true -> rabbit_networking:ensure_ssl(),
true -> _ = rabbit_networking:ensure_ssl(),
case rabbit_networking:poodle_check('HTTP') of
ok -> case proplists:get_value(ssl_opts, Options) of
undefined -> auto_ssl(Options);
_ -> fix_ssl(Options)
end;
danger -> proplists:delete(ssl, Options)
danger -> {ranch_tcp, proplists:delete(ssl, Options)}
end;
_ -> Options
_ -> {ranch_tcp, Options}
end.
auto_ssl(Options) ->
@ -105,8 +95,8 @@ auto_ssl(Options) ->
fix_ssl(Options) ->
SSLOpts = proplists:get_value(ssl_opts, Options),
rabbit_misc:pset(ssl_opts,
rabbit_networking:fix_ssl_options(SSLOpts), Options).
{ranch_ssl, proplists:delete(ssl, proplists:delete(ssl_opts,
Options ++ rabbit_networking:fix_ssl_options(SSLOpts)))}.
check_error(Listener, Error) ->
Ignore = proplists:get_value(ignore_in_use, Listener, false),

View File

@ -19,6 +19,7 @@
-export([parse_auth_header/1]).
-export([relativise/2, unrelativise/2]).
%% @todo remove
parse_auth_header(Header) ->
case Header of
"Basic " ++ Base64 ->

View File

@ -1,68 +0,0 @@
%% This file contains an adapted version of webmachine_mochiweb:loop/1
%% from webmachine (revision 0c4b60ac68b4).
%% All modifications are (C) 2011-2013 GoPivotal, Inc.
-module(rabbit_webmachine).
%% An alternative to webmachine_mochiweb, which places the dispatch
%% table (among other things) into the application env, and thereby
%% makes it impossible to run more than one instance of
%% webmachine. Since rabbit_web_dispatch is all about multi-tenanting
%% webapps, clearly this won't do for us.
%% Instead of using webmachine_mochiweb:start/1 or
%% webmachine_mochiweb:loop/1, construct a loop procedure using
%% makeloop/1 and supply it as the argument to
%% rabbit_web_dispatch:register_context_handler or to mochiweb_http:start.
%% We hardwire the "error handler" and use a "logging module" if
%% supplied.
-export([makeloop/1, setup/0]).
setup() ->
application:set_env(
webmachine, error_handler, rabbit_webmachine_error_handler).
makeloop(Dispatch) ->
fun (MochiReq) ->
Req = webmachine:new_request(mochiweb, MochiReq),
{Path, _} = Req:path(),
{ReqData, _} = Req:get_reqdata(),
%% webmachine_mochiweb:loop/1 uses dispatch/4 here;
%% however, we don't need to dispatch by the host name.
case webmachine_dispatcher:dispatch(Path, Dispatch, ReqData) of
{no_dispatch_match, _Host, _PathElements} ->
{ErrorBody, ReqState1} =
rabbit_webmachine_error_handler:render_error(
404, Req, {none, none, []}),
Req1 = {webmachine_request, ReqState1},
{ok, ReqState2} = Req1:append_to_response_body(ErrorBody),
Req2 = {webmachine_request, ReqState2},
{ok, ReqState3} = Req2:send_response(404),
maybe_log_access(ReqState3);
{Mod, ModOpts, HostTokens, Port, PathTokens, Bindings,
AppRoot, StringPath} ->
BootstrapResource = webmachine_resource:new(x,x,x,x),
{ok, Resource} = BootstrapResource:wrap(Mod, ModOpts),
{ok, RS1} = Req:load_dispatch_data(Bindings, HostTokens, Port,
PathTokens,
AppRoot, StringPath),
XReq1 = {webmachine_request, RS1},
{ok, RS2} = XReq1:set_metadata('resource_module', Mod),
try
webmachine_decision_core:handle_request(Resource, RS2)
catch
error:_ ->
FailReq = {webmachine_request, RS2},
{ok, RS3} = FailReq:send_response(500),
maybe_log_access(RS3)
end
end
end.
maybe_log_access(ReqState) ->
Req = {webmachine_request, ReqState},
{LogData, _ReqState1} = Req:log_data(),
webmachine_log:log_access(LogData).

View File

@ -1,63 +0,0 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at http://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2007-2016 Pivotal Software, Inc. All rights reserved.
%%
%% We need to ensure all responses are application/json; anything
%% coming back as text/html could constitute an XSS vector. Also I'm
%% sure it's easier on our clients if they can always expect JSON
%% responses.
%%
%% Based on webmachine_error_handler, but I'm not sure enough remains
%% to be copyrightable.
-module(rabbit_webmachine_error_handler).
-export([render_error/3]).
render_error(Code, Req, Reason) ->
case Req:has_response_body() of
{true, _} ->
maybe_log(Req, Reason),
{Body, ReqState0} = Req:response_body(),
{ok, ReqState} =
webmachine_request:remove_response_header("Content-Encoding",
ReqState0),
{Body, ReqState};
{false, _} -> render_error_body(Code, Req:trim_state(), Reason)
end.
render_error_body(404, Req, _) -> error_body(404, Req, "Not Found");
render_error_body(Code, Req, Reason) -> error_body(Code, Req, Reason).
error_body(Code, Req, Reason) ->
{ok, _ReqState0} = Req:add_response_header("Content-Type","application/json"),
{ok, ReqState} = Req:remove_response_header("Content-Encoding"),
case Code of
500 -> maybe_log(Req, Reason);
_ -> ok
end,
Json = {struct,
[{error, list_to_binary(httpd_util:reason_phrase(Code))},
{reason, list_to_binary(rabbit_misc:format("~p~n", [Reason]))}]},
{mochijson2:encode(Json), ReqState}.
maybe_log(_Req, {error, {exit, normal, _Stack}}) ->
%% webmachine_request did an exit(normal), so suppress this
%% message. This usually happens when a chunked upload is
%% interrupted by network failure.
ok;
maybe_log(Req, Reason) ->
{Path, _} = Req:path(),
error_logger:error_msg("webmachine error: path=~p~n~p~n", [Path, Reason]).

View File

@ -5,4 +5,4 @@
{registered, []},
{mod, {rabbit_web_dispatch_app, []}},
{env, []},
{applications, [kernel, stdlib, mochiweb, webmachine]}]}.
{applications, [kernel, stdlib, cowboy]}]}.

View File

@ -0,0 +1,239 @@
%% Copyright (c) 2011-2012 Basho Technologies, Inc. All Rights Reserved.
%%
%% This file is provided to you under the Apache License,
%% Version 2.0 (the "License"); you may not use this file
%% except in compliance with the License. You may obtain
%% a copy of the License at
%%
%% http://www.apache.org/licenses/LICENSE-2.0
%%
%% Unless required by applicable law or agreed to in writing,
%% software distributed under the License is distributed on an
%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
%% KIND, either express or implied. See the License for the
%% specific language governing permissions and limitations
%% under the License.
%% @doc Helper functions for webmachine's default log handlers
-module(webmachine_log).
-include("webmachine_logger.hrl").
-export([add_handler/2,
call/2,
call/3,
datehour/0,
datehour/1,
defer_refresh/1,
delete_handler/1,
fix_log/2,
fmt_ip/1,
fmtnow/0,
log_access/1,
log_close/3,
log_open/1,
log_open/2,
log_write/2,
maybe_rotate/3,
month/1,
refresh/2,
suffix/1,
zeropad/2,
zone/0]).
-record(state, {hourstamp :: non_neg_integer(),
filename :: string(),
handle :: file:io_device()}).
%% @doc Add a handler to receive log events
-type add_handler_result() :: ok | {'EXIT', term()} | term().
-spec add_handler(atom() | {atom(), term()}, term()) -> add_handler_result().
add_handler(Mod, Args) ->
gen_event:add_handler(?EVENT_LOGGER, Mod, Args).
%% @doc Make a synchronous call directly to a specific event handler
%% module
-type error() :: {error, bad_module} | {'EXIT', term()} | term().
-spec call(atom(), term()) -> term() | error().
call(Mod, Msg) ->
gen_event:call(?EVENT_LOGGER, Mod, Msg).
%% @doc Make a synchronous call directly to a specific event handler
%% module
-spec call(atom(), term(), timeout()) -> term() | error().
call(Mod, Msg, Timeout) ->
gen_event:call(?EVENT_LOGGER, Mod, Msg, Timeout).
%% @doc Return a four-tuple containing year, month, day, and hour
%% of the current time.
-type datehour() :: {calendar:year(), calendar:month(), calendar:day(), calendar:hour()}.
-spec datehour() -> datehour().
datehour() ->
datehour(os:timestamp()).
%% @doc Return a four-tuple containing year, month, day, and hour
%% of the specified time.
-spec datehour(erlang:timestamp()) -> datehour().
datehour(TS) ->
{{Y, M, D}, {H, _, _}} = calendar:now_to_universal_time(TS),
{Y, M, D, H}.
%% @doc Defer the refresh of a log file.
-spec defer_refresh(atom()) -> {ok, timer:tref()} | {error, term()}.
defer_refresh(Mod) ->
{_, {_, M, S}} = calendar:universal_time(),
Time = 1000 * (3600 - ((M * 60) + S)),
timer:apply_after(Time, ?MODULE, refresh, [Mod, os:timestamp()]).
%% @doc Remove a log handler
-type delete_handler_result() :: term() | {error, module_not_found} | {'EXIT', term()}.
-spec delete_handler(atom() | {atom(), term()}) -> delete_handler_result().
delete_handler(Mod) ->
gen_event:delete_handler(?EVENT_LOGGER, Mod, []).
%% Seek backwards to the last valid log entry
-spec fix_log(file:io_device(), non_neg_integer()) -> ok.
fix_log(_FD, 0) ->
ok;
fix_log(FD, 1) ->
{ok, 0} = file:position(FD, 0),
ok;
fix_log(FD, Location) ->
case file:pread(FD, Location - 1, 1) of
{ok, [$\n | _]} ->
ok;
{ok, _} ->
fix_log(FD, Location - 1)
end.
%% @doc Format an IP address or host name
-spec fmt_ip(undefined | string() | inet:ip4_address() | inet:ip6_address()) -> string().
fmt_ip(IP) when is_tuple(IP) ->
inet_parse:ntoa(IP);
fmt_ip(undefined) ->
"0.0.0.0";
fmt_ip(HostName) ->
HostName.
%% @doc Format the current time into a string
-spec fmtnow() -> string().
fmtnow() ->
{{Year, Month, Date}, {Hour, Min, Sec}} = calendar:local_time(),
io_lib:format("[~2..0w/~s/~4..0w:~2..0w:~2..0w:~2..0w ~s]",
[Date,month(Month),Year, Hour, Min, Sec, zone()]).
%% @doc Notify registered log event handler of an access event.
-spec log_access(tuple()) -> ok.
log_access({_, _, _}=LogData) ->
gen_event:sync_notify(?EVENT_LOGGER, {log_access, LogData}).
%% @doc Close a log file.
-spec log_close(atom(), string(), file:io_device()) -> ok | {error, term()}.
log_close(Mod, Name, FD) ->
error_logger:info_msg("~p: closing log file: ~p~n", [Mod, Name]),
file:close(FD).
%% @doc Open a new log file for writing
-spec log_open(string()) -> {file:io_device(), non_neg_integer()}.
log_open(FileName) ->
DateHour = datehour(),
{log_open(FileName, DateHour), DateHour}.
%% @doc Open a new log file for writing
-spec log_open(string(), non_neg_integer()) -> file:io_device().
log_open(FileName, DateHour) ->
LogName = FileName ++ suffix(DateHour),
error_logger:info_msg("opening log file: ~p~n", [LogName]),
filelib:ensure_dir(LogName),
{ok, FD} = file:open(LogName, [read, write, raw]),
{ok, Location} = file:position(FD, eof),
fix_log(FD, Location),
file:truncate(FD),
FD.
-spec log_write(file:io_device(), iolist()) -> ok | {error, term()}.
log_write(FD, IoData) ->
file:write(FD, lists:flatten(IoData)).
%% @doc Rotate a log file if the hour it represents
%% has passed.
-spec maybe_rotate(atom(), erlang:timestamp(), #state{}) -> #state{}.
maybe_rotate(Mod, Time, State) ->
ThisHour = datehour(Time),
if ThisHour == State#state.hourstamp ->
State;
true ->
defer_refresh(Mod),
log_close(Mod, State#state.filename, State#state.handle),
Handle = log_open(State#state.filename, ThisHour),
State#state{hourstamp=ThisHour, handle=Handle}
end.
%% @doc Convert numeric month value to the abbreviation
-spec month(1..12) -> string().
month(1) ->
"Jan";
month(2) ->
"Feb";
month(3) ->
"Mar";
month(4) ->
"Apr";
month(5) ->
"May";
month(6) ->
"Jun";
month(7) ->
"Jul";
month(8) ->
"Aug";
month(9) ->
"Sep";
month(10) ->
"Oct";
month(11) ->
"Nov";
month(12) ->
"Dec".
%% @doc Make a synchronous call to instruct a log handler to refresh
%% itself.
-spec refresh(atom(), erlang:timestamp()) -> ok | {error, term()}.
refresh(Mod, Time) ->
call(Mod, {refresh, Time}, infinity).
-spec suffix(datehour()) -> string().
suffix({Y, M, D, H}) ->
YS = zeropad(Y, 4),
MS = zeropad(M, 2),
DS = zeropad(D, 2),
HS = zeropad(H, 2),
lists:flatten([$., YS, $_, MS, $_, DS, $_, HS]).
-spec zeropad(integer(), integer()) -> string().
zeropad(Num, MinLength) ->
NumStr = integer_to_list(Num),
zeropad_str(NumStr, MinLength - length(NumStr)).
-spec zeropad_str(string(), integer()) -> string().
zeropad_str(NumStr, Zeros) when Zeros > 0 ->
zeropad_str([$0 | NumStr], Zeros - 1);
zeropad_str(NumStr, _) ->
NumStr.
-spec zone() -> string().
zone() ->
Time = erlang:universaltime(),
LocalTime = calendar:universal_time_to_local_time(Time),
DiffSecs = calendar:datetime_to_gregorian_seconds(LocalTime) -
calendar:datetime_to_gregorian_seconds(Time),
zone((DiffSecs/3600)*100).
%% Ugly reformatting code to get times like +0000 and -1300
-spec zone(integer()) -> string().
zone(Val) when Val < 0 ->
io_lib:format("-~4..0w", [trunc(abs(Val))]);
zone(Val) when Val >= 0 ->
io_lib:format("+~4..0w", [trunc(abs(Val))]).

View File

@ -0,0 +1,111 @@
%% Copyright (c) 2011-2013 Basho Technologies, Inc. All Rights Reserved.
%%
%% This file is provided to you under the Apache License,
%% Version 2.0 (the "License"); you may not use this file
%% except in compliance with the License. You may obtain
%% a copy of the License at
%%
%% http://www.apache.org/licenses/LICENSE-2.0
%%
%% Unless required by applicable law or agreed to in writing,
%% software distributed under the License is distributed on an
%% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
%% KIND, either express or implied. See the License for the
%% specific language governing permissions and limitations
%% under the License.
%% @doc Default log handler for webmachine
-module(webmachine_log_handler).
-behaviour(gen_event).
%% gen_event callbacks
-export([init/1,
handle_call/2,
handle_event/2,
handle_info/2,
terminate/2,
code_change/3]).
-include("webmachine_logger.hrl").
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
-endif.
-record(state, {hourstamp, filename, handle}).
-define(FILENAME, "access.log").
%% ===================================================================
%% gen_event callbacks
%% ===================================================================
%% @private
init([BaseDir]) ->
webmachine_log:defer_refresh(?MODULE),
FileName = filename:join(BaseDir, ?FILENAME),
{Handle, DateHour} = webmachine_log:log_open(FileName),
{ok, #state{filename=FileName, handle=Handle, hourstamp=DateHour}}.
%% @private
handle_call({_Label, MRef, get_modules}, State) ->
{ok, {MRef, [?MODULE]}, State};
handle_call({refresh, Time}, State) ->
{ok, ok, webmachine_log:maybe_rotate(?MODULE, Time, State)};
handle_call(_Request, State) ->
{ok, ok, State}.
%% @private
handle_event({log_access, LogData}, State) ->
NewState = webmachine_log:maybe_rotate(?MODULE, os:timestamp(), State),
Msg = format_req(LogData),
webmachine_log:log_write(NewState#state.handle, Msg),
{ok, NewState};
handle_event(_Event, State) ->
{ok, State}.
%% @private
handle_info(_Info, State) ->
{ok, State}.
%% @private
terminate(_Reason, _State) ->
ok.
%% @private
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%% ===================================================================
%% Internal functions
%% ===================================================================
%% We currently keep most of the Webmachine logging facility. But
%% since we are now using Cowboy, a few small parts had to change.
%% This is one such part. The code is however equivalent to Webmachine's.
format_req({Status0, Body, Req}) ->
User = "-",
Time = webmachine_log:fmtnow(),
Status = integer_to_list(Status0),
Length = integer_to_list(iolist_size(Body)),
{Method, _} = cowboy_req:method(Req),
{Path, _} = cowboy_req:path(Req),
{{Peer, _}, _} = cowboy_req:peer(Req),
Version = case cowboy_req:version(Req) of
{'HTTP/1.1', _} -> {1, 1};
{'HTTP/1.0', _} -> {1, 0}
end,
{Referer, _} = cowboy_req:header(<<"referer">>, Req, <<>>),
{UserAgent, _} = cowboy_req:header(<<"user-agent">>, Req, <<>>),
fmt_alog(Time, Peer, User, Method, Path, Version,
Status, Length, Referer, UserAgent).
fmt_alog(Time, Ip, User, Method, Path, {VM,Vm},
Status, Length, Referrer, UserAgent) ->
[webmachine_log:fmt_ip(Ip), " - ", User, [$\s], Time, [$\s, $"], Method, " ", Path,
" HTTP/", integer_to_list(VM), ".", integer_to_list(Vm), [$",$\s],
Status, [$\s], Length, [$\s,$"], Referrer,
[$",$\s,$"], UserAgent, [$",$\n]].

View File

@ -0,0 +1,16 @@
-record(wm_log_data,
{resource_module :: atom(),
start_time :: tuple(),
method :: atom(),
headers,
peer,
path :: string(),
version,
response_code,
response_length,
end_time :: tuple(),
finish_time :: tuple(),
notes}).
-type wm_log_data() :: #wm_log_data{}.
-define(EVENT_LOGGER, webmachine_log_event).