]> granicus.if.org Git - ejabberd/commitdiff
Implement database backend interface for MUC, BOSH and auth_anonyous
authorEvgeniy Khramtsov <ekhramtsov@process-one.net>
Fri, 13 Jan 2017 09:03:39 +0000 (12:03 +0300)
committerEvgeniy Khramtsov <ekhramtsov@process-one.net>
Fri, 13 Jan 2017 09:03:39 +0000 (12:03 +0300)
22 files changed:
include/ejabberd_router.hrl [new file with mode: 0644]
include/mod_muc.hrl
include/mod_muc_room.hrl
src/ejabberd_admin.erl
src/ejabberd_auth_anonymous.erl
src/ejabberd_sm.erl
src/gen_mod.erl
src/mod_bosh.erl
src/mod_bosh_mnesia.erl [new file with mode: 0644]
src/mod_http_upload.erl
src/mod_mam.erl
src/mod_muc.erl
src/mod_muc_admin.erl
src/mod_muc_log.erl
src/mod_muc_mnesia.erl
src/mod_muc_riak.erl
src/mod_muc_room.erl
src/mod_muc_sql.erl
src/mod_offline.erl
src/mod_pubsub.erl
src/mod_roster.erl
src/mod_shared_roster.erl

diff --git a/include/ejabberd_router.hrl b/include/ejabberd_router.hrl
new file mode 100644 (file)
index 0000000..8de23c4
--- /dev/null
@@ -0,0 +1,6 @@
+-type local_hint() :: undefined | integer() | {apply, atom(), atom()}.
+
+-record(route, {domain :: binary(),
+               server_host :: binary(),
+               pid :: undefined | pid(),
+               local_hint :: local_hint()}).
index 8d0c1b09decfc94554d128aec924e531e9fc98ba..b2cdb9544740c014651f31e37ebe4b35d09efd3a 100644 (file)
                                                    {'_', binary()},
                    opts = [] :: list() | '_'}).
 
--record(muc_online_room,
-        {name_host = {<<"">>, <<"">>} :: {binary(), binary()} | '$1' |
-                                         {'_', binary()} | '_',
-         pid = self() :: pid() | '$2' | '_' | '$1'}).
-
 -record(muc_registered,
         {us_host = {{<<"">>, <<"">>}, <<"">>} :: {{binary(), binary()}, binary()} | '$1',
          nick = <<"">> :: binary()}).
+
+-record(muc_online_room,
+        {name_host :: {binary(), binary()} | '$1' | {'_', binary()} | '_',
+         pid :: pid() | '$2' | '_' | '$1'}).
+
+-record(muc_online_users, {us :: {binary(), binary()},
+                           resource :: binary() | '_',
+                           room :: binary() | '_' | '$1',
+                           host :: binary() | '_' | '$2'}).
index dd414a8d8bee6ecfce64ffc6b2f9cab210ef63c2..4fbd1466ede4eb4f3903db1645792fcb04f1ca16 100644 (file)
     room_shaper             = none :: shaper:shaper(),
     room_queue              = queue:new() :: ?TQUEUE
 }).
-
--record(muc_online_users, {us = {<<>>, <<>>} :: {binary(), binary()},
-                           resource = <<>> :: binary() | '_',
-                           room = <<>> :: binary() | '_' | '$1',
-                           host = <<>> :: binary() | '_' | '$2'}).
-
--type muc_online_users() :: #muc_online_users{}.
index 9592c2702ec8fea8a300307852021235147e1fda..cf89439b2f7ce91997093d8cf9f03bd0cf8f27ac 100644 (file)
@@ -343,7 +343,7 @@ send_service_message_all_mucs(Subject, AnnouncementText) ->
       fun(ServerHost) ->
              MUCHost = gen_mod:get_module_opt_host(
                          ServerHost, mod_muc, <<"conference.@HOST@">>),
-             mod_muc:broadcast_service_message(MUCHost, Message)
+             mod_muc:broadcast_service_message(ServerHost, MUCHost, Message)
       end,
       ?MYHOSTS).
 
index e0c4d471f0aaa42042cf3b86c8bae618c76fdb65..d8947033ae2b98374c43709e3fe4058e9152d3f4 100644 (file)
 -include("logger.hrl").
 -include("jid.hrl").
 
-%% Create the anonymous table if at least one virtual host has anonymous features enabled
-%% Register to login / logout events
--record(anonymous, {us = {<<"">>, <<"">>} :: {binary(), binary()},
-                    sid = ejabberd_sm:make_sid() :: ejabberd_sm:sid()}).
-
 start(Host) ->
-    %% TODO: Check cluster mode
-    ejabberd_mnesia:create(?MODULE, anonymous, [{ram_copies, [node()]},
-                                   {type, bag},
-                                   {attributes, record_info(fields, anonymous)}]),
-    %% The hooks are needed to add / remove users from the anonymous tables
     ejabberd_hooks:add(sm_register_connection_hook, Host,
                       ?MODULE, register_connection, 100),
     ejabberd_hooks:add(sm_remove_connection_hook, Host,
@@ -119,56 +109,33 @@ allow_multiple_connections(Host) ->
       fun(V) when is_boolean(V) -> V end,
       false).
 
-%% Check if user exist in the anonymus database
 anonymous_user_exist(User, Server) ->
-    LUser = jid:nodeprep(User),
-    LServer = jid:nameprep(Server),
-    US = {LUser, LServer},
-    case catch mnesia:dirty_read({anonymous, US}) of
-       [] ->
-           false;
-       [_H|_T] ->
-           true
-    end.
-
-%% Remove connection from Mnesia tables
-remove_connection(SID, LUser, LServer) ->
-    US = {LUser, LServer},
-    F = fun () -> mnesia:delete_object({anonymous, US, SID})
-       end,
-    mnesia:transaction(F).
+    lists:any(
+      fun({_LResource, Info}) ->
+             proplists:get_value(auth_module, Info) == ?MODULE
+      end, ejabberd_sm:get_user_info(User, Server)).
 
 %% Register connection
 -spec register_connection(ejabberd_sm:sid(), jid(), ejabberd_sm:info()) -> ok.
-register_connection(SID,
+register_connection(_SID,
                    #jid{luser = LUser, lserver = LServer}, Info) ->
-    AuthModule = proplists:get_value(auth_module, Info, undefined),
-    case AuthModule == (?MODULE) of
-      true ->
-         ejabberd_hooks:run(register_user, LServer,
-                            [LUser, LServer]),
-         US = {LUser, LServer},
-         mnesia:sync_dirty(fun () ->
-                                    mnesia:write(#anonymous{us = US,
-                                                            sid = SID})
-                            end);
-      false -> ok
+    case proplists:get_value(auth_module, Info) of
+       ?MODULE ->
+           ejabberd_hooks:run(register_user, LServer, [LUser, LServer]);
+       false ->
+           ok
     end.
 
 %% Remove an anonymous user from the anonymous users table
 -spec unregister_connection(ejabberd_sm:sid(), jid(), ejabberd_sm:info()) -> any().
-unregister_connection(SID,
-                     #jid{luser = LUser, lserver = LServer}, _) ->
-    purge_hook(anonymous_user_exist(LUser, LServer), LUser,
-              LServer),
-    remove_connection(SID, LUser, LServer).
-
-%% Launch the hook to purge user data only for anonymous users
-purge_hook(false, _LUser, _LServer) ->
-    ok;
-purge_hook(true, LUser, LServer) ->
-    ejabberd_hooks:run(anonymous_purge_hook, LServer,
-                      [LUser, LServer]).
+unregister_connection(_SID,
+                     #jid{luser = LUser, lserver = LServer}, Info) ->
+    case proplists:get_value(auth_module, Info) of
+       ?MODULE ->
+           ejabberd_hooks:run(remove_user, LServer, [LUser, LServer]);
+       _ ->
+           ok
+    end.
 
 %% ---------------------------------
 %% Specific anonymous auth functions
@@ -258,8 +225,6 @@ get_password_s(User, Server) ->
             Password
     end.
 
-%% Returns true if the user exists in the DB or if an anonymous user is logged
-%% under the given name
 is_user_exists(User, Server) ->
     anonymous_user_exist(User, Server).
 
index 327c79b6c24f32cd57a98f775bb45fbee7cd719b..5eb67114953cd17dcf61aff477a2dc5e68b7c262 100644 (file)
@@ -64,6 +64,7 @@
         user_resources/2,
         kick_user/2,
         get_session_pid/3,
+        get_user_info/2,
         get_user_info/3,
         get_user_ip/3,
         get_max_user_sessions/2,
@@ -215,6 +216,17 @@ get_user_ip(User, Server, Resource) ->
            proplists:get_value(ip, Session#session.info)
     end.
 
+-spec get_user_info(binary(), binary()) -> [{binary(), info()}].
+get_user_info(User, Server) ->
+    LUser = jid:nodeprep(User),
+    LServer = jid:nameprep(Server),
+    Mod = get_sm_backend(LServer),
+    Ss = online(Mod:get_sessions(LUser, LServer)),
+    [{LResource, [{node, node(Pid)}|Info]}
+     || #session{usr = {_, _, LResource},
+                info = Info,
+                sid = {_, Pid}} <- clean_session_list(Ss)].
+
 -spec get_user_info(binary(), binary(), binary()) -> info() | offline.
 
 get_user_info(User, Server, Resource) ->
@@ -228,9 +240,7 @@ get_user_info(User, Server, Resource) ->
        Ss ->
            Session = lists:max(Ss),
            Node = node(element(2, Session#session.sid)),
-           Conn = proplists:get_value(conn, Session#session.info),
-           IP = proplists:get_value(ip, Session#session.info),
-           [{node, Node}, {conn, Conn}, {ip, IP}]
+           [{node, Node}|Session#session.info]
     end.
 
 -spec set_presence(sid(), binary(), binary(), binary(),
index f4aaaa4cbd52a5c0275c605ca784e66f9700b717..1ee949912627e3fc0bb8cea21bfd89ff9d76af90 100644 (file)
 
 -export([start/0, start_module/2, start_module/3,
         stop_module/2, stop_module_keep_config/2, get_opt/3,
-        get_opt/4, get_opt_host/3, db_type/2, db_type/3,
+        get_opt/4, get_opt_host/3, opt_type/1,
         get_module_opt/4, get_module_opt/5, get_module_opt_host/3,
         loaded_modules/1, loaded_modules_with_opts/1,
         get_hosts/2, get_module_proc/2, is_loaded/2,
         start_modules/0, start_modules/1, stop_modules/0, stop_modules/1,
-        opt_type/1, db_mod/2, db_mod/3]).
+        db_mod/2, db_mod/3, ram_db_mod/2, ram_db_mod/3,
+        db_type/2, db_type/3, ram_db_type/2, ram_db_type/3]).
 
 %%-export([behaviour_info/1]).
 
@@ -424,6 +425,43 @@ db_mod(Host, Module) when is_binary(Host) orelse Host == global ->
 db_mod(Host, Opts, Module) when is_list(Opts) ->
     db_mod(db_type(Host, Opts, Module), Module).
 
+-spec ram_db_type(binary() | global, module()) -> db_type();
+                (opts(), module()) -> db_type().
+ram_db_type(Opts, Module) when is_list(Opts) ->
+    ram_db_type(global, Opts, Module);
+ram_db_type(Host, Module) when is_atom(Module) ->
+    case catch Module:mod_opt_type(ram_db_type) of
+       F when is_function(F) ->
+           case get_module_opt(Host, Module, ram_db_type, F) of
+               undefined -> ejabberd_config:default_ram_db(Host, Module);
+               Type -> Type
+           end;
+       _ ->
+           undefined
+    end.
+
+-spec ram_db_type(binary(), opts(), module()) -> db_type().
+ram_db_type(Host, Opts, Module) ->
+    case catch Module:mod_opt_type(ram_db_type) of
+       F when is_function(F) ->
+           case get_opt(ram_db_type, Opts, F) of
+               undefined -> ejabberd_config:default_ram_db(Host, Module);
+               Type -> Type
+           end;
+       _ ->
+           undefined
+    end.
+
+-spec ram_db_mod(binary() | global | db_type(), module()) -> module().
+ram_db_mod(Type, Module) when is_atom(Type), Type /= global ->
+    list_to_atom(atom_to_list(Module) ++ "_" ++ atom_to_list(Type));
+ram_db_mod(Host, Module) when is_binary(Host) orelse Host == global ->
+    ram_db_mod(ram_db_type(Host, Module), Module).
+
+-spec ram_db_mod(binary() | global, opts(), module()) -> module().
+ram_db_mod(Host, Opts, Module) when is_list(Opts) ->
+    ram_db_mod(ram_db_type(Host, Opts, Module), Module).
+
 -spec loaded_modules(binary()) -> [atom()].
 
 loaded_modules(Host) ->
index 038218739a6b169231cf245b33b5f60c0aacd660..43a00edce58a1f477527142689edd2b6d3800ae7 100644 (file)
 
 %%-define(ejabberd_debug, true).
 
--behaviour(gen_server).
 -behaviour(gen_mod).
 
 -export([start_link/0]).
 -export([start/2, stop/1, process/2, open_session/2,
         close_session/1, find_session/1]).
 
--export([init/1, handle_call/3, handle_cast/2,
-        handle_info/2, terminate/2, code_change/3,
-        depends/2, mod_opt_type/1]).
+-export([depends/2, mod_opt_type/1]).
 
 -include("ejabberd.hrl").
 -include("logger.hrl").
 -include_lib("stdlib/include/ms_transform.hrl").
 -include("jlib.hrl").
-
 -include("ejabberd_http.hrl").
-
 -include("bosh.hrl").
 
--record(bosh, {sid = <<"">>      :: binary() | '_',
-               timestamp = p1_time_compat:timestamp() :: erlang:timestamp() | '_',
-               pid = self()      :: pid() | '$1'}).
-
--record(state, {}).
+-callback init() -> any().
+-callback open_session(binary(), pid()) -> any().
+-callback close_session(binary()) -> any().
+-callback find_session(binary()) -> {ok, pid()} | error.
 
 %%%----------------------------------------------------------------------
 %%% API
@@ -114,137 +108,35 @@ get_human_html_xmlel() ->
                                           "client that supports it.">>}]}]}]}.
 
 open_session(SID, Pid) ->
