24
1
mirror of https://github.com/processone/ejabberd.git synced 2024-07-14 23:44:18 +02:00

added antiflood and filter modules from TeamLeader

This commit is contained in:
Christophe Romain 2010-09-24 16:11:59 +02:00
parent db2baa8f84
commit b6dcd41225
4 changed files with 1002 additions and 0 deletions

205
src/floodcheck.erl Normal file
View File

@ -0,0 +1,205 @@
%%%-------------------------------------------------------------------
%%% File : floodcheck.erl
%%% Author : Christophe Romain <christophe.romain@process-one.net>
%%% Description :
%%%
%%% Created : 11 Sep 2008 by Christophe Romain <christophe.romain@process-one.net>
%%%-------------------------------------------------------------------
-module(floodcheck).
-behaviour(gen_server).
%% API
-export([start_link/0, stop/0]).
-export([monitor/5, demonitor/1, interval/1, check/0]).
%% gen_server callbacks
-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2, code_change/3]).
-define(DEFAULT_INTERVAL, 300). %% check every 5mn
-define(SERVER, ?MODULE).
-record(state, {timer, interval, monitors}).
-record(monitor, {id, pid, ref, info, rule, value, handler}).
%%====================================================================
%% API
%%====================================================================
%%--------------------------------------------------------------------
%% Function: start_link() -> {ok,Pid} | ignore | {error,Error}
%% Description: Starts the server
%%--------------------------------------------------------------------
start_link() ->
case whereis(?SERVER) of
undefined -> gen_server:start_link({local, ?SERVER}, ?MODULE, [], []);
Pid -> {ok, Pid}
end.
stop() ->
gen_server:call(?SERVER, stop).
monitor(Id, Pid, Info, Spec, {Mod, Fun}) ->
gen_server:cast(?SERVER, {monitor, Id, Pid, Info, Spec, {Mod, Fun}}).
demonitor(Id) ->
gen_server:cast(?SERVER, {demonitor, Id}).
interval(Value) ->
gen_server:cast(?SERVER, {interval, Value}).
check() ->
gen_server:call(?SERVER, check).
%%====================================================================
%% gen_server callbacks
%%====================================================================
%%--------------------------------------------------------------------
%% Function: init(Args) -> {ok, State} |
%% {ok, State, Timeout} |
%% ignore |
%% {stop, Reason}
%% Description: Initiates the server
%%--------------------------------------------------------------------
init([]) ->
Timer = erlang:send_after(?DEFAULT_INTERVAL*1000, ?SERVER, monitor),
{ok, #state{timer=Timer, interval=?DEFAULT_INTERVAL, monitors=[]}}.
%%--------------------------------------------------------------------
%% Function: %% handle_call(Request, From, State) -> {reply, Reply, State} |
%% {reply, Reply, State, Timeout} |
%% {noreply, State} |
%% {noreply, State, Timeout} |
%% {stop, Reason, Reply, State} |
%% {stop, Reason, State}
%% Description: Handling call messages
%%--------------------------------------------------------------------
handle_call(check, _From, State) ->
Reply = lists:map(fun(#monitor{id=Id}=M) ->
{Id, check(M)}
end, State#state.monitors),
{reply, Reply, State};
handle_call(stop, _From, State) ->
erlang:cancel_timer(State#state.timer),
{stop, normal, ok, State}.
%%--------------------------------------------------------------------
%% Function: handle_cast(Msg, State) -> {noreply, State} |
%% {noreply, State, Timeout} |
%% {stop, Reason, State}
%% Description: Handling cast messages
%%--------------------------------------------------------------------
handle_cast({monitor, Id, Pid, Info, Spec, Handler}, State) ->
Monitors = State#state.monitors,
Ref = erlang:monitor(process, Pid),
{Rule, Value} = case Spec of
{Op, V} -> {Op, V};
V -> {'>', V}
end,
Monitor = #monitor{id=Id, pid=Pid, ref=Ref, info=Info, rule=Rule, value=Value, handler=Handler},
New = case lists:keysearch(Id, #monitor.id, Monitors) of
{value, #monitor{ref=OldRef}} ->
erlang:demonitor(OldRef),
lists:keyreplace(Id, #monitor.id, Monitors, Monitor);
_ ->
[Monitor|Monitors]
end,
{noreply, State#state{monitors=New}};
handle_cast({demonitor, Id}, State) ->
Monitors = State#state.monitors,
New = case lists:keysearch(Id, #monitor.id, Monitors) of
{value, #monitor{ref=Ref}} ->
erlang:demonitor(Ref),
lists:keydelete(Id, #monitor.id, Monitors);
_ ->
Monitors
end,
{noreply, State#state{monitors=New}};
handle_cast({interval, Value}, State) ->
erlang:cancel_timer(State#state.timer),
Timer = erlang:send_after(Value*1000, ?SERVER, monitor),
{noreply, State#state{timer=Timer, interval=Value}};
handle_cast(_Msg, State) ->
{noreply, State}.
%%--------------------------------------------------------------------
%% Function: handle_info(Info, State) -> {noreply, State} |
%% {noreply, State, Timeout} |
%% {stop, Reason, State}
%% Description: Handling all non call/cast messages
%%--------------------------------------------------------------------
handle_info({'DOWN', Ref, _Type, _Pid, _Info}, State) ->
Monitors = State#state.monitors,
New = lists:keydelete(Ref, #monitor.ref, Monitors),
{noreply, State#state{monitors=New}};
handle_info(monitor, State) ->
lists:foreach(fun(#monitor{id=Id, pid=Pid, info=Info, handler={Mod, Fun}}=M) ->
case check(M) of
ok -> ok;
Value -> spawn(Mod, Fun, [Id, Pid, Info, Value])
end
end, State#state.monitors),
Timer = erlang:send_after(State#state.interval*1000, ?SERVER, monitor),
{noreply, State#state{timer=Timer}};
handle_info(_Info, State) ->
{noreply, State}.
%%--------------------------------------------------------------------
%% Function: terminate(Reason, State) -> void()
%% Description: This function is called by a gen_server when it is about to
%% terminate. It should be the opposite of Module:init/1 and do any necessary
%% cleaning up. When it returns, the gen_server terminates with Reason.
%% The return value is ignored.
%%--------------------------------------------------------------------
terminate(_Reason, _State) ->
ok.
%%--------------------------------------------------------------------
%% Func: code_change(OldVsn, State, Extra) -> {ok, NewState}
%% Description: Convert process state when code is changed
%%--------------------------------------------------------------------
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%--------------------------------------------------------------------
%%% Internal functions
%%--------------------------------------------------------------------
check(#monitor{pid=Pid, info=Info, rule=Rule, value=Value}) ->
case catch process_info(Pid, Info) of
{Info, Actual} ->
Check = case Info of
messages -> byte_size(term_to_binary(Actual));
dictionary -> byte_size(term_to_binary(Actual));
_ -> Actual
end,
case Rule of
'>' ->
if Check > Value -> Value;
true -> ok
end;
'<' ->
if Check < Value -> Value;
true -> ok
end;
'=' ->
if Check == Value -> Value;
true -> ok
end;
_ ->
ok
end;
_ ->
ok
end.
%%% Documentation
%%% authorized Info
%%% message_queue_len: number of messages
%%% messages: messages queue size in bytes
%%% dictionary: dictionary size in bytes
%%% total_heap_size: total size in words of all heap fragments
%%% heap_size: size in words of youngest heap generation
%%% stack_size: stack size in words
%%% reductions: number of reductions executed by the process
%%% memory: process size in bytes

186
src/mod_antiflood.erl Normal file
View File

@ -0,0 +1,186 @@
%%%-------------------------------------------------------------------
%%% File : mod_antiflood.erl
%%% Author : Christophe Romain <cromain@process-one.net>
%%% Description :
%%% Created : 12 Sep 2008 by Christophe Romain <cromain@process-one.net>
%%%
%%% ejabberd, Copyright (C) 2002-2009 ProcessOne
%%%
%%% This program is free software; you can redistribute it and/or
%%% modify it under the terms of the GNU General Public License as
%%% published by the Free Software Foundation; either version 2 of the
%%% License, or (at your option) any later version.
%%%
%%% This program is distributed in the hope that it will be useful,
%%% but WITHOUT ANY WARRANTY; without even the implied warranty of
%%% MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
%%% General Public License for more details.
%%%
%%% You should have received a copy of the GNU General Public License
%%% along with this program; if not, write to the Free Software
%%% Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA
%%% 02111-1307 USA
%%%
%%%-------------------------------------------------------------------
-module(mod_antiflood).
-author('cromain@process-one.net').
-behaviour(gen_mod).
%% API
-export([start/2, stop/1]).
-export([handler/4]).
-include("ejabberd.hrl").
%%====================================================================
%% API
%%====================================================================
start(Host, Opts) ->
floodcheck:start_link(),
lists:foreach(fun
({listeners, Info, Op, Val}) ->
lists:foreach(fun({Name, Pid}) ->
Id = {Name, Info},
floodcheck:monitor(Id, Pid, Info, {Op, Val}, {?MODULE, handler})
end, supervised_processes(listeners));
({Module, Info, Op, Val}) ->
case module_pid(Host, Module) of
Pid when is_pid(Pid) ->
Id = {Host, Module, Info},
floodcheck:monitor(Id, Pid, Info, {Op, Val}, {?MODULE, handler});
Error ->
?INFO_MSG("can not monitor ~s (~p)", [Module, Error])
end;
(Arg) ->
?INFO_MSG("invalid argument: ~p", [Arg])
end, Opts).
stop(Host) ->
MList = gen_mod:loaded_modules_with_opts(Host),
case lists:keysearch(?MODULE, 1, MList) of
{value, {?MODULE, Opts}} ->
lists:foreach(fun
({Type, Info, _, _}) ->
case supervised_processes(Type) of
[] ->
Id = {Host, Type, Info},
floodcheck:demonitor(Id);
Childs ->
lists:foreach(fun({Name, _}) ->
Id = {Name, Info},
floodcheck:demonitor(Id)
end, Childs)
end;
(_) ->
ok
end, Opts);
false ->
ok
end,
case floodcheck:check() of
[] -> floodcheck:stop(), ok;
_ -> ok
end.
handler({Host, Module, Info}, Pid, Info, Value) ->
?WARNING_MSG("Flood alert on Process ~p (~s on ~s): ~s=~p", [Pid, Module, Host, Info, Value]),
restart_module(Host, Module),
remonitor({Host, Module, Info});
handler({Name, Info}, Pid, Info, Value) ->
?WARNING_MSG("Flood alert on Process ~p (~s): ~s=~p", [Pid, Name, Info, Value]),
kill_process(Name, Pid),
remonitor({Name, Info});
handler(Id, Pid, Info, Value) ->
?WARNING_MSG("Flood alert on Process ~p (~s): ~s=~p~nUnknown id, alert ignored", [Pid, Id, Info, Value]).
%%====================================================================
%% Internal functions
%%====================================================================
process_pid(Name) -> whereis(Name).
server_pid(Host, Name) -> process_pid(gen_mod:get_module_proc(Host, Name)).
module_pid(Host, mod_caps) -> server_pid(Host, ejabberd_mod_caps);
module_pid(Host, mod_ip_blacklist) -> server_pid(Host, mod_ip_blacklist);
module_pid(Host, mod_offline) -> server_pid(Host, ejabberd_offline);
module_pid(Host, mod_offline_odbc) -> server_pid(Host, ejabberd_offline);
module_pid(Host, mod_vcard) -> server_pid(Host, ejabberd_mod_vcard);
module_pid(Host, mod_vcard_odbc) -> server_pid(Host, ejabberd_mod_vcard);
module_pid(Host, mod_vcard_ldap) -> server_pid(Host, ejabberd_mod_vcard_ldap);
module_pid(Host, mod_irc) -> server_pid(Host, ejabberd_mod_irc);
module_pid(Host, mod_muc) -> server_pid(Host, ejabberd_mod_muc);
module_pid(Host, mod_muc_log) -> server_pid(Host, ejabberd_mod_muc_log);
module_pid(Host, mod_proxy65) -> server_pid(Host, ejabberd_mod_proxy65);
module_pid(Host, mod_proxy65_service) -> server_pid(Host, ejabberd_mod_proxy65_service);
module_pid(Host, mod_proxy65_sm) -> server_pid(Host, ejabberd_mod_proxy65_sm);
module_pid(Host, mod_pubsub) -> server_pid(Host, ejabberd_mod_pubsub);
module_pid(_, _) -> unsupported.
supervised_processes(listeners) ->
{links, Links} = process_info(whereis(ejabberd_listeners), links),
lists:map(fun(Pid) ->
{dictionary, Dict} = process_info(Pid, dictionary),
{_, _, [Port|_]} = proplists:get_value('$initial_call', Dict),
Name = list_to_atom("listener_"++integer_to_list(Port)),
{Name, Pid}
end, Links);
supervised_processes(_) -> [].
remonitor({Host, Module, Info}) ->
MList = gen_mod:loaded_modules_with_opts(Host),
case lists:keysearch(?MODULE, 1, MList) of
{value, {?MODULE, Opts}} ->
lists:foreach(fun
({M, I, Op, Val}) when M =:= Module, I =:= Info ->
case module_pid(Host, Module) of
Pid when is_pid(Pid) ->
Id = {Host, Module, Info},
floodcheck:monitor(Id, Pid, Info, {Op, Val}, {?MODULE, handler});
Error ->
?INFO_MSG("can not monitor ~s (~p)", [Module, Error])
end;
(_) ->
ok
end, Opts);
_ ->
ok
end;
remonitor({Name, Info}) ->
[Host|_] = ejabberd_config:get_global_option(hosts),
MList = gen_mod:loaded_modules_with_opts(Host),
case lists:keysearch(?MODULE, 1, MList) of
{value, {?MODULE, Opts}} ->
lists:foreach(fun
({Type, I, Op, Val}) when I =:= Info ->
lists:foreach(fun
({N, Pid}) when N =:= Name ->
Id = {Name, Info},
floodcheck:monitor(Id, Pid, Info, {Op, Val}, {?MODULE, handler});
(_) ->
ok
end, supervised_processes(Type));
(_) ->
ok
end, Opts);
_ ->
ok
end;
remonitor(Id) ->
?INFO_MSG("can not monitor ~s", [Id]).
restart_module(Host, Module) ->
MList = gen_mod:loaded_modules_with_opts(Host),
case lists:keysearch(Module, 1, MList) of
{value, {Module, Opts}} ->
?WARNING_MSG("restarting module ~s on ~s", [Module, Host]),
gen_mod:stop_module(Host, Module),
gen_mod:start_module(Host, Module, Opts);
_ ->
not_running
end.
kill_process(Name, Pid) ->
?WARNING_MSG("killing process ~s(~p)", [Name, Pid]),
exit(Pid, kill).

350
src/mod_filter.erl Normal file
View File

@ -0,0 +1,350 @@
%%% ====================================================================
%%% This software is copyright 2006-2009, ProcessOne.
%%%
%%% mod_filter
%%% allow message filtering using regexp on message body
%%% THIS MODULE NEEDS A PATCHED ERLANG VM AGAINST
%%% THE PCRE PATCH AND NEEDS LIBPCRE INSTALLED
%%% ejabberd MUST USE THAT PATCHED ERLANG VM
%%% BUT, if patch is not available, mod_filter uses re.beam module
%%% instead, with speed degradation.
%%%
%%% @copyright 2006-2009 ProcessOne
%%% @author Christophe Romain <christophe.romain@process-one.net>
%%% [http://www.process-one.net/]
%%% @version {@vsn}, {@date} {@time}
%%% @end
%%% ====================================================================
-module(mod_filter).
-author('christophe.romain@process-one.net').
-vsn('$Id: mod_filter.erl 767 2009-04-20 16:31:40Z cromain $').
-behaviour(gen_mod).
% module functions
-export([start/2,stop/1,init/2,update/2,is_loaded/0,loop/5]).
-export([add_regexp/4,add_regexp/3,del_regexp/3,del_regexp/2]).
-export([purge_logs/0,purge_regexps/1,reload/1]).
-export([logged/0,logged/1,rules/0]).
-export([process_local_iq/3]).
% handled ejabberd hooks
-export([filter_packet/1]).
-include("ejabberd.hrl").
-include("jlib.hrl").
-include("licence.hrl").
-record(filter_rule, {id, type="filter", regexp, binre}).
-record(filter_log, {date, from, to, message}).
-define(TIMEOUT, 5000). % deliver message anyway if filter does not respond after 5s
-define(PROCNAME(VH), list_to_atom(VH++"_message_filter")).
-define(NS_FILTER, "p1:iq:filter").
-define(ALLHOSTS, "all hosts"). %% must be sync with filter.erl
start(Host, Opts) ->
case ?IS_VALID of
true ->
mnesia:create_table(filter_rule, [
{disc_copies, [node()]}, {type, set},
{attributes, record_info(fields, filter_rule)} ]),
mnesia:create_table(filter_log, [
{disc_only_copies, [node()]}, {type, bag},
{attributes, record_info(fields, filter_log)} ]),
%% this force the last code to be used
case whereis(?PROCNAME(Host)) of
undefined ->
ok;
_ ->
ejabberd_hooks:delete(filter_packet, ?MODULE, filter_packet, 10),
gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_FILTER),
?PROCNAME(Host) ! quit
end,
ejabberd_hooks:add(filter_packet, ?MODULE, filter_packet, 10),
gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_FILTER,
?MODULE, process_local_iq, one_queue),
case whereis(?PROCNAME(Host)) of
undefined -> register(?PROCNAME(Host), spawn(?MODULE, init, [Host, Opts]));
_ -> ok
end,
%% start the all_alias handler
case whereis(?PROCNAME(?ALLHOSTS)) of
undefined -> init_all_hosts_handler();
_ -> ok
end,
start;
false ->
not_started
end.
stop(Host) ->
ejabberd_hooks:delete(filter_packet, ?MODULE, filter_packet, 10),
gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_FILTER),
exit(whereis(?PROCNAME(Host)), kill),
{wait, ?PROCNAME(Host)}.
% this is used by team_leader to check code presence
is_loaded() ->
ok.
% this loads rules and return {Types, BinRegExps}
% RegExps are text regexp list
% Types are regexp type list
% both list share the same ordered
load_rules(Host) ->
Rules = mnesia:dirty_match_object(#filter_rule{id={'_', Host}, _ = '_'})
++ mnesia:dirty_match_object(#filter_rule{id={'_', ?ALLHOSTS}, _ = '_'}),
lists:map(fun({filter_rule, _, Type, _, BinRegExp}) -> {Type, BinRegExp} end, Rules).
%lists:unzip(Config).
% this call init or reset local rules reading database
init(Host, Opts) ->
Rules = load_rules(Host),
Scope = gen_mod:get_opt(scope, Opts, message),
Pattern = gen_mod:get_opt(pattern, Opts, ""),
?MODULE:loop(Host, Opts, Rules, Scope, Pattern).
init_all_hosts_handler() ->
register(?PROCNAME(?ALLHOSTS), spawn(?MODULE, loop, [?ALLHOSTS, [], [], none, []])).
% this call reset local rules reading database
% and tell other nodes to reset rules as well
update(?ALLHOSTS, _Opts) ->
lists:foreach(fun(Host) ->
lists:foreach(fun(Node) ->
catch rpc:call(Node,mod_filter,reload,[Host])
end, mnesia:system_info(running_db_nodes))
end, ejabberd_config:get_global_option(hosts)),
?MODULE:loop(?ALLHOSTS, [], [], none, []);
update(Host, Opts) ->
% tell other nodes to update filter
lists:foreach(fun(Node) ->
catch rpc:call(Node,mod_filter,reload,[Host])
end, mnesia:system_info(running_db_nodes)--[node()]),
% update rules
init(Host, Opts).
loop(Host, Opts, Rules, Scope, Pattern) ->
receive
{add, Id, RegExp} ->
[BinRegExp] = tlre:compile([RegExp]),
mnesia:dirty_write(#filter_rule{id={Id, Host}, regexp=RegExp, binre=BinRegExp}),
?MODULE:update(Host, Opts);
{add, Id, RegExp, Type} ->
[BinRegExp] = tlre:compile([RegExp]),
mnesia:dirty_write(#filter_rule{id={Id, Host}, regexp=RegExp, binre=BinRegExp, type=Type}),
?MODULE:update(Host, Opts);
{del, Id} ->
mnesia:dirty_delete_object(#filter_rule{id={Id, Host}, _='_'}),
?MODULE:update(Host, Opts);
{del, Id, RegExp} ->
mnesia:dirty_delete_object(#filter_rule{id={Id, Host}, regexp=RegExp, _='_'}),
?MODULE:update(Host, Opts);
{match, From, String} ->
From ! {match, string_filter(String, Rules, Scope, Pattern)},
?MODULE:loop(Host, Opts, Rules, Scope, Pattern);
reload ->
?MODULE:init(Host, Opts);
quit ->
unregister(?PROCNAME(Host)),
ok
end.
string_filter(String, Rules, Scope, Pattern) ->
lists:foldl(fun
(_, {Pass, []}) -> {Pass, []};
({Type, RegExp}, {Pass, NewString}) -> string_filter(NewString, Pass, RegExp, Type, Scope, Pattern)
end, {"pass", String}, Rules).
string_filter(String, Pass, RegExp, Type, Scope, Pattern) ->
case tlre:grep(String, [RegExp]) of
[no_match] ->
{Pass, String};
[{S1, S2, _}] ->
case Scope of
word ->
Start = string:sub_string(String, 1, S1),
StringTail = string:sub_string(String, S2+1, length(String)),
NewPass = pass_rule(Pass, Type),
{LastPass, End} = string_filter(StringTail, NewPass, RegExp, Type, Scope, Pattern),
NewString = case Type of
"log" -> lists:append([string:sub_string(String, 1, S2), End]);
_ -> lists:append([Start, Pattern, End])
end,
{LastPass, NewString};
_ ->
NewString = case Type of
"log" -> String;
_ -> []
end,
{pass_rule(Pass, Type), NewString}
end
end.
pass_rule("pass", New) -> New;
pass_rule("log", "log") -> "log";
pass_rule("log", "log and filter") -> "log and filter";
pass_rule("log", "filter") -> "log and filter";
pass_rule("filter", "log") -> "log and filter";
pass_rule("filter", "log and filter") -> "log and filter";
pass_rule("filter", "filter") -> "filter";
pass_rule("log and filter", _) -> "log and filter".
add_regexp(VH, Id, RegExp) ->
?PROCNAME(VH) ! {add, Id, RegExp},
ok.
add_regexp(VH, Id, RegExp, Type) ->
?PROCNAME(VH) ! {add, Id, RegExp, Type},
ok.
del_regexp(VH, Id) ->
?PROCNAME(VH) ! {del, Id},
ok.
del_regexp(VH, Id, RegExp) ->
?PROCNAME(VH) ! {del, Id, RegExp},
ok.
reload(VH) ->
?PROCNAME(VH) ! reload,
ok.
purge_logs() ->
mnesia:dirty_delete_object(#filter_log{_='_'}).
%purge_regexps() ->
% mnesia:dirty_delete_object(#filter_rule{_='_'}),
% reload().
purge_regexps(VH) ->
mnesia:dirty_delete_object(#filter_rule{id={'_', VH}, _='_'}),
reload(VH).
rules() ->
lists:map(fun(#filter_rule{id={Label, VH}, type=Type, regexp=Regexp}) ->
{VH, Label, Type, Regexp}
end, mnesia:dirty_match_object(#filter_rule{_='_'})).
logged() ->
lists:reverse(lists:map(fun(#filter_log{date=Date, from=From, to=To, message=Msg}) ->
{Date, jlib:jid_to_string(From), jlib:jid_to_string(To), Msg}
end, mnesia:dirty_match_object(#filter_log{_='_'}))).
logged(Limit) ->
List = mnesia:dirty_match_object(#filter_log{_='_'}),
Len = length(List),
FinalList = if
Len < Limit -> List;
true -> lists:nthtail(Len-Limit, List)
end,
lists:reverse(lists:map(fun(#filter_log{date=Date, from=From, to=To, message=Msg}) ->
{Date, jlib:jid_to_string(From), jlib:jid_to_string(To), Msg}
end, FinalList)).
%% filter_packet can receive drop if a previous filter already dropped
%% the packet
filter_packet(drop) -> drop;
filter_packet({From, To, Packet}) ->
case Packet of
{xmlelement, "message", MsgAttrs, Els} ->
case lists:keysearch("body", 2, Els) of
{value, {xmlelement, "body", BodyAttrs, Data}} ->
NewData = lists:foldl(fun
({xmlcdata, CData}, DataAcc) when is_binary(CData) ->
#jid{lserver = Host} = To,
case lists:member(Host, ejabberd_config:get_global_option(hosts)) of
true ->
Msg = binary_to_list(CData),
?PROCNAME(Host) ! {match, self(), Msg},
receive
{match, {"pass", _}} ->
[{xmlcdata, CData}|DataAcc];
{match, {"log", FinalString}} ->
mnesia:dirty_write(#filter_log{
date=erlang:localtime(),
from=From, to=To, message=Msg}),
[{xmlcdata, CData}|DataAcc];
{match, {"log and filter", FinalString}} ->
mnesia:dirty_write(#filter_log{
date=erlang:localtime(),
from=From, to=To, message=Msg}),
case FinalString of
[] -> % entire message is dropped
DataAcc;
S -> % message must be regenerated
[{xmlcdata, list_to_binary(S)}|DataAcc]
end;
{match, {"filter", FinalString}} ->
case FinalString of
[] -> % entire message is dropped
DataAcc;
S -> % message must be regenerated
[{xmlcdata, list_to_binary(S)}|DataAcc]
end
after ?TIMEOUT ->
[{xmlcdata, CData}|DataAcc]
end;
false ->
[{xmlcdata, CData}|DataAcc]
end;
(Item, DataAcc) -> %% to not filter internal messages
[Item|DataAcc]
end, [], Data),
case NewData of
[] ->
drop;
D ->
NewEls = lists:keyreplace("body", 2, Els, {xmlelement, "body", BodyAttrs, lists:reverse(D)}),
{From, To, {xmlelement, "message", MsgAttrs, NewEls}}
end;
_ ->
{From, To, Packet}
end;
_ ->
{From, To, Packet}
end.
process_local_iq(From, #jid{lserver=VH} = _To, #iq{type = Type, sub_el = SubEl} = IQ) ->
case Type of
get ->
IQ#iq{type = error, sub_el = [SubEl, ?ERR_NOT_ALLOWED]};
set ->
#jid{luser = User, lserver = Server, lresource = Resource} = From,
case acl:match_rule(global, configure, {User, Server, Resource}) of
allow ->
case xml:get_subtag(SubEl, "add") of
{xmlelement, "add", AddAttrs, _} ->
AID = xml:get_attr_s("id", AddAttrs),
ARE = xml:get_attr_s("re", AddAttrs),
case xml:get_attr_s("type", AddAttrs) of
"" -> add_regexp(VH, AID, ARE);
ATP -> add_regexp(VH, AID, ARE, ATP)
end;
_ -> ok
end,
case xml:get_subtag(SubEl, "del") of
{xmlelement, "del", DelAttrs, _} ->
DID = xml:get_attr_s("id", DelAttrs),
case xml:get_attr_s("re", DelAttrs) of
"" -> del_regexp(VH, DID);
DRE -> del_regexp(VH, DID, DRE)
end;
_ -> ok
end,
case xml:get_subtag(SubEl, "dellogs") of
{xmlelement, "dellogs", _, _} -> purge_logs();
_ -> ok
end,
case xml:get_subtag(SubEl, "delrules") of
{xmlelement, "delrules", _, _} -> purge_regexps(VH);
_ -> ok
end,
IQ#iq{type = result, sub_el = []};
_ ->
IQ#iq{type = error, sub_el = [SubEl, ?ERR_NOT_ALLOWED]}
end
end.

261
src/mod_support.erl Normal file
View File

@ -0,0 +1,261 @@
%%% ====================================================================
%%% This software is copyright 2006-2009, ProcessOne.
%%%
%%% mod_support
%%% allow automatic build of support archive to be sent to Process-One
%%%
%%% @copyright 2006-2009 ProcessOne
%%% @author Christophe Romain <christophe.romain@process-one.net>
%%% [http://www.process-one.net/]
%%% @version {@vsn}, {@date} {@time}
%%% @end
%%% ====================================================================
-module(mod_support).
-author('christophe.romain@process-one.net').
-vsn('$Id: mod_support.erl 856 2009-09-21 18:46:38Z jpcarlino $').
-behaviour(gen_mod).
%-behaviour(gen_server).
% module functions
-export([start/2,stop/1,is_loaded/0,loop/1,dump/0]).
-compile(export_all).
-include("ejabberd.hrl").
-include("jlib.hrl").
-include("licence.hrl").
-include_lib("kernel/include/file.hrl").
-define(LOG_FETCH_SIZE, 1000000).
-define(RPC_TIMEOUT, 10000). % 10
-define(MAX_FILE_SIZE, 2147483648). %%2Gb
start(Host, Opts) ->
case ?IS_VALID of
true ->
case gen_mod:get_opt(dump_freq, Opts, 0) of
0 -> no_dump;
Freq -> spawn(?MODULE, loop, [Freq*60000])
end,
ok;
false ->
not_started
end.
stop(Host) ->
ok.
is_loaded() ->
ok.
loop(Timeout) ->
receive
quit -> ok
after Timeout ->
Dump = dump(),
BaseName = get_base_name(),
%%{Data,EjabberdLog,SaslLog,ECrash} = Dump,
write_logs(tuple_to_list(Dump),BaseName,["_memory.bin",
"_ejabberd.log.gz",
"_sasl.log.gz",
"_erlang_crash_dump.log.gz"]),
loop(Timeout)
end.
get_base_name() ->
{{Y,M,D},{Hr,Mn,_Sc}} = calendar:local_time(),
case os:getenv("EJABBERD_LOG_PATH") of
false ->
filename:join(filename:dirname(filename:absname("")),
lists:flatten(io_lib:format("~b~b~b~b~b",[Y,M,D,Hr,Mn])));
Path ->
filename:join(filename:dirname(Path),
lists:flatten(io_lib:format("~b~b~b~b~b",[Y,M,D,Hr,Mn])))
end.
write_logs([BinaryData|T],BaseName,[Filename|Filenames]) ->
Log = BaseName++Filename,
file:write_file(Log, BinaryData),
write_logs(T,BaseName,Filenames);
write_logs([],BaseName,_)-> ok.
dump() ->
Dump = lists:map(fun(LogFile) ->
Content = case file:open(LogFile,[read,raw]) of
{ok, IO} ->
Size = case file:read_file_info(LogFile) of
{ok, FileInfo} -> FileInfo#file_info.size;
_ -> ?LOG_FETCH_SIZE
end,
case Size>?MAX_FILE_SIZE of
true -> io_lib:format("File ~s is too big: ~p bytes.",[LogFile, Size]);
false ->
if Size>?LOG_FETCH_SIZE ->
file:position(IO, Size-?LOG_FETCH_SIZE),
case file:read(IO, ?LOG_FETCH_SIZE) of
{ok, Data1} -> Data1;
Error1 -> io_lib:format("can not read log file (~s): ~p",[LogFile, Error1])
end;
true ->
case file:read(IO, Size) of
{ok, Data2} -> Data2;
Error2 -> io_lib:format("can not read log file (~s): ~p",[LogFile, Error2])
end
end
end;
{error, Reason} ->
io_lib:format("can not open log file (~s): ~p",[LogFile, Reason])
end,
zlib:gzip(list_to_binary(Content))
end, [ejabberd_logs(), sasl_logs(), erl_crash()]),
NodeState = get_node_state(),
list_to_tuple([NodeState|Dump]).
ejabberd_logs() ->
LogPath = case application:get_env(log_path) of
{ok, Path} ->
Path;
undefined ->
case os:getenv("EJABBERD_LOG_PATH") of
false -> ?LOG_PATH;
Path -> Path
end
end.
sasl_logs() ->
case os:getenv("SASL_LOG_PATH") of
false -> filename:join([filename:dirname(ejabberd_logs()),"sasl.log"]);
Path -> Path
end.
erl_crash() ->
LogsDir = filename:dirname(ejabberd_logs()),
CrashDumpWildcard = filename:join([LogsDir,"erl_crash*dump"]),
FileName = case filelib:wildcard(CrashDumpWildcard) of
[Files] -> [LastFile|T] = lists:reverse([Files]),
LastFile;
_ -> case os:getenv("ERL_CRASH_DUMP") of
false -> "erl_crash.dump";
Path -> Path
end
end.
proc_info(Pid) ->
Info = process_info(Pid),
lists:map(fun(Elem) ->
List = proplists:get_value(Elem, Info),
{Elem, size(term_to_binary(List))}
end, [messages, dictionary])
++ [X || X <- Info,
lists:member(element(1,X),
[heap_size,stack_size,reductions,links,status,initial_call,current_function])].
environment() ->
{ok, KE} = application:get_key(kernel,env),
{ok, EE} = application:get_key(ejabberd,env),
Env = [{inetrc, os:getenv("ERL_INETRC")},
{sopath, os:getenv("EJABBERD_SO_PATH")},
{maxports, os:getenv("ERL_MAX_PORTS")},
{maxtables, os:getenv("ERL_MAX_ETS_TABLES")},
{crashdump, os:getenv("ERL_CRASH_DUMP")},
{archdir, os:getenv("ARCHDIR")},
{mnesia, mnesia:system_info(all)}],
Args = [{args, init:get_arguments()}, {plain, init:get_plain_arguments()}],
KE++EE++Env++Args.
memtop(N) ->
E = lists:sublist(lists:reverse(lists:keysort(2,lists:map(fun(Tab) -> {Tab, ets:info(Tab,memory)} end, ets:all()))),N),
M = lists:sublist(lists:reverse(lists:keysort(2,lists:map(fun(Tab) -> {Tab, mnesia:table_info(Tab,memory)} end, mnesia:system_info(tables)))),N),
E++M.
maxmsgqueue() ->
lists:max(lists:map(fun(Pid) -> proplists:get_value(message_queue_len,process_info(Pid)) end, erlang:processes())).
msgqueue(N) ->
lists:filter(fun(L) -> proplists:get_value(message_queue_len, L) > N
end, lists:map(fun(Pid) -> process_info(Pid) end, erlang:processes())).
%lists:sublist(lists:reverse(lists:keysort(2,lists:map(fun(Pid) -> {E,L} = process_info(Pid, dictionary), {E,length(L)} end, erlang:processes()))), 10)
%%Entry point to invoke mod_support via command line.
%%Example: erl -sname debug@localhost -s mod_support report ejabberd@localhost
%%See issue #TECH-286.
report(Node) ->
[NodeId|T]=Node,
UploadResult = force_load_code_into_node(NodeId, ?MODULE),
case UploadResult of
ok -> NodeState = rpc:call(NodeId,mod_support,get_node_state,[],?RPC_TIMEOUT),
Dump = rpc:call(NodeId,mod_support,dump,[],?RPC_TIMEOUT),
BaseName = get_base_name(),
%%{Data,EjabberdLog,SaslLog,ECrash} = Dump,
write_logs(tuple_to_list(Dump),BaseName,["_memory.bin",
"_ejabberd.log.gz",
"_sasl.log.gz",
"_erlang_crash_dump.log.gz"]),
error_logger:info_msg("State in node ~p was written to log~n",[NodeId]),
error_logger:info_msg("Unloading module ~s from node ~p. ",[?MODULE,NodeId]),
force_unload_code_from_node(NodeId, ?MODULE);
_ -> error_logger:info_msg("Error uploading module ~s from node ~p~n",[?MODULE,NodeId])
end.
%%Load Module into the ejabberd Node specified.
force_load_code_into_node(Node, Module) ->
CodeFile = code:where_is_file(atom_to_list(Module)++".beam"),
case file:read_file(CodeFile) of
{ok, Code} ->
rpc:call(Node, code, purge, [Module], ?RPC_TIMEOUT),
rpc:call(Node, code, delete, [Module], ?RPC_TIMEOUT),
case rpc:call(Node, code, load_binary, [Module, CodeFile, Code], ?RPC_TIMEOUT) of
{module, _} ->
error_logger:info_msg("Loading ~s module into ~p : success ~n", [Module,Node]),
rpc:block_call(Node, Module, is_loaded, [], ?RPC_TIMEOUT);
{error, badfile} ->
error_logger:info_msg("Loading ~s module into ~p : incorrect format ~n", [Module,Node]),
{error, badfile};
{error, not_purged} ->
% this should never happen anyway..
error_logger:info_msg("Loading ~s module into ~p : old code already exists ~n", [Module,Node]),
{error, not_purged};
{badrpc, Reason} ->
error_logger:info_msg("Loading ~s module into ~p: badrpc ~p ~n", [Module,Node,Reason]),
{badrpc, Reason}
end;
Error ->
error_logger:error_msg("Cannot read module file ~s ~p : ~p ~n", [Module, CodeFile, Error]),
Error
end.
%%Unload erlang Module from the Node specified. Used to ensure cleanup after rpc calls.
force_unload_code_from_node(Node, Module) ->
rpc:call(Node, code, purge, [Module], ?RPC_TIMEOUT),
rpc:call(Node, code, delete, [Module], ?RPC_TIMEOUT).
%%Retrieve system state and pack it into Data
%%TODO enhance state info. See #TECH-286.
get_node_state() ->
Mem = erlang:memory(),
Ets = lists:map(fun(Tab) -> ets:info(Tab) end, ets:all()),
Mnesia = lists:map(fun(Tab) -> mnesia:table_info(Tab,all) end, mnesia:system_info(tables)),
Procs = lists:map(fun(Pid) -> proc_info(Pid) end, erlang:processes()),
Data = term_to_binary({Mem, Ets, Mnesia, Procs}).
crash_dump() ->
SystemInfo = [erlang:system_info(X) || X<-[info,loaded,procs]],
[zlib:gzip(list_to_binary(lists:flatten(SystemInfo)))].
crash_dump(Node) ->
[NodeId|T]=Node,
UploadResult = force_load_code_into_node(NodeId, ?MODULE),
case UploadResult of
ok -> Dump = rpc:call(NodeId,mod_support,crash_dump,[],?RPC_TIMEOUT),
BaseName = get_base_name(),
write_logs(Dump,BaseName,["_realtime_crash_dump.gz"]),
error_logger:info_msg("Unloading module ~s from node ~p. ",[?MODULE,NodeId]),
force_unload_code_from_node(NodeId, ?MODULE);
_ -> error_logger:info_msg("Error uploading module ~s from node ~p~n",[?MODULE,NodeId])
end.