-    Session = #bosh{sid = SID, timestamp = p1_time_compat:timestamp(), pid = Pid},
-    lists:foreach(
-      fun(Node) when Node == node() ->
-             gen_server:call(?MODULE, {write, Session});
-        (Node) ->
-             cluster_send({?MODULE, Node}, {write, Session})
-      end, ejabberd_cluster:get_nodes()).
+    Mod = gen_mod:ram_db_mod(global, ?MODULE),
+    Mod:open_session(SID, Pid).
 
 close_session(SID) ->
-    case mnesia:dirty_read(bosh, SID) of
-       [Session] ->
-           lists:foreach(
-             fun(Node) when Node == node() ->
-                     gen_server:call(?MODULE, {delete, Session});
-                (Node) ->
-                     cluster_send({?MODULE, Node}, {delete, Session})
-             end, ejabberd_cluster:get_nodes());
-       [] ->
-           ok
-    end.
-
-write_session(#bosh{pid = Pid1, sid = SID, timestamp = T1} = S1) ->
-    case mnesia:dirty_read(bosh, SID) of
-       [#bosh{pid = Pid2, timestamp = T2} = S2] ->
-           if Pid1 == Pid2 ->
-                   mnesia:dirty_write(S1);
-              T1 < T2 ->
-                   cluster_send(Pid2, replaced),
-                   mnesia:dirty_write(S1);
-              true ->
-                   cluster_send(Pid1, replaced),
-                   mnesia:dirty_write(S2)
-           end;
-       [] ->
-           mnesia:dirty_write(S1)
-    end.
-
-delete_session(#bosh{sid = SID, pid = Pid1}) ->
-    case mnesia:dirty_read(bosh, SID) of
-       [#bosh{pid = Pid2}] ->
-           if Pid1 == Pid2 ->
-                   mnesia:dirty_delete(bosh, SID);
-              true ->
-                   ok
-           end;
-       [] ->
-           ok
-    end.
+    Mod = gen_mod:ram_db_mod(global, ?MODULE),
+    Mod:close_session(SID).
 
 find_session(SID) ->
-    case mnesia:dirty_read(bosh, SID) of
-        [#bosh{pid = Pid}] ->
-            {ok, Pid};
-        [] ->
-            error
-    end.
+    Mod = gen_mod:ram_db_mod(global, ?MODULE),
+    Mod:find_session(SID).
 
 start(Host, Opts) ->
-    setup_database(),
     start_jiffy(Opts),
     TmpSup = gen_mod:get_module_proc(Host, ?PROCNAME),
     TmpSupSpec = {TmpSup,
                  {ejabberd_tmp_sup, start_link, [TmpSup, ejabberd_bosh]},
                  permanent, infinity, supervisor, [ejabberd_tmp_sup]},
-    ProcSpec = {?MODULE,
-               {?MODULE, start_link, []},
-               transient, 2000, worker, [?MODULE]},
-    case supervisor:start_child(ejabberd_sup, ProcSpec) of
-       {ok, _} ->
-           supervisor:start_child(ejabberd_sup, TmpSupSpec);
-       {error, {already_started, _}} ->
-           supervisor:start_child(ejabberd_sup, TmpSupSpec);
-       Err ->
-           Err
-    end.
+    supervisor:start_child(ejabberd_sup, TmpSupSpec),
+    Mod = gen_mod:ram_db_mod(global, ?MODULE),
+    Mod:init().
 
 stop(Host) ->
     TmpSup = gen_mod:get_module_proc(Host, ?PROCNAME),
     supervisor:terminate_child(ejabberd_sup, TmpSup),
     supervisor:delete_child(ejabberd_sup, TmpSup).
 
-%%%===================================================================
-%%% gen_server callbacks
-%%%===================================================================
-init([]) ->
-    {ok, #state{}}.
-
-handle_call({write, Session}, _From, State) ->
-    Res = write_session(Session),
-    {reply, Res, State};
-handle_call({delete, Session}, _From, State) ->
-    Res = delete_session(Session),
-    {reply, Res, State};
-handle_call(_Request, _From, State) ->
-    Reply = ok,
-    {reply, Reply, State}.
-
-handle_cast(_Msg, State) ->
-    {noreply, State}.
-
-handle_info({write, Session}, State) ->
-    write_session(Session),
-    {noreply, State};
-handle_info({delete, Session}, State) ->
-    delete_session(Session),
-    {noreply, State};
-handle_info(_Info, State) ->
-    ?ERROR_MSG("got unexpected info: ~p", [_Info]),
-    {noreply, State}.
-
-terminate(_Reason, _State) ->
-    ok.
-
-code_change(_OldVsn, State, _Extra) ->
-    {ok, State}.
-
 %%%===================================================================
 %%% Internal functions
 %%%===================================================================
-setup_database() ->
-    case catch mnesia:table_info(bosh, attributes) of
-        [sid, pid] ->
-            mnesia:delete_table(bosh);
-        _ ->
-            ok
-    end,
-    ejabberd_mnesia:create(?MODULE, bosh,
-                       [{ram_copies, [node()]}, {local_content, true},
-                        {attributes, record_info(fields, bosh)}]),
-    mnesia:add_table_copy(bosh, node(), ram_copies).
-
 start_jiffy(Opts) ->
     case gen_mod:get_opt(json, Opts,
                          fun(false) -> false;
@@ -272,9 +164,6 @@ get_type(Hdrs) ->
             xml
     end.
 
-cluster_send(NodePid, Msg) ->
-    erlang:send(NodePid, Msg, [noconnect, nosuspend]).
-
 depends(_Host, _Opts) ->
     [].
 
@@ -292,5 +181,7 @@ mod_opt_type(max_pause) ->
     fun (I) when is_integer(I), I > 0 -> I end;
 mod_opt_type(prebind) ->
     fun (B) when is_boolean(B) -> B end;
+mod_opt_type(ram_db_type) ->
+    fun(T) -> ejabberd_config:v_db(?MODULE, T) end;
 mod_opt_type(_) ->
-    [json, max_concat, max_inactivity, max_pause, prebind].
+    [json, max_concat, max_inactivity, max_pause, prebind, ram_db_type].
diff --git a/src/mod_bosh_mnesia.erl b/src/mod_bosh_mnesia.erl
new file mode 100644 (file)
index 0000000..b61ef20
--- /dev/null
@@ -0,0 +1,163 @@
+%%%-------------------------------------------------------------------
+%%% Created : 12 Jan 2017 by Evgeny Khramtsov <ekhramtsov@process-one.net>
+%%%
+%%%
+%%% ejabberd, Copyright (C) 2002-2017   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.,
+%%% 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
+%%%
+%%%-------------------------------------------------------------------
+-module(mod_bosh_mnesia).
+
+-behaviour(gen_server).
+-behaviour(mod_bosh).
+
+%% mod_bosh API
+-export([init/0, open_session/2, close_session/1, find_session/1]).
+
+%% gen_server callbacks
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+        terminate/2, code_change/3]).
+
+-include("logger.hrl").
+
+-record(bosh, {sid = <<"">>      :: binary() | '_',
+               timestamp = p1_time_compat:timestamp() :: erlang:timestamp() | '_',
+               pid = self()      :: pid() | '$1'}).
+
+-record(state, {}).
+
+%%%===================================================================
+%%% API
+%%%===================================================================
+init() ->
+    case gen_server:start_link({local, ?MODULE}, ?MODULE, [], []) of
+       {ok, _Pid} ->
+           ok;
+       Err ->
+           Err
+    end.
+
+open_session(SID, Pid) ->
+    Session = #bosh{sid = SID, timestamp = p1_time_compat:timestamp(), pid = Pid},
+    lists:foreach(
+      fun(Node) when Node == node() ->
+             gen_server:call(?MODULE, {write, Session});
+        (Node) ->
+             cluster_send({?MODULE, Node}, {write, Session})
+      end, ejabberd_cluster:get_nodes()).
+
+close_session(SID) ->
+    case mnesia:dirty_read(bosh, SID) of
+       [Session] ->
+           lists:foreach(
+             fun(Node) when Node == node() ->
+                     gen_server:call(?MODULE, {delete, Session});
+                (Node) ->
+                     cluster_send({?MODULE, Node}, {delete, Session})
+             end, ejabberd_cluster:get_nodes());
+       [] ->
+           ok
+    end.
+
+find_session(SID) ->
+    case mnesia:dirty_read(bosh, SID) of
+        [#bosh{pid = Pid}] ->
+            {ok, Pid};
+        [] ->
+            error
+    end.
+
+%%%===================================================================
+%%% gen_server callbacks
+%%%===================================================================
+init([]) ->
+    setup_database(),
+    {ok, #state{}}.
+
+handle_call({write, Session}, _From, State) ->
+    Res = write_session(Session),
+    {reply, Res, State};
+handle_call({delete, Session}, _From, State) ->
+    Res = delete_session(Session),
+    {reply, Res, State};
+handle_call(_Request, _From, State) ->
+    Reply = ok,
+    {reply, Reply, State}.
+
+handle_cast(_Msg, State) ->
+    {noreply, State}.
+
+handle_info({write, Session}, State) ->
+    write_session(Session),
+    {noreply, State};
+handle_info({delete, Session}, State) ->
+    delete_session(Session),
+    {noreply, State};
+handle_info(_Info, State) ->
+    ?ERROR_MSG("got unexpected info: ~p", [_Info]),
+    {noreply, State}.
+
+terminate(_Reason, _State) ->
+    ok.
+
+code_change(_OldVsn, State, _Extra) ->
+    {ok, State}.
+
+%%%===================================================================
+%%% Internal functions
+%%%===================================================================
+write_session(#bosh{pid = Pid1, sid = SID, timestamp = T1} = S1) ->
+    case mnesia:dirty_read(bosh, SID) of
+       [#bosh{pid = Pid2, timestamp = T2} = S2] ->
+           if Pid1 == Pid2 ->
+                   mnesia:dirty_write(S1);
+              T1 < T2 ->
+                   cluster_send(Pid2, replaced),
+                   mnesia:dirty_write(S1);
+              true ->
+                   cluster_send(Pid1, replaced),
+                   mnesia:dirty_write(S2)
+           end;
+       [] ->
+           mnesia:dirty_write(S1)
+    end.
+
+delete_session(#bosh{sid = SID, pid = Pid1}) ->
+    case mnesia:dirty_read(bosh, SID) of
+       [#bosh{pid = Pid2}] ->
+           if Pid1 == Pid2 ->
+                   mnesia:dirty_delete(bosh, SID);
+              true ->
+                   ok
+           end;
+       [] ->
+           ok
+    end.
+
+cluster_send(NodePid, Msg) ->
+    erlang:send(NodePid, Msg, [noconnect, nosuspend]).
+
+setup_database() ->
+    case catch mnesia:table_info(bosh, attributes) of
+        [sid, pid] ->
+            mnesia:delete_table(bosh);
+        _ ->
+            ok
+    end,
+    ejabberd_mnesia:create(?MODULE, bosh,
+                       [{ram_copies, [node()]}, {local_content, true},
+                        {attributes, record_info(fields, bosh)}]),
+    mnesia:add_table_copy(bosh, node(), ram_copies).
index 37eaad27a37e35bd86e4e14c40dafc4fabeeb9f8..5b1d279a367d856c8b36983d2abb09dec5062480 100644 (file)
@@ -139,8 +139,6 @@ start(ServerHost, Opts) ->
                         true) of
        true ->
            ejabberd_hooks:add(remove_user, ServerHost, ?MODULE,
-                              remove_user, 50),
-           ejabberd_hooks:add(anonymous_purge_hook, ServerHost, ?MODULE,
                               remove_user, 50);
        false ->
            ok
@@ -162,8 +160,6 @@ stop(ServerHost) ->
                                true) of
        true ->
            ejabberd_hooks:delete(remove_user, ServerHost, ?MODULE,
-                                 remove_user, 50),
-           ejabberd_hooks:delete(anonymous_purge_hook, ServerHost, ?MODULE,
                                  remove_user, 50);
        false ->
            ok
index 0e2d77d9667fbcd2c2310efa787682fc8c9a147a..ca5930218bbde606cd961a66f991e261b27eca94 100644 (file)
@@ -104,8 +104,6 @@ start(Host, Opts) ->
                       get_room_config, 50),
     ejabberd_hooks:add(set_room_option, Host, ?MODULE,
                       set_room_option, 50),
-    ejabberd_hooks:add(anonymous_purge_hook, Host, ?MODULE,
-                      remove_user, 50),
     case gen_mod:get_opt(assume_mam_usage, Opts,
                         fun(B) when is_boolean(B) -> B end, false) of
        true ->
@@ -154,8 +152,6 @@ stop(Host) ->
                          get_room_config, 50),
     ejabberd_hooks:delete(set_room_option, Host, ?MODULE,
                          set_room_option, 50),
-    ejabberd_hooks:delete(anonymous_purge_hook, Host,
-                         ?MODULE, remove_user, 50),
     case gen_mod:get_module_opt(Host, ?MODULE, assume_mam_usage,
                                fun(B) when is_boolean(B) -> B end, false) of
        true ->
index 298749329cd675b36ec3d73b155a8d731a839f28..149c25f4622a44b8de643f654f3b2befe53551f3 100644 (file)
         process_register/1,
         process_muc_unique/1,
         process_mucsub/1,
-        broadcast_service_message/2,
+        broadcast_service_message/3,
         export/1,
         import_info/0,
         import/5,
         import_start/2,
         opts_to_binary/1,
+        find_online_room/2,
+        register_online_room/3,
+        get_online_rooms/1,
+        count_online_rooms/1,
+        register_online_user/4,
+        unregister_online_user/4,
+        count_online_rooms_by_user/3,
+        get_online_rooms_by_user/3,
         can_use_nick/4]).
 
 -export([init/1, handle_call/3, handle_cast/2,
@@ -63,7 +71,6 @@
 
 -include("ejabberd.hrl").
 -include("logger.hrl").
--include_lib("stdlib/include/ms_transform.hrl").
 -include("xmpp.hrl").
 -include("mod_muc.hrl").
 
 -callback get_rooms(binary(), binary()) -> [#muc_room{}].
 -callback get_nick(binary(), binary(), jid()) -> binary() | error.
 -callback set_nick(binary(), binary(), jid(), binary()) -> {atomic, ok | false}.
+-callback register_online_room(binary(), binary(), pid()) -> any().
+-callback unregister_online_room(binary(), binary(), pid()) -> any().
+-callback find_online_room(binary(), binary()) -> {ok, pid()} | error.
+-callback get_online_rooms(binary(), undefined | rsm_set()) -> [{binary(), binary(), pid()}].
+-callback count_online_rooms(binary()) -> non_neg_integer().
+-callback rsm_supported() -> boolean().
+-callback register_online_user(ljid(), binary(), binary()) -> any().
+-callback unregister_online_user(ljid(), binary(), binary()) -> any().
+-callback count_online_rooms_by_user(binary(), binary()) -> non_neg_integer().
+-callback get_online_rooms_by_user(binary(), binary()) -> [{binary(), binary()}].
+-callback handle_event(term()) -> any().
 
 %%====================================================================
 %% API
@@ -114,16 +132,17 @@ depends(_Host, _Opts) ->
     [{mod_mam, soft}].
 
 shutdown_rooms(Host) ->
+    RMod = gen_mod:ram_db_mod(Host, ?MODULE),
     MyHost = gen_mod:get_module_opt_host(Host, mod_muc,
                                         <<"conference.@HOST@">>),
-    Rooms = mnesia:dirty_select(muc_online_room,
-                               [{#muc_online_room{name_host = '$1',
-                                                  pid = '$2'},
-                                 [{'==', {element, 2, '$1'}, MyHost},
-                                  {'==', {node, '$2'}, node()}],
-                                 ['$2']}]),
-    [Pid ! shutdown || Pid <- Rooms],
-    Rooms.
+    Rooms = RMod:get_online_rooms(MyHost, undefined),
+    lists:filter(
+      fun({_, _, Pid}) when node(Pid) == node() ->
+             Pid ! shutdown,
+             true;
+        (_) ->
+             false
+      end, Rooms).
 
 %% This function is called by a room in three situations:
 %% A) The owner of the room destroyed it
@@ -165,6 +184,48 @@ can_use_nick(ServerHost, Host, JID, Nick) ->
     Mod = gen_mod:db_mod(LServer, ?MODULE),
     Mod:can_use_nick(LServer, Host, JID, Nick).
 
+-spec find_online_room(binary(), binary()) -> {ok, pid()} | error.
+find_online_room(Room, Host) ->
+    ServerHost = ejabberd_router:host_of_route(Host),
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:find_online_room(Room, Host).
+
+-spec register_online_room(binary(), binary(), pid()) -> any().
+register_online_room(Room, Host, Pid) ->
+    ServerHost = ejabberd_router:host_of_route(Host),
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:register_online_room(Room, Host, Pid).
+
+-spec get_online_rooms(binary()) -> [{binary(), binary(), pid()}].
+get_online_rooms(Host) ->
+    ServerHost = ejabberd_router:host_of_route(Host),
+    get_online_rooms(ServerHost, Host).
+
+-spec count_online_rooms(binary()) -> non_neg_integer().
+count_online_rooms(Host) ->
+    ServerHost = ejabberd_router:host_of_route(Host),
+    count_online_rooms(ServerHost, Host).
+
+-spec register_online_user(binary(), ljid(), binary(), binary()) -> any().
+register_online_user(ServerHost, LJID, Name, Host) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:register_online_user(LJID, Name, Host).
+
+-spec unregister_online_user(binary(), ljid(), binary(), binary()) -> any().
+unregister_online_user(ServerHost, LJID, Name, Host) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:unregister_online_user(LJID, Name, Host).
+
+-spec count_online_rooms_by_user(binary(), binary(), binary()) -> non_neg_integer().
+count_online_rooms_by_user(ServerHost, LUser, LServer) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:count_online_rooms_by_user(LUser, LServer).
+
+-spec get_online_rooms_by_user(binary(), binary(), binary()) -> [{binary(), binary()}].
+get_online_rooms_by_user(ServerHost, LUser, LServer) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:get_online_rooms_by_user(LUser, LServer).
+
 %%====================================================================
 %% gen_server callbacks
 %%====================================================================
@@ -175,16 +236,9 @@ init([Host, Opts]) ->
     MyHost = gen_mod:get_opt_host(Host, Opts,
                                  <<"conference.@HOST@">>),
     Mod = gen_mod:db_mod(Host, Opts, ?MODULE),
+    RMod = gen_mod:ram_db_mod(Host, Opts, ?MODULE),
     Mod:init(Host, [{host, MyHost}|Opts]),
-    update_tables(),
-    ejabberd_mnesia:create(?MODULE, muc_online_room,
-                       [{ram_copies, [node()]},
-                        {type, ordered_set},
-                        {attributes, record_info(fields, muc_online_room)}]),
-    mnesia:add_table_copy(muc_online_room, node(), ram_copies),
-    catch ets:new(muc_online_users, [bag, named_table, public, {keypos, 2}]),
-    clean_table_from_bad_node(node(), MyHost),
-    mnesia:subscribe(system),
+    RMod:init(Host, [{host, MyHost}|Opts]),
     Access = gen_mod:get_opt(access, Opts,
                              fun acl:access_rules_validator/1, all),
     AccessCreate = gen_mod:get_opt(access_create, Opts,
@@ -298,7 +352,8 @@ handle_call({create, Room, From, Nick, Opts}, _From,
                  Room, HistorySize,
                  RoomShaper, From,
                  Nick, NewOpts),
-    register_room(Host, Room, Pid),
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:register_online_room(Room, Host, Pid),
     {reply, ok, State}.
 
 handle_cast(_Msg, State) -> {noreply, State}.
@@ -317,18 +372,15 @@ handle_info({route, From, To, Packet},
            ok
     end,
     {noreply, State};
-handle_info({room_destroyed, RoomHost, Pid}, State) ->
-    F = fun () ->
-               mnesia:delete_object(#muc_online_room{name_host =
-                                                         RoomHost,
-                                                     pid = Pid})
-       end,
-    mnesia:transaction(F),
+handle_info({room_destroyed, {Room, Host}, Pid}, State) ->
+    ServerHost = State#state.server_host,
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:unregister_online_room(Room, Host, Pid),
     {noreply, State};
-handle_info({mnesia_system_event, {mnesia_down, Node}}, State) ->
-    clean_table_from_bad_node(Node),
-    {noreply, State};
-handle_info(_Info, State) -> {noreply, State}.
+handle_info(Event, #state{server_host = LServer} = State) ->
+    RMod = gen_mod:ram_db_mod(LServer, ?MODULE),
+    RMod:handle_event(Event),
+    {noreply, State}.
 
 terminate(_Reason, #state{host = MyHost}) ->
     ejabberd_router:unregister_route(MyHost),
@@ -374,7 +426,7 @@ do_route1(Host, ServerHost, Access, _HistorySize, _RoomShaper,
            case acl:match_rule(ServerHost, AccessAdmin, From) of
                allow ->
                    Msg = xmpp:get_text(Body),
-                   broadcast_service_message(Host, Msg);
+                   broadcast_service_message(ServerHost, Host, Msg);
                deny ->
                    ErrText = <<"Only service administrators are allowed "
                                "to send service messages">>,
@@ -390,8 +442,9 @@ do_route1(Host, ServerHost, Access, HistorySize, RoomShaper,
          From, To, Packet, DefRoomOpts) ->
     {_AccessRoute, AccessCreate, _AccessAdmin, _AccessPersistent} = Access,
     {Room, _, Nick} = jid:tolower(To),
-    case mnesia:dirty_read(muc_online_room, {Room, Host}) of
-       [] ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    case RMod:find_online_room(Room, Host) of
+       error ->
            case is_create_request(Packet) of
                true ->
                    case check_user_can_create_room(
@@ -402,7 +455,7 @@ do_route1(Host, ServerHost, Access, HistorySize, RoomShaper,
                                          Host, ServerHost, Access,
                                          Room, HistorySize,
                                          RoomShaper, From, Nick, DefRoomOpts),
-                           register_room(Host, Room, Pid),
+                           RMod:register_online_room(Room, Host, Pid),
                            mod_muc_room:route(Pid, From, Nick, Packet),
                            ok;
                        false ->
@@ -417,8 +470,7 @@ do_route1(Host, ServerHost, Access, HistorySize, RoomShaper,
                    Err = xmpp:err_item_not_found(ErrText, Lang),
                    ejabberd_router:route_error(To, From, Packet, Err)
            end;
-       [R] ->
-           Pid = R#muc_online_room.pid,
+       {ok, Pid} ->
            ?DEBUG("MUC: send to process ~p~n", [Pid]),
            mod_muc_room:route(Pid, From, Nick, Packet),
            ok
@@ -462,15 +514,20 @@ process_disco_info(#iq{type = set, lang = Lang} = IQ) ->
 process_disco_info(#iq{type = get, to = To, lang = Lang,
                       sub_els = [#disco_info{node = <<"">>}]} = IQ) ->
     ServerHost = ejabberd_router:host_of_route(To#jid.lserver),
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
     X = ejabberd_hooks:run_fold(disco_info, ServerHost, [],
                                [ServerHost, ?MODULE, <<"">>, Lang]),
     MAMFeatures = case gen_mod:is_loaded(ServerHost, mod_mam) of
                      true -> [?NS_MAM_TMP, ?NS_MAM_0, ?NS_MAM_1];
                      false -> []
                  end,
+    RSMFeatures = case RMod:rsm_supported() of
+                     true -> [?NS_RSM];
+                     false -> []
+                 end,
     Features = [?NS_DISCO_INFO, ?NS_DISCO_ITEMS,
-               ?NS_REGISTER, ?NS_MUC, ?NS_RSM,
-               ?NS_VCARD, ?NS_MUCSUB, ?NS_MUC_UNIQUE | MAMFeatures],
+               ?NS_REGISTER, ?NS_MUC, ?NS_VCARD, ?NS_MUCSUB, ?NS_MUC_UNIQUE
+               | RSMFeatures ++ MAMFeatures],
     Identity = #identity{category = <<"conference">>,
                         type = <<"text">>,
                         name = translate:translate(Lang, <<"Chatrooms">>)},
@@ -497,7 +554,8 @@ process_disco_items(#iq{type = get, from = From, to = To, lang = Lang,
                           ServerHost, ?MODULE, max_rooms_discoitems,
                           fun(I) when is_integer(I), I>=0 -> I end,
                           100),
-    case iq_disco_items(Host, From, Lang, MaxRoomsDiscoItems, Node, RSM) of
+    case iq_disco_items(ServerHost, Host, From, Lang,
+                       MaxRoomsDiscoItems, Node, RSM) of
        {error, Err} ->
            xmpp:make_error(IQ, Err);
        {result, Result} ->
@@ -564,20 +622,22 @@ get_rooms(ServerHost, Host) ->
 
 load_permanent_rooms(Host, ServerHost, Access,
                     HistorySize, RoomShaper) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
     lists:foreach(
       fun(R) ->
-               {Room, Host} = R#muc_room.name_host,
-               case mnesia:dirty_read(muc_online_room, {Room, Host}) of
-                   [] ->
-                       {ok, Pid} = mod_muc_room:start(Host,
-                               ServerHost, Access, Room,
-                               HistorySize, RoomShaper,
-                               R#muc_room.opts),
-                       register_room(Host, Room, Pid);
-                   _ -> ok
-               end
-       end,
-       get_rooms(ServerHost, Host)).
+             {Room, Host} = R#muc_room.name_host,
+             case RMod:find_online_room(Room, Host) of
+                 error ->
+                     {ok, Pid} = mod_muc_room:start(Host,
+                                                    ServerHost, Access, Room,
+                                                    HistorySize, RoomShaper,
+                                                    R#muc_room.opts),
+                     RMod:register_online_room(Room, Host, Pid);
+                 {ok, _} ->
+                     ok
+             end
+      end,
+      get_rooms(ServerHost, Host)).
 
 start_new_room(Host, ServerHost, Access, Room,
            HistorySize, RoomShaper, From,
@@ -594,19 +654,12 @@ start_new_room(Host, ServerHost, Access, Room,
                HistorySize, RoomShaper, Opts)
     end.
 
-register_room(Host, Room, Pid) ->
-    F = fun() ->
-           mnesia:write(#muc_online_room{name_host = {Room, Host},
-                   pid = Pid})
-    end,
-    mnesia:transaction(F).
-
--spec iq_disco_items(binary(), jid(), binary(), integer(), binary(),
+-spec iq_disco_items(binary(), binary(), jid(), binary(), integer(), binary(),
                     rsm_set() | undefined) ->
                            {result, disco_items()} | {error, stanza_error()}.
-iq_disco_items(Host, From, Lang, MaxRoomsDiscoItems, Node, RSM)
+iq_disco_items(ServerHost, Host, From, Lang, MaxRoomsDiscoItems, Node, RSM)
   when Node == <<"">>; Node == <<"nonemptyrooms">>; Node == <<"emptyrooms">> ->
-    Count = get_vh_rooms_count(Host),
+    Count = count_online_rooms(ServerHost, Host),
     Query = if Node == <<"">>, RSM == undefined, Count > MaxRoomsDiscoItems ->
                    {get_disco_item, only_non_empty, From, Lang};
               Node == <<"nonemptyrooms">> ->
@@ -616,7 +669,13 @@ iq_disco_items(Host, From, Lang, MaxRoomsDiscoItems, Node, RSM)
               true ->
                    {get_disco_item, all, From, Lang}
            end,
-    Items = get_vh_rooms(Host, Query, RSM),
+    Items = lists:flatmap(
+             fun(R) ->
+                     case get_room_disco_item(R, Query) of
+                         {ok, Item} -> [Item];
+                         {error, _} -> []
+                     end
+             end, get_online_rooms(ServerHost, Host, RSM)),
     ResRSM = case Items of
                 [_|_] when RSM /= undefined ->
                     #disco_item{jid = #jid{luser = First}} = hd(Items),
@@ -630,89 +689,30 @@ iq_disco_items(Host, From, Lang, MaxRoomsDiscoItems, Node, RSM)
                     undefined
             end,
     {result, #disco_items{node = Node, items = Items, rsm = ResRSM}};
-iq_disco_items(_Host, _From, Lang, _MaxRoomsDiscoItems, _Node, _RSM) ->
+iq_disco_items(_ServerHost, _Host, _From, Lang, _MaxRoomsDiscoItems, _Node, _RSM) ->
     {error, xmpp:err_item_not_found(<<"Node not found">>, Lang)}.
 
--spec get_vh_rooms(binary, term(), rsm_set() | undefined) -> [disco_item()].
-get_vh_rooms(Host, Query,
-            #rsm_set{max = Max, 'after' = After, before = undefined})
-  when is_binary(After), After /= <<"">> ->
-    lists:reverse(get_vh_rooms(next, {After, Host}, Host, Query, 0, Max, []));
-get_vh_rooms(Host, Query,
-            #rsm_set{max = Max, 'after' = undefined, before = Before})
-  when is_binary(Before), Before /= <<"">> ->
-    get_vh_rooms(prev, {Before, Host}, Host, Query, 0, Max, []);
-get_vh_rooms(Host, Query,
-            #rsm_set{max = Max, 'after' = undefined, before = <<"">>}) ->
-    get_vh_rooms(last, {<<"">>, Host}, Host, Query, 0, Max, []);
-get_vh_rooms(Host, Query, #rsm_set{max = Max}) ->
-    lists:reverse(get_vh_rooms(first, {<<"">>, Host}, Host, Query, 0, Max, []));
-get_vh_rooms(Host, Query, undefined) ->
-    lists:reverse(get_vh_rooms(first, {<<"">>, Host}, Host, Query, 0, undefined, [])).
-
--spec get_vh_rooms(prev | next | last | first,
-                  {binary(), binary()}, binary(), term(),
-                  non_neg_integer(), non_neg_integer() | undefined,
-                  [disco_item()]) -> [disco_item()].
-get_vh_rooms(_Action, _Key, _Host, _Query, Count, Max, Items) when Count >= Max ->
-    Items;
-get_vh_rooms(Action, Key, Host, Query, Count, Max, Items) ->
-    Call = fun() ->
-                  case Action of
-                      prev -> mnesia:dirty_prev(muc_online_room, Key);
-                      next -> mnesia:dirty_next(muc_online_room, Key);
-                      last -> mnesia:dirty_last(muc_online_room);
-                      first -> mnesia:dirty_first(muc_online_room)
-                  end
-          end,
-    NewAction = case Action of
-                   last -> prev;
-                   first -> next;
-                   _ -> Action
-               end,
-    try Call() of
-       '$end_of_table' ->
-           Items;
-       {_, Host} = NewKey ->
-           case get_room_disco_item(NewKey, Query) of
-               {ok, Item} ->
-                   get_vh_rooms(NewAction, NewKey, Host, Query,
-                                Count + 1, Max, [Item|Items]);
-               {error, _} ->
-                   get_vh_rooms(NewAction, NewKey, Host, Query,
-                                Count, Max, Items)
-           end;
-       NewKey ->
-           get_vh_rooms(NewAction, NewKey, Host, Query, Count, Max, Items)
-    catch _:{aborted, {badarg, _}} ->
-           Items
-    end.
-
--spec get_room_disco_item({binary(), binary()}, term()) -> {ok, disco_item()} |
-                                                          {error, timeout | notfound}.
-get_room_disco_item({Name, Host}, Query) ->
-    case mnesia:dirty_read(muc_online_room, {Name, Host}) of
-       [#muc_online_room{pid = Pid}|_] ->
-           RoomJID = jid:make(Name, Host),
-           try gen_fsm:sync_send_all_state_event(Pid, Query, 100) of
-               {item, Desc} ->
-                   {ok, #disco_item{jid = RoomJID, name = Desc}};
-               false ->
-                   {error, notfound}
-           catch _:{timeout, _} ->
-                   {error, timeout};
-                 _:{noproc, _} ->
-                   {error, notfound}
-           end;
-       _ ->
+-spec get_room_disco_item({binary(), binary(), pid()},
+                         term()) -> {ok, disco_item()} |
+                                    {error, timeout | notfound}.
+get_room_disco_item({Name, Host, Pid}, Query) ->
+    RoomJID = jid:make(Name, Host),
+    try gen_fsm:sync_send_all_state_event(Pid, Query, 100) of
+       {item, Desc} ->
+           {ok, #disco_item{jid = RoomJID, name = Desc}};
+       false ->
+           {error, notfound}
+    catch _:{timeout, _} ->
+           {error, timeout};
+         _:{noproc, _} ->
            {error, notfound}
     end.
 
-get_subscribed_rooms(_ServerHost, Host, From) ->
-    Rooms = get_vh_rooms(Host),
+get_subscribed_rooms(ServerHost, Host, From) ->
+    Rooms = get_online_rooms(ServerHost, Host),
     BareFrom = jid:remove_resource(From),
     lists:flatmap(
-      fun(#muc_online_room{name_host = {Name, _}, pid = Pid}) ->
+      fun({Name, _, Pid}) ->
              case gen_fsm:sync_send_all_state_event(Pid, {is_subscribed, BareFrom}) of
                  true -> [jid:make(Name, Host)];
                  false -> []
@@ -793,72 +793,28 @@ process_iq_register_set(ServerHost, Host, From,
            {error, xmpp:err_not_acceptable(ErrText, Lang)}
     end.
 
-broadcast_service_message(Host, Msg) ->
+-spec broadcast_service_message(binary(), binary(), message()) -> ok.
+broadcast_service_message(ServerHost, Host, Msg) ->
     lists:foreach(
-       fun(#muc_online_room{pid = Pid}) ->
-               gen_fsm:send_all_state_event(
-                   Pid, {service_message, Msg})
-       end, get_vh_rooms(Host)).
-
-
-get_vh_rooms(Host) ->
-    mnesia:dirty_select(muc_online_room,
-                       [{#muc_online_room{name_host = '$1', _ = '_'},
-                         [{'==', {element, 2, '$1'}, Host}],
-                         ['$_']}]).
-
--spec get_vh_rooms_count(binary()) -> non_neg_integer().
-get_vh_rooms_count(Host) ->
-    ets:select_count(muc_online_room,
-                    ets:fun2ms(
-                      fun(#muc_online_room{name_host = {_, H}}) ->
-                              H == Host
-                      end)).
-
-clean_table_from_bad_node(Node) ->
-    F = fun() ->
-               Es = mnesia:select(
-                      muc_online_room,
-                      [{#muc_online_room{pid = '$1', _ = '_'},
-                        [{'==', {node, '$1'}, Node}],
-                        ['$_']}]),
-               lists:foreach(fun(E) ->
-                                     mnesia:delete_object(E)
-                             end, Es)
-        end,
-    mnesia:async_dirty(F).
-
-clean_table_from_bad_node(Node, Host) ->
-    F = fun() ->
-               Es = mnesia:select(
-                      muc_online_room,
-                      [{#muc_online_room{pid = '$1',
-                                         name_host = {'_', Host},
-                                         _ = '_'},
-                        [{'==', {node, '$1'}, Node}],
-                        ['$_']}]),
-               lists:foreach(fun(E) ->
-                                     mnesia:delete_object(E)
-                             end, Es)
-        end,
-    mnesia:async_dirty(F).
-
-update_tables() ->
-    try
-       case mnesia:table_info(muc_online_room, type) of
-           ordered_set -> ok;
-           _ ->
-               case mnesia:delete_table(muc_online_room) of
-                   {atomic, ok} -> ok;
-                   Err -> erlang:error(Err)
-               end
-       end
-    catch _:{aborted, {no_exists, muc_online_room}} -> ok;
-         _:{aborted, {no_exists, muc_online_room, type}} -> ok;
-         E:R ->
-           ?ERROR_MSG("failed to update mnesia table '~s': ~p",
-                      [muc_online_room, {E, R}])
-    end.
+      fun({_, _, Pid}) ->
+             gen_fsm:send_all_state_event(
+               Pid, {service_message, Msg})
+      end, get_online_rooms(ServerHost, Host)).
+
+-spec get_online_rooms(binary(), binary()) -> [{binary(), binary(), pid()}].
+get_online_rooms(ServerHost, Host) ->
+    get_online_rooms(ServerHost, Host, undefined).
+
+-spec get_online_rooms(binary(), binary(), undefined | rsm_set()) ->
+                         [{binary(), binary(), pid()}].
+get_online_rooms(ServerHost, Host, RSM) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:get_online_rooms(Host, RSM).
+
+-spec count_online_rooms(binary(), binary()) -> non_neg_integer().
+count_online_rooms(ServerHost, Host) ->
+    RMod = gen_mod:ram_db_mod(ServerHost, ?MODULE),
+    RMod:count_online_rooms(Host).
 
 opts_to_binary(Opts) ->
     lists:map(
@@ -922,6 +878,7 @@ mod_opt_type(access_create) ->
 mod_opt_type(access_persistent) ->
     fun acl:access_rules_validator/1;
 mod_opt_type(db_type) -> fun(T) -> ejabberd_config:v_db(?MODULE, T) end;
+mod_opt_type(ram_db_type) -> fun(T) -> ejabberd_config:v_db(?MODULE, T) end;
 mod_opt_type(default_room_options) ->
     fun (L) when is_list(L) -> L end;
 mod_opt_type(history_size) ->
@@ -963,7 +920,7 @@ mod_opt_type(user_presence_shaper) ->
     fun (A) when is_atom(A) -> A end;
 mod_opt_type(_) ->
     [access, access_admin, access_create, access_persistent,
-     db_type, default_room_options, history_size, host,
+     db_type, ram_db_type, default_room_options, history_size, host,
      max_room_desc, max_room_id, max_room_name,
      max_rooms_discoitems, max_user_conferences, max_users,
      max_users_admin_threshold, max_users_presence,
index 91ccce5590da2b0d528d71da04da5ca2b0799182..48835c01609c6c98ff441d5d194be6c489651ba4 100644 (file)
@@ -28,7 +28,6 @@
 -include("logger.hrl").
 -include("xmpp.hrl").
 -include("mod_muc_room.hrl").
--include("mod_muc.hrl").
 -include("ejabberd_http.hrl").
 -include("ejabberd_web_admin.hrl").
 -include("ejabberd_commands.hrl").
@@ -207,22 +206,12 @@ get_commands_spec() ->
 %%%
 
 muc_online_rooms(ServerHost) ->
-    MUCHost = find_host(ServerHost),
-    Rooms = ets:tab2list(muc_online_room),
-    lists:foldl(
-      fun(Room, Results) ->
-             {Roomname, Host} = Room#muc_online_room.name_host,
-             case MUCHost of
-                 global ->
-                     [<<Roomname/binary, "@", Host/binary>> | Results];
-                 Host ->
-                     [<<Roomname/binary, "@", Host/binary>> | Results];
-                 _ ->
-                     Results
-             end
-      end,
-      [],
-      Rooms).
+    Hosts = find_hosts(ServerHost),
+    lists:flatmap(
+      fun(Host) ->
+             [{<<Name/binary, "@", Host/binary>>}
+              || {Name, _, _} <- mod_muc:get_online_rooms(Host)]
+      end, Hosts).
 
 muc_unregister_nick(Nick) ->
     F2 = fun(N) ->
@@ -237,14 +226,18 @@ muc_unregister_nick(Nick) ->
     end.
 
 get_user_rooms(LUser, LServer) ->
-    US = {LUser, LServer},
-    case catch ets:select(muc_online_users,
-                          [{#muc_online_users{us = US, room='$1', host='$2', _ = '_'}, [], [{{'$1', '$2'}}]}])
-        of
-      Res when is_list(Res) ->
-       [<<R/binary, "@", H/binary>> || {R, H} <- Res];
-      _ -> []
-    end.
+    lists:flatmap(
+      fun(ServerHost) ->
+             case gen_mod:is_loaded(ServerHost, mod_muc) of
+                 true ->
+                     Rooms = mod_muc:get_online_rooms_by_user(
+                               ServerHost, LUser, LServer),
+                     [<<Name/binary, "@", Host/binary>>
+                          || {Name, Host} <- Rooms];
+                 false ->
+                     []
+             end
+      end, ?MYHOSTS).
 
 %%----------------------------
 %% Ad-hoc commands
@@ -274,10 +267,14 @@ web_menu_host(Acc, _Host, Lang) ->
                      ])).
 
 web_page_main(_, #request{path=[<<"muc">>], lang = Lang} = _Request) ->
+    OnlineRoomsNumber = lists:foldl(
+                         fun(Host, Acc) ->
+                                 Acc ++ mod_muc:count_online_rooms(Host)
+                         end, 0, find_hosts(global)),
     Res = [?XCT(<<"h1">>, <<"Multi-User Chat">>),
           ?XCT(<<"h3">>, <<"Statistics">>),
           ?XAE(<<"table">>, [],
-               [?XE(<<"tbody">>, [?TDTD(<<"Total rooms">>, ets:info(muc_online_room, size)),
+               [?XE(<<"tbody">>, [?TDTD(<<"Total rooms">>, OnlineRoomsNumber),
                                   ?TDTD(<<"Permanent rooms">>, mnesia:table_info(muc_room, size)),
                                   ?TDTD(<<"Registered nicknames">>, mnesia:table_info(muc_registered, size))
                                  ])
@@ -456,8 +453,8 @@ create_room_with_opts(Name1, Host1, ServerHost, CustomRoomOpts) ->
     RoomShaper = gen_mod:get_module_opt(ServerHost, mod_muc, room_shaper, fun(X) -> X end, none),
 
     %% If the room does not exist yet in the muc_online_room
-    case mnesia:dirty_read(muc_online_room, {Name, Host}) of
-        [] ->
+    case mod_muc:find_online_room(Name, Host) of
+       error ->
            %% Start the room
            {ok, Pid} = mod_muc_room:start(
                          Host,
@@ -467,19 +464,12 @@ create_room_with_opts(Name1, Host1, ServerHost, CustomRoomOpts) ->
                          HistorySize,
                          RoomShaper,
                          RoomOpts),
-           {atomic, ok} = register_room(Host, Name, Pid),
+           mod_muc:register_online_room(Host, Name, Pid),
            ok;
-       _ ->
+       {ok, _} ->
            error
     end.
 
-register_room(Host, Name, Pid) ->
-    F = fun() ->
-               mnesia:write(#muc_online_room{name_host = {Name, Host},
-                                             pid = Pid})
-       end,
-    mnesia:transaction(F).
-
 %% Create the room only in the database.
 %% It is required to restart the MUC service for the room to appear.
 muc_create_room(ServerHost, {Name, Host, _}, DefRoomOpts) ->
@@ -492,12 +482,11 @@ muc_create_room(ServerHost, {Name, Host, _}, DefRoomOpts) ->
 %% If the room has participants, they are not notified that the room was destroyed;
 %% they will notice when they try to chat and receive an error that the room doesn't exist.
 destroy_room(Name, Service) ->
-    case mnesia:dirty_read(muc_online_room, {Name, Service}) of
-       [R] ->
-           Pid = R#muc_online_room.pid,
+    case mod_muc:find_online_room(Name, Service) of
+       {ok, Pid} ->
            gen_fsm:send_all_state_event(Pid, destroy),
            ok;
-       [] ->
+       error ->
            error
     end.
 
@@ -602,19 +591,12 @@ muc_unused2(Action, ServerHost, Host, Last_allowed) ->
 %%---------------
 %% Get info
 
-get_rooms(Host) ->
-    Get_room_names = fun(Room_reg, Names) ->
-                            Pid = Room_reg#muc_online_room.pid,
-                            case {Host, Room_reg#muc_online_room.name_host} of
-                                {Host, {Name1, Host}} ->
-                                    [{Name1, Host, Pid} | Names];
-                                {global, {Name1, Host1}} ->
-                                    [{Name1, Host1, Pid} | Names];
-                                _ ->
-                                    Names
-                            end
-                    end,
-    ets:foldr(Get_room_names, [], muc_online_room).
+get_rooms(ServerHost) ->
+    Hosts = find_hosts(ServerHost),
+    lists:flatmap(
+      fun(Host) ->
+             mod_muc:get_online_rooms(Host)
+      end, Hosts).
 
 get_room_config(Room_pid) ->
     {ok, R} = gen_fsm:sync_send_all_state_event(Room_pid, get_config),
@@ -813,11 +795,11 @@ format_room_option(OptionString, ValueString) ->
 
 %% @doc Get the Pid of an existing MUC room, or 'room_not_found'.
 get_room_pid(Name, Service) ->
-    case mnesia:dirty_read(muc_online_room, {Name, Service}) of
-       [] ->
+    case mod_muc:find_online_room(Name, Service) of
+       error ->
            room_not_found;
-       [Room] ->
-           Room#muc_online_room.pid
+       {ok, Pid} ->
+           Pid
     end.
 
 %% It is required to put explicitely all the options because
@@ -884,10 +866,9 @@ get_options(Config) ->
 %%    [{JID::string(), Domain::string(), Role::string(), Reason::string()}]
 %% @doc Get the affiliations of  the room Name@Service.
 get_room_affiliations(Name, Service) ->
-    case mnesia:dirty_read(muc_online_room, {Name, Service}) of
-       [R] ->
+    case mod_muc:find_online_room(Name, Service) of
+       {ok, Pid} ->
            %% Get the PID of the online room, then request its state
-           Pid = R#muc_online_room.pid,
            {ok, StateData} = gen_fsm:sync_send_all_state_event(Pid, get_state),
            Affiliations = ?DICT:to_list(StateData#state.affiliations),
            lists:map(
@@ -896,7 +877,7 @@ get_room_affiliations(Name, Service) ->
                 ({{Uname, Domain, _Res}, Aff}) when is_atom(Aff)->
                      {Uname, Domain, Aff, <<>>}
              end, Affiliations);
-       [] ->
+       error ->
            throw({error, "The room does not exist."})
     end.
 
@@ -914,14 +895,13 @@ get_room_affiliations(Name, Service) ->
 %% In any other case the action will be to create the affiliation.
 set_room_affiliation(Name, Service, JID, AffiliationString) ->
     Affiliation = jlib:binary_to_atom(AffiliationString),
-    case mnesia:dirty_read(muc_online_room, {Name, Service}) of
-       [R] ->
+    case mod_muc:find_online_room(Name, Service) of
+       {ok, Pid} ->
            %% Get the PID for the online room so we can get the state of the room
-           Pid = R#muc_online_room.pid,
            {ok, StateData} = gen_fsm:sync_send_all_state_event(Pid, {process_item_change, {jid:from_string(JID), affiliation, Affiliation, <<"">>}, <<"">>}),
            mod_muc:store_room(StateData#state.server_host, StateData#state.host, StateData#state.room, make_opts(StateData)),
            ok;
-       [] ->
+       error ->
            error
     end.
 
@@ -1045,4 +1025,28 @@ find_host(ServerHost) when is_list(ServerHost) ->
 find_host(ServerHost) ->
     gen_mod:get_module_opt_host(ServerHost, mod_muc, <<"conference.@HOST@">>).
 
+find_hosts(Global) when Global == global;
+                       Global == "global";
+                       Global == <<"global">> ->
+    lists:flatmap(
+      fun(ServerHost) ->
+             case gen_mod:is_loaded(ServerHost, mod_muc) of
+                 true ->
+                     [gen_mod:get_module_opt_host(
+                        ServerHost, mod_muc, <<"conference.@HOST@">>)];
+                 false ->
+                     []
+             end
+      end, ?MYHOSTS);
+find_hosts(ServerHost) when is_list(ServerHost) ->
+    find_hosts(list_to_binary(ServerHost));
+find_hosts(ServerHost) ->
+    case gen_mod:is_loaded(ServerHost, mod_muc) of
+       true ->
+           [gen_mod:get_module_opt_host(
+              ServerHost, mod_muc, <<"conference.@HOST@">>)];
+       false ->
+           []
+    end.
+
 mod_opt_type(_) -> [].
index 2675db9b535aa8e5cde0639a76746786532791c9..2f5b319922c12d882e9b1cc2e6cafa4298b03bb3 100644 (file)
@@ -47,7 +47,6 @@
 -include("logger.hrl").
 
 -include("xmpp.hrl").
--include("mod_muc.hrl").
 -include("mod_muc_room.hrl").
 
 -define(T(Text), translate:translate(Lang, Text)).
@@ -1169,13 +1168,11 @@ get_room_occupants(RoomJIDString) ->
 -spec get_room_state(binary(), binary()) -> mod_muc_room:state().
 
 get_room_state(RoomName, MucService) ->
-    case mnesia:dirty_read(muc_online_room,
-                          {RoomName, MucService})
-       of
-      [R] ->
-         RoomPid = R#muc_online_room.pid,
-         get_room_state(RoomPid);
-      [] -> #state{}
+    case mod_muc:find_online_room(RoomName, MucService) of
+       {ok, RoomPid} ->
+           get_room_state(RoomPid);
+       error ->
+           #state{}
     end.
 
 -spec get_room_state(pid()) -> mod_muc_room:state().
index 6464d4312084eeb11d93f56584ab82224dcb07b4..0552184b4b49da36b56ccf9795c803ee0e869ec0 100644 (file)
 %% API
 -export([init/2, import/3, store_room/4, restore_room/3, forget_room/3,
         can_use_nick/4, get_rooms/2, get_nick/3, set_nick/4]).
+-export([register_online_room/3, unregister_online_room/3, find_online_room/2,
+        get_online_rooms/2, count_online_rooms/1, rsm_supported/0,
+        register_online_user/3, unregister_online_user/3,
+        count_online_rooms_by_user/2, get_online_rooms_by_user/2,
+        handle_event/1]).
 -export([set_affiliation/6, set_affiliations/4, get_affiliation/5,
         get_affiliations/3, search_affiliation/4]).
 
--include("jid.hrl").
 -include("mod_muc.hrl").
 -include("logger.hrl").
+-include("xmpp.hrl").
+-include_lib("stdlib/include/ms_transform.hrl").
 
 %%%===================================================================
 %%% API
 %%%===================================================================
-init(_Host, Opts) ->
+init(Host, Opts) ->
     MyHost = proplists:get_value(host, Opts),
-    ejabberd_mnesia:create(?MODULE, muc_room,
-                       [{disc_copies, [node()]},
-                        {attributes,
-                         record_info(fields, muc_room)}]),
-    ejabberd_mnesia:create(?MODULE, muc_registered,
-                       [{disc_copies, [node()]},
-                        {attributes,
-                         record_info(fields, muc_registered)}]),
-    update_tables(MyHost),
-    mnesia:add_table_index(muc_registered, nick).
+    case gen_mod:db_mod(Host, Opts, mod_muc) of
+       ?MODULE ->
+           ejabberd_mnesia:create(?MODULE, muc_room,
+                                  [{disc_copies, [node()]},
+                                   {attributes,
+                                    record_info(fields, muc_room)}]),
+           ejabberd_mnesia:create(?MODULE, muc_registered,
+                                  [{disc_copies, [node()]},
+                                   {attributes,
+                                    record_info(fields, muc_registered)}]),
+           update_tables(MyHost),
+           mnesia:add_table_index(muc_registered, nick);
+       _ ->
+           ok
+    end,
+    case gen_mod:ram_db_mod(Host, Opts, mod_muc) of
+       ?MODULE ->
+           update_muc_online_table(),
+           ejabberd_mnesia:create(?MODULE, muc_online_room,
+                                  [{ram_copies, [node()]},
+                                   {type, ordered_set},
+                                   {attributes,
+                                    record_info(fields, muc_online_room)}]),
+           mnesia:add_table_copy(muc_online_room, node(), ram_copies),
+           catch ets:new(muc_online_users,
+                         [bag, named_table, public, {keypos, 2}]),
+           clean_table_from_bad_node(node(), MyHost),
+           mnesia:subscribe(system);
+       _ ->
+           ok
+    end.
 
 store_room(_LServer, Host, Name, Opts) ->
     F = fun () ->
@@ -132,6 +159,128 @@ get_affiliations(_ServerHost, _Room, _Host) ->
 search_affiliation(_ServerHost, _Room, _Host, _Affiliation) ->
     {error, not_implemented}.
 
+register_online_room(Room, Host, Pid) ->
+    F = fun() ->
+               mnesia:write(
+                 #muc_online_room{name_host = {Room, Host}, pid = Pid})
+       end,
+    mnesia:transaction(F).
+
+unregister_online_room(Room, Host, Pid) ->
+    F = fun () ->
+               mnesia:delete_object(
+                 #muc_online_room{name_host = {Room, Host}, pid = Pid})
+       end,
+    mnesia:transaction(F).
+
+find_online_room(Room, Host) ->
+    case mnesia:dirty_read(muc_online_room, {Room, Host}) of
+       [] -> error;
+       [#muc_online_room{pid = Pid}] -> {ok, Pid}
+    end.
+
+count_online_rooms(Host) ->
+    ets:select_count(
+      muc_online_room,
+      ets:fun2ms(
+       fun(#muc_online_room{name_host = {_, H}}) ->
+               H == Host
+       end)).
+
+get_online_rooms(Host,
+                #rsm_set{max = Max, 'after' = After, before = undefined})
+  when is_binary(After), After /= <<"">> ->
+    lists:reverse(get_online_rooms(next, {After, Host}, Host, 0, Max, []));
+get_online_rooms(Host,
+                #rsm_set{max = Max, 'after' = undefined, before = Before})
+  when is_binary(Before), Before /= <<"">> ->
+    get_online_rooms(prev, {Before, Host}, Host, 0, Max, []);
+get_online_rooms(Host,
+                #rsm_set{max = Max, 'after' = undefined, before = <<"">>}) ->
+    get_online_rooms(last, {<<"">>, Host}, Host, 0, Max, []);
+get_online_rooms(Host, #rsm_set{max = Max}) ->
+    lists:reverse(get_online_rooms(first, {<<"">>, Host}, Host, 0, Max, []));
+get_online_rooms(Host, undefined) ->
+    mnesia:dirty_select(
+      muc_online_room,
+      ets:fun2ms(
+       fun(#muc_online_room{name_host = {Name, H}, pid = Pid})
+             when H == Host -> {Name, Host, Pid}
+       end)).
+
+-spec get_online_rooms(prev | next | last | first,
+                      {binary(), binary()}, binary(),
+                      non_neg_integer(), non_neg_integer() | undefined,
+                      [{binary(), binary(), pid()}]) ->
+                             [{binary(), binary(), pid()}].
+get_online_rooms(_Action, _Key, _Host, Count, Max, Items) when Count >= Max ->
+    Items;
+get_online_rooms(Action, Key, Host, Count, Max, Items) ->
+    Call = fun() ->
+                  case Action of
+                      prev -> mnesia:dirty_prev(muc_online_room, Key);
+                      next -> mnesia:dirty_next(muc_online_room, Key);
+                      last -> mnesia:dirty_last(muc_online_room);
+                      first -> mnesia:dirty_first(muc_online_room)
+                  end
+          end,
+    NewAction = case Action of
+                   last -> prev;
+                   first -> next;
+                   _ -> Action
+               end,
+    try Call() of
+       '$end_of_table' ->
+           Items;
+       {Room, Host} = NewKey ->
+           case find_online_room(Room, Host) of
+               {ok, Pid} ->
+                   get_online_rooms(NewAction, NewKey, Host,
+                                    Count + 1, Max, [{Room, Host, Pid}|Items]);
+               {error, _} ->
+                   get_online_rooms(NewAction, NewKey, Host,
+                                    Count, Max, Items)
+           end;
+       NewKey ->
+           get_online_rooms(NewAction, NewKey, Host, Count, Max, Items)
+    catch _:{aborted, {badarg, _}} ->
+           Items
+    end.
+
+handle_event({mnesia_system_event, {mnesia_down, Node}}) ->
+    clean_table_from_bad_node(Node);
+handle_event(_) ->
+    ok.
+
+rsm_supported() ->
+    true.
+
+register_online_user({U, S, R}, Room, Host) ->
+    ets:insert(muc_online_users,
+              #muc_online_users{us = {U, S}, resource = R,
+                                room = Room, host = Host}).
+
+unregister_online_user({U, S, R}, Room, Host) ->
+    ets:delete_object(muc_online_users,
+                     #muc_online_users{us = {U, S}, resource = R,
+                                       room = Room, host = Host}).
+
+count_online_rooms_by_user(U, S) ->
+    ets:select_count(
+      muc_online_users,
+      ets:fun2ms(
+       fun(#muc_online_users{us = {U1, S1}}) ->
+               U == U1 andalso S == S1
+       end)).
+
+get_online_rooms_by_user(U, S) ->
+    ets:select(
+      muc_online_users,
+      ets:fun2ms(
+       fun(#muc_online_users{us = {U1, S1}, room = Room, host = Host})
+             when U == U1 andalso S == S1 -> {Room, Host}
+       end)).
+
 import(_LServer, <<"muc_room">>,
        [Name, RoomHost, SOpts, _TimeStamp]) ->
     Opts = mod_muc:opts_to_binary(ejabberd_sql:decode_term(SOpts)),
@@ -148,6 +297,34 @@ import(_LServer, <<"muc_registered">>,
 %%%===================================================================
 %%% Internal functions
 %%%===================================================================
+clean_table_from_bad_node(Node) ->
+    F = fun() ->
+               Es = mnesia:select(
+                      muc_online_room,
+                      [{#muc_online_room{pid = '$1', _ = '_'},
+                        [{'==', {node, '$1'}, Node}],
+                        ['$_']}]),
+               lists:foreach(fun(E) ->
+                                     mnesia:delete_object(E)
+                             end, Es)
+        end,
+    mnesia:async_dirty(F).
+
+clean_table_from_bad_node(Node, Host) ->
+    F = fun() ->
+               Es = mnesia:select(
+                      muc_online_room,
+                      [{#muc_online_room{pid = '$1',
+                                         name_host = {'_', Host},
+                                         _ = '_'},
+                        [{'==', {node, '$1'}, Node}],
+                        ['$_']}]),
+               lists:foreach(fun(E) ->
+                                     mnesia:delete_object(E)
+                             end, Es)
+        end,
+    mnesia:async_dirty(F).
+
 update_tables(Host) ->
     update_muc_room_table(Host),
     update_muc_registered_table(Host).
@@ -188,3 +365,20 @@ update_muc_registered_table(_Host) ->
          ?INFO_MSG("Recreating muc_registered table", []),
          mnesia:transform_table(muc_registered, ignore, Fields)
     end.
+
+update_muc_online_table() ->
+    try
+       case mnesia:table_info(muc_online_room, type) of
+           ordered_set -> ok;
+           _ ->
+               case mnesia:delete_table(muc_online_room) of
+                   {atomic, ok} -> ok;
+                   Err -> erlang:error(Err)
+               end
+       end
+    catch _:{aborted, {no_exists, muc_online_room}} -> ok;
+         _:{aborted, {no_exists, muc_online_room, type}} -> ok;
+         E:R ->
+           ?ERROR_MSG("failed to update mnesia table '~s': ~p",
+                      [muc_online_room, {E, R, erlang:get_stacktrace()}])
+    end.
index d88092152fea6aa812c9ee40ef273bcaac4b680c..f53d945fe80a71f120e1ea2d7a8271d5bb7e0c21 100644 (file)
 %% API
 -export([init/2, import/3, store_room/4, restore_room/3, forget_room/3,
         can_use_nick/4, get_rooms/2, get_nick/3, set_nick/4]).
+-export([register_online_room/3, unregister_online_room/3, find_online_room/2,
+        get_online_rooms/2, count_online_rooms/1, rsm_supported/0,
+        register_online_user/3, unregister_online_user/3,
+        count_online_rooms_by_user/2, get_online_rooms_by_user/2,
+        handle_event/1]).
 -export([set_affiliation/6, set_affiliations/4, get_affiliation/5,
         get_affiliations/3, search_affiliation/4]).
 
@@ -120,6 +125,39 @@ get_affiliations(_ServerHost, _Room, _Host) ->
 search_affiliation(_ServerHost, _Room, _Host, _Affiliation) ->
     {error, not_implemented}.
 
+register_online_room(_, _, _) ->
+    erlang:error(not_implemented).
+
+unregister_online_room(_, _, _) ->
+    erlang:error(not_implemented).
+
+find_online_room(_, _) ->
+    erlang:error(not_implemented).
+
+count_online_rooms(_) ->
+    erlang:error(not_implemented).
+
+get_online_rooms(_, _) ->
+    erlang:error(not_implemented).
+
+handle_event(_) ->
+    ok.
+
+rsm_supported() ->
+    false.
+
+register_online_user(_, _, _) ->
+    erlang:error(not_implemented).
+
+unregister_online_user(_, _, _) ->
+    erlang:error(not_implemented).
+
+count_online_rooms_by_user(_, _) ->
+    erlang:error(not_implemented).
+
+get_online_rooms_by_user(_, _) ->
+    erlang:error(not_implemented).
+
 import(_LServer, <<"muc_room">>,
        [Name, RoomHost, SOpts, _TimeStamp]) ->
     Opts = mod_muc:opts_to_binary(ejabberd_sql:decode_term(SOpts)),
index 07689db1977b8f78ef24b2faae714a06719f69df..73bcdd166a2533182248131beef2048aceaef046 100644 (file)
@@ -1798,7 +1798,7 @@ add_new_user(From, Nick, Packet, StateData) ->
     Affiliation = get_affiliation(From, StateData),
     ServiceAffiliation = get_service_affiliation(From,
                                                 StateData),
-    NConferences = tab_count_user(From),
+    NConferences = tab_count_user(From, StateData),
     MaxConferences =
        gen_mod:get_module_opt(StateData#state.server_host,
                               mod_muc, max_user_conferences,
@@ -3968,38 +3968,25 @@ add_to_log(Type, Data, StateData) ->
 %%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%%
 %% Users number checking
 
--spec tab_add_online_user(jid(), state()) -> ok.
+-spec tab_add_online_user(jid(), state()) -> any().
 tab_add_online_user(JID, StateData) ->
-    {LUser, LServer, LResource} = jid:tolower(JID),
-    US = {LUser, LServer},
     Room = StateData#state.room,
     Host = StateData#state.host,
-    catch ets:insert(muc_online_users,
-                    #muc_online_users{us = US, resource = LResource,
-                                      room = Room, host = Host}),
-    ok.
+    ServerHost = StateData#state.server_host,
+    mod_muc:register_online_user(ServerHost, jid:tolower(JID), Room, Host).
 
--spec tab_remove_online_user(jid(), state()) -> ok.
+-spec tab_remove_online_user(jid(), state()) -> any().
 tab_remove_online_user(JID, StateData) ->
-    {LUser, LServer, LResource} = jid:tolower(JID),
-    US = {LUser, LServer},
     Room = StateData#state.room,
     Host = StateData#state.host,
-    catch ets:delete_object(muc_online_users,
-                           #muc_online_users{us = US, resource = LResource,
-                                             room = Room, host = Host}),
-    ok.
+    ServerHost = StateData#state.server_host,
+    mod_muc:unregister_online_user(ServerHost, jid:tolower(JID), Room, Host).
 
--spec tab_count_user(jid()) -> non_neg_integer().
-tab_count_user(JID) ->
+-spec tab_count_user(jid(), state()) -> non_neg_integer().
+tab_count_user(JID, StateData) ->
+    ServerHost = StateData#state.server_host,
     {LUser, LServer, _} = jid:tolower(JID),
-    US = {LUser, LServer},
-    case catch ets:select(muc_online_users,
-                         [{#muc_online_users{us = US, _ = '_'}, [], [[]]}])
-       of
-      Res when is_list(Res) -> length(Res);
-      _ -> 0
-    end.
+    mod_muc:count_online_rooms_by_user(ServerHost, LUser, LServer).
 
 -spec element_size(stanza()) -> non_neg_integer().
 element_size(El) ->
index 4b6be7d068ab6b183eda06f5b7ad913e83d6a6aa..08b8bf9e0b4b5a603daaef9ea901bb5e5b6d4826 100644 (file)
 -export([init/2, store_room/4, restore_room/3, forget_room/3,
         can_use_nick/4, get_rooms/2, get_nick/3, set_nick/4,
         import/3, export/1]).
+-export([register_online_room/3, unregister_online_room/3, find_online_room/2,
+        get_online_rooms/2, count_online_rooms/1, rsm_supported/0,
+        register_online_user/3, unregister_online_user/3,
+        count_online_rooms_by_user/2, get_online_rooms_by_user/2,
+        handle_event/1]).
 -export([set_affiliation/6, set_affiliations/4, get_affiliation/5,
         get_affiliations/3, search_affiliation/4]).
 
@@ -145,6 +150,39 @@ get_affiliations(_ServerHost, _Room, _Host) ->
 search_affiliation(_ServerHost, _Room, _Host, _Affiliation) ->
     {error, not_implemented}.
 
+register_online_room(_, _, _) ->
+    erlang:error(not_implemented).
+
+unregister_online_room(_, _, _) ->
+    erlang:error(not_implemented).
+
+find_online_room(_, _) ->
+    erlang:error(not_implemented).
+
+count_online_rooms(_) ->
+    erlang:error(not_implemented).
+
+get_online_rooms(_, _) ->
+    erlang:error(not_implemented).
+
+handle_event(_) ->
+    ok.
+
+rsm_supported() ->
+    false.
+
+register_online_user(_, _, _) ->
+    erlang:error(not_implemented).
+
+unregister_online_user(_, _, _) ->
+    erlang:error(not_implemented).
+
+count_online_rooms_by_user(_, _) ->
+    erlang:error(not_implemented).
+
+get_online_rooms_by_user(_, _) ->
+    erlang:error(not_implemented).
+
 export(_Server) ->
     [{muc_room,
       fun(Host, #muc_room{name_host = {Name, RoomHost}, opts = Opts}) ->
index c1768bf1cb1dd9102a6e52c512f2cb0317c40431..8443fde0cd75b8bf77a165737cf2509c9bf9c9c1 100644 (file)
@@ -148,8 +148,6 @@ init([Host, Opts]) ->
     ejabberd_hooks:add(c2s_self_presence, Host, ?MODULE, c2s_self_presence, 50),
     ejabberd_hooks:add(remove_user, Host,
                       ?MODULE, remove_user, 50),
-    ejabberd_hooks:add(anonymous_purge_hook, Host,
-                      ?MODULE, remove_user, 50),
     ejabberd_hooks:add(disco_sm_features, Host,
                       ?MODULE, get_sm_features, 50),
     ejabberd_hooks:add(disco_local_features, Host,
@@ -208,8 +206,6 @@ terminate(_Reason, State) ->
     ejabberd_hooks:delete(c2s_self_presence, Host, ?MODULE, c2s_self_presence, 50),
     ejabberd_hooks:delete(remove_user, Host, ?MODULE,
                          remove_user, 50),
-    ejabberd_hooks:delete(anonymous_purge_hook, Host,
-                         ?MODULE, remove_user, 50),
     ejabberd_hooks:delete(disco_sm_features, Host, ?MODULE, get_sm_features, 50),
     ejabberd_hooks:delete(disco_local_features, Host, ?MODULE, get_sm_features, 50),
     ejabberd_hooks:delete(disco_sm_identity, Host, ?MODULE, get_sm_identity, 50),
index d631b0ad052e14a9e8deb478d5bb368d79eecc53..527c010a0ba0b2bee051d6eebb9d268dd8fb3f65 100644 (file)
@@ -303,8 +303,6 @@ init([ServerHost, Opts]) ->
        ?MODULE, out_subscription, 50),
     ejabberd_hooks:add(remove_user, ServerHost,
        ?MODULE, remove_user, 50),
-    ejabberd_hooks:add(anonymous_purge_hook, ServerHost,
-       ?MODULE, remove_user, 50),
     ejabberd_hooks:add(c2s_handle_info, ServerHost,
        ?MODULE, c2s_handle_info, 50),
     gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_DISCO_INFO,
@@ -912,8 +910,6 @@ terminate(_Reason,
        ?MODULE, out_subscription, 50),
     ejabberd_hooks:delete(remove_user, ServerHost,
        ?MODULE, remove_user, 50),
-    ejabberd_hooks:delete(anonymous_purge_hook, ServerHost,
-       ?MODULE, remove_user, 50),
     ejabberd_hooks:delete(c2s_handle_info, ServerHost,
        ?MODULE, c2s_handle_info, 50),
     gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_DISCO_INFO),
index 085f502253619532e8c61f6c0a37151cc6123896..c22d02bb6f1793d5db146faf26c1e0557128bd04 100644 (file)
@@ -100,8 +100,6 @@ start(Host, Opts) ->
                       get_jid_info, 50),
     ejabberd_hooks:add(remove_user, Host, ?MODULE,
                       remove_user, 50),
-    ejabberd_hooks:add(anonymous_purge_hook, Host, ?MODULE,
-                      remove_user, 50),
     ejabberd_hooks:add(c2s_self_presence, Host, ?MODULE,
                       c2s_self_presence, 50),
     ejabberd_hooks:add(c2s_post_auth_features, Host,
@@ -128,8 +126,6 @@ stop(Host) ->
                          ?MODULE, get_jid_info, 50),
     ejabberd_hooks:delete(remove_user, Host, ?MODULE,
                          remove_user, 50),
-    ejabberd_hooks:delete(anonymous_purge_hook, Host,
-                         ?MODULE, remove_user, 50),
     ejabberd_hooks:delete(c2s_self_presence, Host, ?MODULE,
                          c2s_self_presence, 50),
     ejabberd_hooks:delete(c2s_post_auth_features,
index e7510936f125c1e58824f69cdd5852b196189a17..7839543178fb9d297289a1eaa415fe4ef026cd2b 100644 (file)
@@ -98,8 +98,6 @@ start(Host, Opts) ->
                       unset_presence, 50),
     ejabberd_hooks:add(register_user, Host, ?MODULE,
                       register_user, 50),
-    ejabberd_hooks:add(anonymous_purge_hook, Host, ?MODULE,
-                      remove_user, 50),
     ejabberd_hooks:add(remove_user, Host, ?MODULE,
                       remove_user, 50).
 
@@ -126,8 +124,6 @@ stop(Host) ->
                          ?MODULE, unset_presence, 50),
     ejabberd_hooks:delete(register_user, Host, ?MODULE,
                          register_user, 50),
-    ejabberd_hooks:delete(anonymous_purge_hook, Host,
-                         ?MODULE, remove_user, 50),
     ejabberd_hooks:delete(remove_user, Host, ?MODULE,
                          remove_user,
                          50).