aboutsummaryrefslogtreecommitdiff
path: root/src/mod_pubsub.erl
diff options
context:
space:
mode:
Diffstat (limited to 'src/mod_pubsub.erl')
-rw-r--r--src/mod_pubsub.erl5113
1 files changed, 2480 insertions, 2633 deletions
diff --git a/src/mod_pubsub.erl b/src/mod_pubsub.erl
index 81afc9a06..48a19b99f 100644
--- a/src/mod_pubsub.erl
+++ b/src/mod_pubsub.erl
@@ -5,7 +5,7 @@
%%% Created : 1 Dec 2007 by Christophe Romain <christophe.romain@process-one.net>
%%%
%%%
-%%% ejabberd, Copyright (C) 2002-2016 ProcessOne
+%%% ejabberd, Copyright (C) 2002-2019 ProcessOne
%%%
%%% This program is free software; you can redistribute it and/or
%%% modify it under the terms of the GNU General Public License as
@@ -35,15 +35,16 @@
-behaviour(gen_mod).
-behaviour(gen_server).
-author('christophe.romain@process-one.net').
--protocol({xep, 60, '1.13-1'}).
+-protocol({xep, 60, '1.14'}).
-protocol({xep, 163, '1.2'}).
-protocol({xep, 248, '0.2'}).
--include("ejabberd.hrl").
-include("logger.hrl").
--include("adhoc.hrl").
--include("jlib.hrl").
+-include("xmpp.hrl").
-include("pubsub.hrl").
+-include("mod_roster.hrl").
+-include("translate.hrl").
+-include("ejabberd_stacktrace.hrl").
-define(STDTREE, <<"tree">>).
-define(STDNODE, <<"flat">>).
@@ -51,38 +52,48 @@
%% exports for hooks
-export([presence_probe/3, caps_add/3, caps_update/3,
- in_subscription/6, out_subscription/4,
- on_user_offline/3, remove_user/2,
+ in_subscription/2, out_subscription/1,
+ on_self_presence/1, on_user_offline/2, remove_user/2,
disco_local_identity/5, disco_local_features/5,
disco_local_items/5, disco_sm_identity/5,
- disco_sm_features/5, disco_sm_items/5]).
+ disco_sm_features/5, disco_sm_items/5,
+ c2s_handle_info/2]).
%% exported iq handlers
--export([iq_sm/3]).
+-export([iq_sm/1, process_disco_info/1, process_disco_items/1,
+ process_pubsub/1, process_pubsub_owner/1, process_vcard/1,
+ process_commands/1]).
%% exports for console debug manual use
-export([create_node/5, create_node/7, delete_node/3,
- subscribe_node/5, unsubscribe_node/5, publish_item/6,
+ subscribe_node/5, unsubscribe_node/5, publish_item/6, publish_item/8,
delete_item/4, delete_item/5, send_items/7, get_items/2, get_item/3,
get_cached_item/2, get_configure/5, set_configure/5,
tree_action/3, node_action/4, node_call/4]).
%% general helpers for plugins
--export([subscription_to_string/1, affiliation_to_string/1,
- string_to_subscription/1, string_to_affiliation/1,
- extended_error/2, extended_error/3, service_jid/1,
+-export([extended_error/2, service_jid/1,
tree/1, tree/2, plugin/2, plugins/1, config/3,
host/1, serverhost/1]).
+%% pubsub#errors
+-export([err_closed_node/0, err_configuration_required/0,
+ err_invalid_jid/0, err_invalid_options/0, err_invalid_payload/0,
+ err_invalid_subid/0, err_item_forbidden/0, err_item_required/0,
+ err_jid_required/0, err_max_items_exceeded/0, err_max_nodes_exceeded/0,
+ err_nodeid_required/0, err_not_in_roster_group/0, err_not_subscribed/0,
+ err_payload_too_big/0, err_payload_required/0,
+ err_pending_subscription/0, err_precondition_not_met/0,
+ err_presence_subscription_required/0, err_subid_required/0,
+ err_too_many_subscriptions/0, err_unsupported/1,
+ err_unsupported_access_model/0]).
+
%% API and gen_server callbacks
--export([start_link/2, start/2, stop/1, init/1,
+-export([start/2, stop/1, init/1,
handle_call/3, handle_cast/2, handle_info/2,
- terminate/2, code_change/3, depends/2]).
-
--export([send_loop/1, mod_opt_type/1]).
+ terminate/2, code_change/3, depends/2, mod_opt_type/1, mod_options/1]).
--define(PROCNAME, ejabberd_mod_pubsub).
--define(LOOPNAME, ejabberd_mod_pubsub_loop).
+-export([route/1]).
%%====================================================================
%% API
@@ -142,6 +153,7 @@
-type(pubsubState() ::
#pubsub_state{
stateid :: {Entity::ljid(), Nidx::mod_pubsub:nodeIdx()},
+ nodeidx :: Nidx::mod_pubsub:nodeIdx(),
items :: [ItemId::mod_pubsub:itemId()],
affiliation :: Affs::mod_pubsub:affiliation(),
subscriptions :: [{Sub::mod_pubsub:subscription(), SubId::mod_pubsub:subId()}]
@@ -151,6 +163,7 @@
-type(pubsubItem() ::
#pubsub_item{
itemid :: {ItemId::mod_pubsub:itemId(), Nidx::mod_pubsub:nodeIdx()},
+ nodeidx :: Nidx::mod_pubsub:nodeIdx(),
creation :: {erlang:timestamp(), ljid()},
modification :: {erlang:timestamp(), ljid()},
payload :: mod_pubsub:payload()
@@ -166,7 +179,7 @@
-type(pubsubLastItem() ::
#pubsub_last_item{
- nodeid :: mod_pubsub:nodeIdx(),
+ nodeid :: {binary(), mod_pubsub:nodeIdx()},
itemid :: mod_pubsub:itemId(),
creation :: {erlang:timestamp(), ljid()},
payload :: mod_pubsub:payload()
@@ -176,7 +189,7 @@
-record(state,
{
server_host,
- host,
+ hosts,
access,
pep_mapping = [],
ignore_pep_from_offline = true,
@@ -192,7 +205,7 @@
-type(state() ::
#state{
server_host :: binary(),
- host :: mod_pubsub:hostPubsub(),
+ hosts :: [mod_pubsub:hostPubsub()],
access :: atom(),
pep_mapping :: [{binary(), binary()}],
ignore_pep_from_offline :: boolean(),
@@ -207,21 +220,13 @@
).
-
-start_link(Host, Opts) ->
- Proc = gen_mod:get_module_proc(Host, ?PROCNAME),
- gen_server:start_link({local, Proc}, ?MODULE, [Host, Opts], []).
+-type subs_by_depth() :: [{integer(), [{#pubsub_node{}, [{ljid(), subId(), subOptions()}]}]}].
start(Host, Opts) ->
- Proc = gen_mod:get_module_proc(Host, ?PROCNAME),
- ChildSpec = {Proc, {?MODULE, start_link, [Host, Opts]},
- transient, 1000, worker, [?MODULE]},
- supervisor:start_child(ejabberd_sup, ChildSpec).
+ gen_mod:start_child(?MODULE, Host, Opts).
stop(Host) ->
- Proc = gen_mod:get_module_proc(Host, ?PROCNAME),
- gen_server:call(Proc, stop),
- supervisor:delete_child(ejabberd_sup, Proc).
+ gen_mod:stop_child(?MODULE, Host).
%%====================================================================
%% gen_server callbacks
@@ -236,48 +241,66 @@ stop(Host) ->
%%--------------------------------------------------------------------
-spec init([binary() | [{_,_}],...]) -> {'ok',state()}.
-init([ServerHost, Opts]) ->
- ?DEBUG("pubsub init ~p ~p", [ServerHost, Opts]),
- Host = gen_mod:get_opt_host(ServerHost, Opts, <<"pubsub.@HOST@">>),
- ejabberd_router:register_route(Host, ServerHost),
- Access = gen_mod:get_opt(access_createnode, Opts,
- fun(A) when is_atom(A) -> A end, all),
- PepOffline = gen_mod:get_opt(ignore_pep_from_offline, Opts,
- fun(A) when is_boolean(A) -> A end, true),
- IQDisc = gen_mod:get_opt(iqdisc, Opts,
- fun gen_iq_handler:check_type/1, one_queue),
- LastItemCache = gen_mod:get_opt(last_item_cache, Opts,
- fun(A) when is_boolean(A) -> A end, false),
- MaxItemsNode = gen_mod:get_opt(max_items_node, Opts,
- fun(A) when is_integer(A) andalso A >= 0 -> A end, ?MAXITEMS),
- MaxSubsNode = gen_mod:get_opt(max_subscriptions_node, Opts,
- fun(A) when is_integer(A) andalso A >= 0 -> A end, undefined),
- pubsub_index:init(Host, ServerHost, Opts),
- {Plugins, NodeTree, PepMapping} = init_plugins(Host, ServerHost, Opts),
- DefaultModule = plugin(Host, hd(Plugins)),
- BaseOptions = DefaultModule:options(),
- DefaultNodeCfg = gen_mod:get_opt(default_node_config, Opts,
- fun(A) when is_list(A) -> filter_node_options(A, BaseOptions) end, []),
- mnesia:create_table(pubsub_last_item,
- [{ram_copies, [node()]},
- {attributes, record_info(fields, pubsub_last_item)}]),
- mod_disco:register_feature(ServerHost, ?NS_PUBSUB),
- lists:foreach(
- fun(H) ->
- T = gen_mod:get_module_proc(H, config),
- ets:new(T, [set, named_table]),
- ets:insert(T, {nodetree, NodeTree}),
- ets:insert(T, {plugins, Plugins}),
- ets:insert(T, {last_item_cache, LastItemCache}),
- ets:insert(T, {max_items_node, MaxItemsNode}),
- ets:insert(T, {max_subscriptions_node, MaxSubsNode}),
- ets:insert(T, {default_node_config, DefaultNodeCfg}),
- ets:insert(T, {pep_mapping, PepMapping}),
- ets:insert(T, {ignore_pep_from_offline, PepOffline}),
- ets:insert(T, {host, Host}),
- ets:insert(T, {access, Access})
- end, [Host, ServerHost]),
- ejabberd_hooks:add(sm_remove_connection_hook, ServerHost,
+init([ServerHost|_]) ->
+ process_flag(trap_exit, true),
+ Opts = gen_mod:get_module_opts(ServerHost, ?MODULE),
+ Hosts = gen_mod:get_opt_hosts(Opts),
+ Access = mod_pubsub_opt:access_createnode(Opts),
+ PepOffline = mod_pubsub_opt:ignore_pep_from_offline(Opts),
+ LastItemCache = mod_pubsub_opt:last_item_cache(Opts),
+ MaxItemsNode = mod_pubsub_opt:max_items_node(Opts),
+ MaxSubsNode = mod_pubsub_opt:max_subscriptions_node(Opts),
+ ejabberd_mnesia:create(?MODULE, pubsub_last_item,
+ [{ram_copies, [node()]},
+ {attributes, record_info(fields, pubsub_last_item)}]),
+ DBMod = gen_mod:db_mod(Opts, ?MODULE),
+ AllPlugins =
+ lists:flatmap(
+ fun(Host) ->
+ DBMod:init(Host, ServerHost, Opts),
+ ejabberd_router:register_route(
+ Host, ServerHost, {apply, ?MODULE, route}),
+ {Plugins, NodeTree, PepMapping} = init_plugins(Host, ServerHost, Opts),
+ DefaultModule = plugin(Host, hd(Plugins)),
+ DefaultNodeCfg = merge_config(
+ [mod_pubsub_opt:default_node_config(Opts),
+ DefaultModule:options()]),
+ lists:foreach(
+ fun(H) ->
+ T = gen_mod:get_module_proc(H, config),
+ try
+ ets:new(T, [set, named_table]),
+ ets:insert(T, {nodetree, NodeTree}),
+ ets:insert(T, {plugins, Plugins}),
+ ets:insert(T, {last_item_cache, LastItemCache}),
+ ets:insert(T, {max_items_node, MaxItemsNode}),
+ ets:insert(T, {max_subscriptions_node, MaxSubsNode}),
+ ets:insert(T, {default_node_config, DefaultNodeCfg}),
+ ets:insert(T, {pep_mapping, PepMapping}),
+ ets:insert(T, {ignore_pep_from_offline, PepOffline}),
+ ets:insert(T, {host, Host}),
+ ets:insert(T, {access, Access})
+ catch error:badarg when H == ServerHost ->
+ ok
+ end
+ end, [Host, ServerHost]),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_DISCO_INFO,
+ ?MODULE, process_disco_info),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_DISCO_ITEMS,
+ ?MODULE, process_disco_items),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_PUBSUB,
+ ?MODULE, process_pubsub),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_PUBSUB_OWNER,
+ ?MODULE, process_pubsub_owner),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_VCARD,
+ ?MODULE, process_vcard),
+ gen_iq_handler:add_iq_handler(ejabberd_local, Host, ?NS_COMMANDS,
+ ?MODULE, process_commands),
+ Plugins
+ end, Hosts),
+ ejabberd_hooks:add(c2s_self_presence, ServerHost,
+ ?MODULE, on_self_presence, 75),
+ ejabberd_hooks:add(c2s_terminated, ServerHost,
?MODULE, on_user_offline, 75),
ejabberd_hooks:add(disco_local_identity, ServerHost,
?MODULE, disco_local_identity, 75),
@@ -293,9 +316,9 @@ 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),
- case lists:member(?PEPNODE, Plugins) of
+ ejabberd_hooks:add(c2s_handle_info, ServerHost,
+ ?MODULE, c2s_handle_info, 50),
+ case lists:member(?PEPNODE, AllPlugins) of
true ->
ejabberd_hooks:add(caps_add, ServerHost,
?MODULE, caps_add, 80),
@@ -308,52 +331,30 @@ init([ServerHost, Opts]) ->
ejabberd_hooks:add(disco_sm_items, ServerHost,
?MODULE, disco_sm_items, 75),
gen_iq_handler:add_iq_handler(ejabberd_sm, ServerHost,
- ?NS_PUBSUB, ?MODULE, iq_sm, IQDisc),
+ ?NS_PUBSUB, ?MODULE, iq_sm),
gen_iq_handler:add_iq_handler(ejabberd_sm, ServerHost,
- ?NS_PUBSUB_OWNER, ?MODULE, iq_sm, IQDisc);
+ ?NS_PUBSUB_OWNER, ?MODULE, iq_sm);
false ->
ok
end,
- pubsub_migrate:update_node_database(Host, ServerHost),
- pubsub_migrate:update_state_database(Host, ServerHost),
- pubsub_migrate:update_lastitem_database(Host, ServerHost),
- {_, State} = init_send_loop(ServerHost),
- {ok, State}.
-
-init_send_loop(ServerHost) ->
NodeTree = config(ServerHost, nodetree),
Plugins = config(ServerHost, plugins),
- LastItemCache = config(ServerHost, last_item_cache),
- MaxItemsNode = config(ServerHost, max_items_node),
PepMapping = config(ServerHost, pep_mapping),
- PepOffline = config(ServerHost, ignore_pep_from_offline),
- Host = config(ServerHost, host),
- Access = config(ServerHost, access),
- DBType = gen_mod:db_type(ServerHost, ?MODULE),
- State = #state{host = Host, server_host = ServerHost,
- access = Access, pep_mapping = PepMapping,
- ignore_pep_from_offline = PepOffline,
- last_item_cache = LastItemCache,
- max_items_node = MaxItemsNode, nodetree = NodeTree,
- plugins = Plugins, db_type = DBType},
- Proc = gen_mod:get_module_proc(ServerHost, ?LOOPNAME),
- Pid = case whereis(Proc) of
- undefined ->
- SendLoop = spawn(?MODULE, send_loop, [State]),
- register(Proc, SendLoop),
- SendLoop;
- Loop ->
- Loop
- end,
- {Pid, State}.
+ DBType = mod_pubsub_opt:db_type(ServerHost),
+ {ok, #state{hosts = Hosts, server_host = ServerHost,
+ access = Access, pep_mapping = PepMapping,
+ ignore_pep_from_offline = PepOffline,
+ last_item_cache = LastItemCache,
+ max_items_node = MaxItemsNode, nodetree = NodeTree,
+ plugins = Plugins, db_type = DBType}}.
depends(ServerHost, Opts) ->
- Host = gen_mod:get_opt_host(ServerHost, Opts, <<"pubsub.@HOST@">>),
- Plugins = gen_mod:get_opt(plugins, Opts,
- fun(A) when is_list(A) -> A end, [?STDNODE]),
+ [Host|_] = gen_mod:get_opt_hosts(Opts),
+ Plugins = mod_pubsub_opt:plugins(Opts),
+ Db = mod_pubsub_opt:db_type(Opts),
lists:flatmap(
fun(Name) ->
- Plugin = plugin(ServerHost, Name),
+ Plugin = plugin(Db, Name),
try apply(Plugin, depends, [Host, ServerHost, Opts])
catch _:undef -> []
end
@@ -363,30 +364,17 @@ depends(ServerHost, Opts) ->
%% The default plugin module is implicit.
%% <p>The Erlang code for the plugin is located in a module called
%% <em>node_plugin</em>. The 'node_' prefix is mandatory.</p>
-%% <p>The modules are initialized in alphetical order and the list is checked
-%% and sorted to ensure that each module is initialized only once.</p>
%% <p>See {@link node_hometree:init/1} for an example implementation.</p>
init_plugins(Host, ServerHost, Opts) ->
- TreePlugin = tree(Host, gen_mod:get_opt(nodetree, Opts,
- fun(A) when is_binary(A) -> A end,
- ?STDTREE)),
- ?DEBUG("** tree plugin is ~p", [TreePlugin]),
+ TreePlugin = tree(Host, mod_pubsub_opt:nodetree(Opts)),
TreePlugin:init(Host, ServerHost, Opts),
- Plugins = gen_mod:get_opt(plugins, Opts,
- fun(A) when is_list(A) -> A end, [?STDNODE]),
- PepMapping = gen_mod:get_opt(pep_mapping, Opts,
- fun(A) when is_list(A) -> A end, []),
- ?DEBUG("** PEP Mapping : ~p~n", [PepMapping]),
+ Plugins = mod_pubsub_opt:plugins(Opts),
+ PepMapping = mod_pubsub_opt:pep_mapping(Opts),
PluginsOK = lists:foldl(
fun (Name, Acc) ->
Plugin = plugin(Host, Name),
- case catch apply(Plugin, init, [Host, ServerHost, Opts]) of
- {'EXIT', _Error} ->
- Acc;
- _ ->
- ?DEBUG("** init ~s plugin", [Name]),
- [Name | Acc]
- end
+ apply(Plugin, init, [Host, ServerHost, Opts]),
+ [Name | Acc]
end,
[], Plugins),
{lists:reverse(PluginsOK), TreePlugin, PepMapping}.
@@ -394,7 +382,6 @@ init_plugins(Host, ServerHost, Opts) ->
terminate_plugins(Host, ServerHost, Plugins, TreePlugin) ->
lists:foreach(
fun (Name) ->
- ?DEBUG("** terminate ~s plugin", [Name]),
Plugin = plugin(Host, Name),
Plugin:terminate(Host, ServerHost)
end,
@@ -402,206 +389,106 @@ terminate_plugins(Host, ServerHost, Plugins, TreePlugin) ->
TreePlugin:terminate(Host, ServerHost),
ok.
-send_loop(State) ->
- receive
- {presence, JID, Pid} ->
- Host = State#state.host,
- ServerHost = State#state.server_host,
- DBType = State#state.db_type,
- LJID = jid:tolower(JID),
- BJID = jid:remove_resource(LJID),
- lists:foreach(
- fun(PType) ->
- Subs = get_subscriptions_for_send_last(Host, PType, DBType, JID, LJID, BJID),
- lists:foreach(
- fun({NodeRec, _, _, SubJID}) ->
- {_, Node} = NodeRec#pubsub_node.nodeid,
- Nidx = NodeRec#pubsub_node.id,
- Options = NodeRec#pubsub_node.options,
- [send_items(Host, Node, Nidx, PType, Options, SubJID, last)
- || NodeRec#pubsub_node.type == PType]
- end,
- lists:usort(Subs))
- end,
- State#state.plugins),
- if not State#state.ignore_pep_from_offline ->
- {User, Server, Resource} = LJID,
- case catch ejabberd_c2s:get_subscribed(Pid) of
- Contacts when is_list(Contacts) ->
- lists:foreach(
- fun({U, S, R}) when S == ServerHost ->
- case user_resources(U, S) of
- [] -> %% offline
- PeerJID = jid:make(U, S, R),
- self() ! {presence, User, Server, [Resource], PeerJID};
- _ -> %% online
- % this is already handled by presence probe
- ok
- end;
- (_) ->
- % we can not do anything in any cases
- ok
- end,
- Contacts);
- _ ->
- ok
- end;
- true ->
- ok
- end,
- send_loop(State);
- {presence, User, Server, Resources, JID} ->
- spawn(fun() ->
- Host = State#state.host,
- Owner = jid:remove_resource(jid:tolower(JID)),
- lists:foreach(fun(#pubsub_node{nodeid = {_, Node}, type = Type, id = Nidx, options = Options}) ->
- case match_option(Options, send_last_published_item, on_sub_and_presence) of
- true ->
- lists:foreach(fun(Resource) ->
- LJID = {User, Server, Resource},
- Subscribed = case get_option(Options, access_model) of
- open -> true;
- presence -> true;
- whitelist -> false; % subscribers are added manually
- authorize -> false; % likewise
- roster ->
- Grps = get_option(Options, roster_groups_allowed, []),
- {OU, OS, _} = Owner,
- element(2, get_roster_info(OU, OS, LJID, Grps))
- end,
- if Subscribed -> send_items(Owner, Node, Nidx, Type, Options, LJID, last);
- true -> ok
- end
- end,
- Resources);
- _ ->
- ok
- end
- end,
- tree_action(Host, get_nodes, [Owner, JID]))
- end),
- send_loop(State);
- stop ->
- ok
- end.
-
%% -------
%% disco hooks handling functions
%%
--spec disco_local_identity(Acc :: [xmlel()], _From :: jid(),
- To :: jid(), Node :: <<>> | mod_pubsub:nodeId(),
- Lang :: binary()) -> [xmlel()].
-
+-spec disco_local_identity([identity()], jid(), jid(),
+ binary(), binary()) -> [identity()].
disco_local_identity(Acc, _From, To, <<>>, _Lang) ->
case lists:member(?PEPNODE, plugins(host(To#jid.lserver))) of
true ->
- [#xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, <<"pep">>}]}
- | Acc];
+ [#identity{category = <<"pubsub">>, type = <<"pep">>} | Acc];
false ->
Acc
end;
disco_local_identity(Acc, _From, _To, _Node, _Lang) ->
Acc.
--spec disco_local_features(Acc :: [xmlel()], _From :: jid(),
- To :: jid(), Node :: <<>> | mod_pubsub:nodeId(),
- Lang :: binary()) -> [binary(),...].
-
+-spec disco_local_features({error, stanza_error()} | {result, [binary()]} | empty,
+ jid(), jid(), binary(), binary()) ->
+ {error, stanza_error()} | {result, [binary()]} | empty.
disco_local_features(Acc, _From, To, <<>>, _Lang) ->
Host = host(To#jid.lserver),
Feats = case Acc of
{result, I} -> I;
_ -> []
end,
- {result, Feats ++ [feature(F) || F <- features(Host, <<>>)]};
+ {result, Feats ++ [?NS_PUBSUB|[feature(F) || F <- features(Host, <<>>)]]};
disco_local_features(Acc, _From, _To, _Node, _Lang) ->
Acc.
+-spec disco_local_items({error, stanza_error()} | {result, [disco_item()]} | empty,
+ jid(), jid(), binary(), binary()) ->
+ {error, stanza_error()} | {result, [disco_item()]} | empty.
disco_local_items(Acc, _From, _To, <<>>, _Lang) -> Acc;
disco_local_items(Acc, _From, _To, _Node, _Lang) -> Acc.
-%disco_sm_identity(Acc, From, To, Node, Lang)
-% when is_binary(Node) ->
-% disco_sm_identity(Acc, From, To, iolist_to_binary(Node),
-% Lang);
--spec disco_sm_identity(Acc :: empty | [xmlel()], From :: jid(),
- To :: jid(), Node :: mod_pubsub:nodeId(),
- Lang :: binary()) -> [xmlel()].
-
-disco_sm_identity(empty, From, To, Node, Lang) ->
- disco_sm_identity([], From, To, Node, Lang);
+-spec disco_sm_identity([identity()], jid(), jid(),
+ binary(), binary()) -> [identity()].
disco_sm_identity(Acc, From, To, Node, _Lang) ->
disco_identity(jid:tolower(jid:remove_resource(To)), Node, From)
++ Acc.
+-spec disco_identity(host(), binary(), jid()) -> [identity()].
disco_identity(_Host, <<>>, _From) ->
- [#xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, <<"pep">>}]}];
+ [#identity{category = <<"pubsub">>, type = <<"pep">>}];
disco_identity(Host, Node, From) ->
- Action = fun (#pubsub_node{id = Nidx, type = Type, options = Options, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners) of
- {result, _} ->
- {result, [#xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, <<"pep">>}]},
- #xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, <<"leaf">>}
- | case get_option(Options, title) of
- false -> [];
- [Title] -> [{<<"name">>, Title}]
- end]}]};
- _ ->
- {result, []}
- end
- end,
+ Action =
+ fun(#pubsub_node{id = Nidx, type = Type,
+ options = Options, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case get_allowed_items_call(Host, Nidx, From, Type,
+ Options, Owners) of
+ {result, _} ->
+ {result, [#identity{category = <<"pubsub">>, type = <<"pep">>},
+ #identity{category = <<"pubsub">>, type = <<"leaf">>,
+ name = get_option(Options, title, <<>>)}]};
+ _ ->
+ {result, []}
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> Result;
_ -> []
end.
--spec disco_sm_features(Acc :: empty | {result, Features::[Feature::binary()]},
- From :: jid(), To :: jid(), Node :: mod_pubsub:nodeId(),
- Lang :: binary()) -> {result, Features::[Feature::binary()]}.
-%disco_sm_features(Acc, From, To, Node, Lang)
-% when is_binary(Node) ->
-% disco_sm_features(Acc, From, To, iolist_to_binary(Node),
-% Lang);
+-spec disco_sm_features({error, stanza_error()} | {result, [binary()]} | empty,
+ jid(), jid(), binary(), binary()) ->
+ {error, stanza_error()} | {result, [binary()]}.
disco_sm_features(empty, From, To, Node, Lang) ->
disco_sm_features({result, []}, From, To, Node, Lang);
disco_sm_features({result, OtherFeatures} = _Acc, From, To, Node, _Lang) ->
{result,
- OtherFeatures ++
- disco_features(jid:tolower(jid:remove_resource(To)), Node, From)};
+ OtherFeatures ++
+ disco_features(jid:tolower(jid:remove_resource(To)), Node, From)};
disco_sm_features(Acc, _From, _To, _Node, _Lang) -> Acc.
+-spec disco_features(ljid(), binary(), jid()) -> [binary()].
disco_features(Host, <<>>, _From) ->
[?NS_PUBSUB | [feature(F) || F <- plugin_features(Host, <<"pep">>)]];
disco_features(Host, Node, From) ->
- Action = fun (#pubsub_node{id = Nidx, type = Type, options = Options, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners) of
- {result, _} -> {result, [?NS_PUBSUB | [feature(F) || F <- plugin_features(Host, <<"pep">>)]]};
- _ -> {result, []}
- end
- end,
+ Action =
+ fun(#pubsub_node{id = Nidx, type = Type,
+ options = Options, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case get_allowed_items_call(Host, Nidx, From,
+ Type, Options, Owners) of
+ {result, _} ->
+ {result,
+ [?NS_PUBSUB | [feature(F) || F <- plugin_features(Host, <<"pep">>)]]};
+ _ ->
+ {result, []}
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> Result;
_ -> []
end.
--spec disco_sm_items(Acc :: empty | {result, [xmlel()]}, From :: jid(),
- To :: jid(), Node :: mod_pubsub:nodeId(),
- Lang :: binary()) -> {result, [xmlel()]}.
-%disco_sm_items(Acc, From, To, Node, Lang)
-% when is_binary(Node) ->
-% disco_sm_items(Acc, From, To, iolist_to_binary(Node),
-% Lang);
+-spec disco_sm_items({error, stanza_error()} | {result, [disco_item()]} | empty,
+ jid(), jid(), binary(), binary()) ->
+ {error, stanza_error()} | {result, [disco_item()]}.
disco_sm_items(empty, From, To, Node, Lang) ->
disco_sm_items({result, []}, From, To, Node, Lang);
disco_sm_items({result, OtherItems}, From, To, Node, _Lang) ->
@@ -609,59 +496,65 @@ disco_sm_items({result, OtherItems}, From, To, Node, _Lang) ->
disco_items(jid:tolower(jid:remove_resource(To)), Node, From))};
disco_sm_items(Acc, _From, _To, _Node, _Lang) -> Acc.
--spec disco_items(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- From :: jid()) -> [xmlel()].
+-spec disco_items(ljid(), binary(), jid()) -> [disco_item()].
disco_items(Host, <<>>, From) ->
- Action = fun (#pubsub_node{nodeid = {_, Node},
- options = Options, type = Type, id = Nidx, owners = O},
- Acc) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners) of
- {result, _} ->
- [#xmlel{name = <<"item">>,
- attrs = [{<<"node">>, (Node)},
- {<<"jid">>, jid:to_string(Host)}
- | case get_option(Options, title) of
- false -> [];
- [Title] -> [{<<"name">>, Title}]
- end]}
- | Acc];
- _ ->
- Acc
- end
- end,
+ MaxNodes = mod_pubsub_opt:max_nodes_discoitems(serverhost(Host)),
+ Action =
+ fun(#pubsub_node{nodeid = {_, Node}, options = Options,
+ type = Type, id = Nidx, owners = O}, Acc) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case get_allowed_items_call(Host, Nidx, From,
+ Type, Options, Owners) of
+ {result, _} ->
+ [#disco_item{node = Node,
+ jid = jid:make(Host),
+ name = get_option(Options, title, <<>>)} | Acc];
+ _ ->
+ Acc
+ end
+ end,
NodeBloc = fun() ->
- {result,
- lists:foldl(Action, [], tree_call(Host, get_nodes, [Host]))}
- end,
+ case tree_call(Host, get_nodes, [Host, MaxNodes]) of
+ Nodes when is_list(Nodes) ->
+ {result, lists:foldl(Action, [], Nodes)};
+ Error ->
+ Error
+ end
+ end,
case transaction(Host, NodeBloc, sync_dirty) of
{result, Items} -> Items;
_ -> []
end;
disco_items(Host, Node, From) ->
- Action = fun (#pubsub_node{id = Nidx, type = Type, options = Options, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners) of
- {result, Items} ->
- {result, [#xmlel{name = <<"item">>,
- attrs = [{<<"jid">>, jid:to_string(Host)},
- {<<"name">>, ItemId}]}
- || #pubsub_item{itemid = {ItemId, _}} <- Items]};
- _ ->
- {result, []}
- end
- end,
+ Action =
+ fun(#pubsub_node{id = Nidx, type = Type,
+ options = Options, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case get_allowed_items_call(Host, Nidx, From,
+ Type, Options, Owners) of
+ {result, Items} ->
+ {result, [#disco_item{jid = jid:make(Host),
+ name = ItemId}
+ || #pubsub_item{itemid = {ItemId, _}} <- Items]};
+ _ ->
+ {result, []}
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> Result;
_ -> []
end.
%% -------
-%% presence hooks handling functions
+%% presence and session hooks handling functions
%%
-caps_add(#jid{luser = U, lserver = S, lresource = R}, #jid{lserver = Host} = JID, _Features)
- when Host =/= S ->
+-spec caps_add(jid(), jid(), [binary()]) -> ok.
+caps_add(JID, JID, _Features) ->
+ %% Send the owner his last PEP items.
+ send_last_pep(JID, JID);
+caps_add(#jid{lserver = S1} = From, #jid{lserver = S2} = To, _Features)
+ when S1 =/= S2 ->
%% When a remote contact goes online while the local user is offline, the
%% remote contact won't receive last items from the local user even if
%% ignore_pep_from_offline is set to false. To work around this issue a bit,
@@ -671,138 +564,166 @@ caps_add(#jid{luser = U, lserver = S, lresource = R}, #jid{lserver = Host} = JID
%% contact becomes available; the former is also executed when the local
%% user goes online (because that triggers the contact to send a presence
%% packet with CAPS).
- presence(Host, {presence, U, S, [R], JID});
+ send_last_pep(To, From);
caps_add(_From, _To, _Feature) ->
ok.
-caps_update(#jid{luser = U, lserver = S, lresource = R}, #jid{lserver = Host} = JID, _Features) ->
- presence(Host, {presence, U, S, [R], JID}).
+-spec caps_update(jid(), jid(), [binary()]) -> ok.
+caps_update(From, To, _Features) ->
+ send_last_pep(To, From).
-presence_probe(#jid{luser = U, lserver = S, lresource = R} = JID, JID, Pid) ->
- presence(S, {presence, JID, Pid}),
- presence(S, {presence, U, S, [R], JID});
+-spec presence_probe(jid(), jid(), pid()) -> ok.
presence_probe(#jid{luser = U, lserver = S}, #jid{luser = U, lserver = S}, _Pid) ->
- %% ignore presence_probe from my other ressources
- %% to not get duplicated last items
+ %% ignore presence_probe from my other resources
ok;
-presence_probe(#jid{luser = U, lserver = S, lresource = R}, #jid{lserver = S} = JID, _Pid) ->
- presence(S, {presence, U, S, [R], JID});
-presence_probe(_Host, _JID, _Pid) ->
- %% ignore presence_probe from remote contacts,
- %% those are handled via caps_add
+presence_probe(#jid{lserver = S} = From, #jid{lserver = S} = To, _Pid) ->
+ send_last_pep(To, From);
+presence_probe(_From, _To, _Pid) ->
+ %% ignore presence_probe from remote contacts, those are handled via caps_add
ok.
-presence(ServerHost, Presence) ->
- {SendLoop, _} = case whereis(gen_mod:get_module_proc(ServerHost, ?LOOPNAME)) of
- undefined -> init_send_loop(ServerHost);
- Pid -> {Pid, undefined}
- end,
- SendLoop ! Presence.
+-spec on_self_presence({presence(), ejabberd_c2s:state()})
+ -> {presence(), ejabberd_c2s:state()}.
+on_self_presence({_, #{pres_last := _}} = Acc) -> % Just a presence update.
+ Acc;
+on_self_presence({#presence{type = available}, #{jid := JID}} = Acc) ->
+ send_last_items(JID),
+ Acc;
+on_self_presence(Acc) ->
+ Acc.
+
+-spec on_user_offline(ejabberd_c2s:state(), atom()) -> ejabberd_c2s:state().
+on_user_offline(#{jid := JID} = C2SState, _Reason) ->
+ purge_offline(jid:tolower(JID)),
+ C2SState;
+on_user_offline(C2SState, _Reason) ->
+ C2SState.
%% -------
%% subscription hooks handling functions
%%
-out_subscription(User, Server, JID, subscribed) ->
- Owner = jid:make(User, Server, <<>>),
- {PUser, PServer, PResource} = jid:tolower(JID),
- PResources = case PResource of
- <<>> -> user_resources(PUser, PServer);
- _ -> [PResource]
- end,
- presence(Server, {presence, PUser, PServer, PResources, Owner}),
- true;
-out_subscription(_, _, _, _) ->
- true.
+-spec out_subscription(presence()) -> any().
+out_subscription(#presence{type = subscribed, from = From, to = To}) ->
+ if From#jid.lserver == To#jid.lserver ->
+ send_last_pep(jid:remove_resource(From), To);
+ true ->
+ ok
+ end;
+out_subscription(_) ->
+ ok.
-in_subscription(_, User, Server, Owner, unsubscribed, _) ->
- unsubscribe_user(jid:make(User, Server, <<>>), Owner),
+-spec in_subscription(boolean(), presence()) -> true.
+in_subscription(_, #presence{to = To, from = Owner, type = unsubscribed}) ->
+ unsubscribe_user(jid:remove_resource(To), Owner),
true;
-in_subscription(_, _, _, _, _, _) ->
+in_subscription(_, _) ->
true.
+-spec unsubscribe_user(jid(), jid()) -> ok.
unsubscribe_user(Entity, Owner) ->
- spawn(fun () ->
- [unsubscribe_user(ServerHost, Entity, Owner) ||
- ServerHost <- lists:usort(lists:foldl(
- fun(UserHost, Acc) ->
- case gen_mod:is_loaded(UserHost, mod_pubsub) of
- true -> [UserHost|Acc];
- false -> Acc
- end
- end, [], [Entity#jid.lserver, Owner#jid.lserver]))]
- end).
+ lists:foreach(
+ fun(ServerHost) ->
+ unsubscribe_user(ServerHost, Entity, Owner)
+ end,
+ lists:usort(
+ lists:foldl(
+ fun(UserHost, Acc) ->
+ case gen_mod:is_loaded(UserHost, mod_pubsub) of
+ true -> [UserHost|Acc];
+ false -> Acc
+ end
+ end, [], [Entity#jid.lserver, Owner#jid.lserver]))).
+
+-spec unsubscribe_user(binary(), jid(), jid()) -> ok.
unsubscribe_user(Host, Entity, Owner) ->
BJID = jid:tolower(jid:remove_resource(Owner)),
- lists:foreach(fun (PType) ->
- {result, Subs} = node_action(Host, PType,
- get_entity_subscriptions,
- [Host, Entity]),
- lists:foreach(fun
- ({#pubsub_node{options = Options,
- owners = O,
- id = Nidx},
- subscribed, _, JID}) ->
- Unsubscribe = match_option(Options, access_model, presence)
- andalso lists:member(BJID, node_owners_action(Host, PType, Nidx, O)),
- case Unsubscribe of
- true ->
- node_action(Host, PType,
- unsubscribe_node, [Nidx, Entity, JID, all]);
- false ->
- ok
- end;
- (_) ->
- ok
- end,
- Subs)
- end,
- plugins(Host)).
+ lists:foreach(
+ fun (PType) ->
+ case node_action(Host, PType,
+ get_entity_subscriptions,
+ [Host, Entity]) of
+ {result, Subs} ->
+ lists:foreach(
+ fun({#pubsub_node{options = Options,
+ owners = O,
+ id = Nidx},
+ subscribed, _, JID}) ->
+ Unsubscribe = match_option(Options, access_model, presence)
+ andalso lists:member(BJID, node_owners_action(Host, PType, Nidx, O)),
+ case Unsubscribe of
+ true ->
+ node_action(Host, PType,
+ unsubscribe_node, [Nidx, Entity, JID, all]);
+ false ->
+ ok
+ end;
+ (_) ->
+ ok
+ end, Subs);
+ _ ->
+ ok
+ end
+ end, plugins(Host)).
%% -------
%% user remove hook handling function
%%
+-spec remove_user(binary(), binary()) -> ok.
remove_user(User, Server) ->
LUser = jid:nodeprep(User),
LServer = jid:nameprep(Server),
- Entity = jid:make(LUser, LServer, <<>>),
+ Entity = jid:make(LUser, LServer),
Host = host(LServer),
HomeTreeBase = <<"/home/", LServer/binary, "/", LUser/binary>>,
- spawn(fun () ->
- lists:foreach(fun (PType) ->
- {result, Subs} = node_action(Host, PType,
- get_entity_subscriptions,
- [Host, Entity]),
- lists:foreach(fun
- ({#pubsub_node{id = Nidx}, _, _, JID}) ->
- node_action(Host, PType,
+ lists:foreach(
+ fun(PType) ->
+ case node_action(Host, PType,
+ get_entity_subscriptions,
+ [Host, Entity]) of
+ {result, Subs} ->
+ lists:foreach(
+ fun({#pubsub_node{id = Nidx}, _, _, JID}) ->
+ node_action(Host, PType,
unsubscribe_node,
[Nidx, Entity, JID, all]);
- (_) ->
- ok
- end,
- Subs),
- {result, Affs} = node_action(Host, PType,
- get_entity_affiliations,
- [Host, Entity]),
- lists:foreach(fun
- ({#pubsub_node{nodeid = {H, N}, parents = []}, owner}) ->
+ (_) ->
+ ok
+ end, Subs),
+ case node_action(Host, PType,
+ get_entity_affiliations,
+ [Host, Entity]) of
+ {result, Affs} ->
+ lists:foreach(
+ fun({#pubsub_node{nodeid = {H, N}, parents = []}, owner}) ->
delete_node(H, N, Entity);
- ({#pubsub_node{nodeid = {H, N}, type = Type}, owner})
- when N == HomeTreeBase, Type == <<"hometree">> ->
+ ({#pubsub_node{nodeid = {H, N}, type = Type}, owner})
+ when N == HomeTreeBase, Type == <<"hometree">> ->
delete_node(H, N, Entity);
- ({#pubsub_node{id = Nidx}, publisher}) ->
- node_action(Host, PType,
- set_affiliation,
- [Nidx, Entity, none]);
- (_) ->
- ok
- end,
- Affs)
- end,
- plugins(Host))
- end).
+ ({#pubsub_node{id = Nidx}, _}) ->
+ case node_action(Host, PType,
+ get_state,
+ [Nidx, jid:tolower(Entity)]) of
+ {result, State} ->
+ ItemIds = State#pubsub_state.items,
+ node_action(Host, PType,
+ remove_extra_items,
+ [Nidx, 0, ItemIds]),
+ node_action(Host, PType,
+ set_affiliation,
+ [Nidx, Entity, none]);
+ _ ->
+ ok
+ end
+ end, Affs);
+ _ ->
+ ok
+ end;
+ _ ->
+ ok
+ end
+ end, plugins(Host)).
handle_call(server_host, _From, State) ->
{reply, State#state.server_host, State};
@@ -813,47 +734,30 @@ handle_call(pep_mapping, _From, State) ->
handle_call(nodetree, _From, State) ->
{reply, State#state.nodetree, State};
handle_call(stop, _From, State) ->
- {stop, normal, ok, State}.
-
-%%--------------------------------------------------------------------
-%% Function: handle_cast(Msg, State) -> {noreply, State} |
-%% {noreply, State, Timeout} |
-%% {stop, Reason, State}
-%% Description: Handling cast messages
-%%--------------------------------------------------------------------
-%% @private
-handle_cast(_Msg, State) -> {noreply, State}.
+ {stop, normal, ok, State};
+handle_call(Request, From, State) ->
+ ?WARNING_MSG("Unexpected call from ~p: ~p", [From, Request]),
+ {noreply, State}.
--spec handle_info(_ :: {route, From::jid(), To::jid(), Packet::xmlel()},
- State :: state()) -> {noreply, state()}.
+handle_cast(Msg, State) ->
+ ?WARNING_MSG("Unexpected cast: ~p", [Msg]),
+ {noreply, State}.
-%%--------------------------------------------------------------------
-%% Function: handle_info(Info, State) -> {noreply, State} |
-%% {noreply, State, Timeout} |
-%% {stop, Reason, State}
-%% Description: Handling all non call/cast messages
-%%--------------------------------------------------------------------
-%% @private
-handle_info({route, From, To, Packet},
- #state{server_host = ServerHost, access = Access, plugins = Plugins} = State) ->
- case catch do_route(ServerHost, Access, Plugins, To#jid.lserver, From, To, Packet) of
- {'EXIT', Reason} -> ?ERROR_MSG("~p", [Reason]);
- _ -> ok
+handle_info({route, Packet}, State) ->
+ try route(Packet)
+ catch ?EX_RULE(Class, Reason, St) ->
+ StackTrace = ?EX_STACK(St),
+ ?ERROR_MSG("Failed to route packet:~n~ts~n** ~ts",
+ [xmpp:pp(Packet),
+ misc:format_exception(2, Class, Reason, StackTrace)])
end,
{noreply, State};
-handle_info(_Info, State) ->
+handle_info(Info, State) ->
+ ?WARNING_MSG("Unexpected info: ~p", [Info]),
{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.
-%%--------------------------------------------------------------------
-%% @private
terminate(_Reason,
- #state{host = Host, server_host = ServerHost, nodetree = TreePlugin, plugins = Plugins}) ->
+ #state{hosts = Hosts, server_host = ServerHost, nodetree = TreePlugin, plugins = Plugins}) ->
case lists:member(?PEPNODE, Plugins) of
true ->
ejabberd_hooks:delete(caps_add, ServerHost,
@@ -873,7 +777,9 @@ terminate(_Reason,
false ->
ok
end,
- ejabberd_hooks:delete(sm_remove_connection_hook, ServerHost,
+ ejabberd_hooks:delete(c2s_self_presence, ServerHost,
+ ?MODULE, on_self_presence, 75),
+ ejabberd_hooks:delete(c2s_terminated, ServerHost,
?MODULE, on_user_offline, 75),
ejabberd_hooks:delete(disco_local_identity, ServerHost,
?MODULE, disco_local_identity, 75),
@@ -889,17 +795,19 @@ 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),
- mod_disco:unregister_feature(ServerHost, ?NS_PUBSUB),
- case whereis(gen_mod:get_module_proc(ServerHost, ?LOOPNAME)) of
- undefined ->
- ?ERROR_MSG("~s process is dead, pubsub was broken", [?LOOPNAME]);
- Pid ->
- Pid ! stop
- end,
- terminate_plugins(Host, ServerHost, Plugins, TreePlugin),
- ejabberd_router:unregister_route(Host).
+ ejabberd_hooks:delete(c2s_handle_info, ServerHost,
+ ?MODULE, c2s_handle_info, 50),
+ lists:foreach(
+ fun(Host) ->
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_DISCO_INFO),
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_DISCO_ITEMS),
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_PUBSUB),
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_PUBSUB_OWNER),
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_VCARD),
+ gen_iq_handler:remove_iq_handler(ejabberd_local, Host, ?NS_COMMANDS),
+ terminate_plugins(Host, ServerHost, Plugins, TreePlugin),
+ ejabberd_router:unregister_route(Host)
+ end, Hosts).
%%--------------------------------------------------------------------
%% Func: code_change(OldVsn, State, Extra) -> {ok, NewState}
@@ -908,187 +816,188 @@ terminate(_Reason,
%% @private
code_change(_OldVsn, State, _Extra) -> {ok, State}.
--spec do_route(ServerHost :: binary(), Access :: atom(),
- Plugins :: [binary(),...], Host :: mod_pubsub:hostPubsub(),
- From :: jid(), To :: jid(), Packet :: xmlel()) -> ok.
-
%%--------------------------------------------------------------------
%%% Internal functions
%%--------------------------------------------------------------------
-do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
- #xmlel{name = Name, attrs = Attrs} = Packet,
+-spec process_disco_info(iq()) -> iq().
+process_disco_info(#iq{type = set, lang = Lang} = IQ) ->
+ Txt = ?T("Value 'set' of 'type' attribute is not allowed"),
+ xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang));
+process_disco_info(#iq{from = From, to = To, lang = Lang, type = get,
+ sub_els = [#disco_info{node = Node}]} = IQ) ->
+ Host = To#jid.lserver,
+ ServerHost = ejabberd_router:host_of_route(Host),
+ Info = ejabberd_hooks:run_fold(disco_info, ServerHost,
+ [],
+ [ServerHost, ?MODULE, <<>>, <<>>]),
+ case iq_disco_info(ServerHost, Host, Node, From, Lang) of
+ {result, IQRes} ->
+ XData = IQRes#disco_info.xdata ++ Info,
+ xmpp:make_iq_result(IQ, IQRes#disco_info{node = Node, xdata = XData});
+ {error, Error} ->
+ xmpp:make_error(IQ, Error)
+ end.
+
+-spec process_disco_items(iq()) -> iq().
+process_disco_items(#iq{type = set, lang = Lang} = IQ) ->
+ Txt = ?T("Value 'set' of 'type' attribute is not allowed"),
+ xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang));
+process_disco_items(#iq{type = get, from = From, to = To,
+ sub_els = [#disco_items{node = Node} = SubEl]} = IQ) ->
+ Host = To#jid.lserver,
+ case iq_disco_items(Host, Node, From, SubEl#disco_items.rsm) of
+ {result, IQRes} ->
+ xmpp:make_iq_result(IQ, IQRes#disco_items{node = Node});
+ {error, Error} ->
+ xmpp:make_error(IQ, Error)
+ end.
+
+-spec process_pubsub(iq()) -> iq().
+process_pubsub(#iq{to = To} = IQ) ->
+ Host = To#jid.lserver,
+ ServerHost = ejabberd_router:host_of_route(Host),
+ Access = config(ServerHost, access),
+ case iq_pubsub(Host, Access, IQ) of
+ {result, IQRes} ->
+ xmpp:make_iq_result(IQ, IQRes);
+ {error, Error} ->
+ xmpp:make_error(IQ, Error)
+ end.
+
+-spec process_pubsub_owner(iq()) -> iq().
+process_pubsub_owner(#iq{to = To} = IQ) ->
+ Host = To#jid.lserver,
+ case iq_pubsub_owner(Host, IQ) of
+ {result, IQRes} ->
+ xmpp:make_iq_result(IQ, IQRes);
+ {error, Error} ->
+ xmpp:make_error(IQ, Error)
+ end.
+
+-spec process_vcard(iq()) -> iq().
+process_vcard(#iq{type = get, to = To, lang = Lang} = IQ) ->
+ ServerHost = ejabberd_router:host_of_route(To#jid.lserver),
+ xmpp:make_iq_result(IQ, iq_get_vcard(ServerHost, Lang));
+process_vcard(#iq{type = set, lang = Lang} = IQ) ->
+ Txt = ?T("Value 'set' of 'type' attribute is not allowed"),
+ xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang)).
+
+-spec process_commands(iq()) -> iq().
+process_commands(#iq{type = set, to = To, from = From,
+ sub_els = [#adhoc_command{} = Request]} = IQ) ->
+ Host = To#jid.lserver,
+ ServerHost = ejabberd_router:host_of_route(Host),
+ Plugins = config(ServerHost, plugins),
+ Access = config(ServerHost, access),
+ case adhoc_request(Host, ServerHost, From, Request, Access, Plugins) of
+ {error, Error} ->
+ xmpp:make_error(IQ, Error);
+ Response ->
+ xmpp:make_iq_result(
+ IQ, xmpp_util:make_adhoc_response(Request, Response))
+ end;
+process_commands(#iq{type = get, lang = Lang} = IQ) ->
+ Txt = ?T("Value 'get' of 'type' attribute is not allowed"),
+ xmpp:make_error(IQ, xmpp:err_not_allowed(Txt, Lang)).
+
+-spec route(stanza()) -> ok.
+route(#iq{to = To} = IQ) when To#jid.lresource == <<"">> ->
+ ejabberd_router:process_iq(IQ);
+route(Packet) ->
+ To = xmpp:get_to(Packet),
case To of
#jid{luser = <<>>, lresource = <<>>} ->
- case Name of
- <<"iq">> ->
- case jlib:iq_query_info(Packet) of
- #iq{type = get, xmlns = ?NS_DISCO_INFO, sub_el = SubEl, lang = Lang} = IQ ->
- #xmlel{attrs = QAttrs} = SubEl,
- Node = fxml:get_attr_s(<<"node">>, QAttrs),
- Info = ejabberd_hooks:run_fold(disco_info, ServerHost,
- [],
- [ServerHost, ?MODULE, <<>>, <<>>]),
- Res = case iq_disco_info(Host, Node, From, Lang) of
- {result, IQRes} ->
- jlib:iq_to_xml(IQ#iq{type = result,
- sub_el =
- [#xmlel{name = <<"query">>,
- attrs = QAttrs,
- children = IQRes ++ Info}]});
- {error, Error} ->
- jlib:make_error_reply(Packet, Error)
- end,
- ejabberd_router:route(To, From, Res);
- #iq{type = get, xmlns = ?NS_DISCO_ITEMS, sub_el = SubEl} = IQ ->
- #xmlel{attrs = QAttrs} = SubEl,
- Node = fxml:get_attr_s(<<"node">>, QAttrs),
- Res = case iq_disco_items(Host, Node, From, jlib:rsm_decode(IQ)) of
- {result, IQRes} ->
- jlib:iq_to_xml(IQ#iq{type = result,
- sub_el =
- [#xmlel{name = <<"query">>,
- attrs = QAttrs,
- children = IQRes}]})
- %{error, Error} ->
- % jlib:make_error_reply(Packet, Error)
- end,
- ejabberd_router:route(To, From, Res);
- #iq{type = IQType, xmlns = ?NS_PUBSUB, lang = Lang, sub_el = SubEl} = IQ ->
- Res = case iq_pubsub(Host, ServerHost, From, IQType,
- SubEl, Lang, Access, Plugins)
- of
- {result, IQRes} ->
- jlib:iq_to_xml(IQ#iq{type = result, sub_el = IQRes});
- {error, Error} ->
- jlib:make_error_reply(Packet, Error)
- end,
- ejabberd_router:route(To, From, Res);
- #iq{type = IQType, xmlns = ?NS_PUBSUB_OWNER, lang = Lang, sub_el = SubEl} = IQ ->
- Res = case iq_pubsub_owner(Host, ServerHost, From,
- IQType, SubEl, Lang)
- of
- {result, IQRes} ->
- jlib:iq_to_xml(IQ#iq{type = result, sub_el = IQRes});
- {error, Error} ->
- jlib:make_error_reply(Packet, Error)
- end,
- ejabberd_router:route(To, From, Res);
- #iq{type = get, xmlns = (?NS_VCARD) = XMLNS, lang = Lang, sub_el = _SubEl} = IQ ->
- Res = IQ#iq{type = result,
- sub_el =
- [#xmlel{name = <<"vCard">>,
- attrs = [{<<"xmlns">>, XMLNS}],
- children = iq_get_vcard(Lang)}]},
- ejabberd_router:route(To, From, jlib:iq_to_xml(Res));
- #iq{type = set, xmlns = ?NS_COMMANDS} = IQ ->
- Res = case iq_command(Host, ServerHost, From, IQ, Access, Plugins) of
- {error, Error} ->
- jlib:make_error_reply(Packet, Error);
- {result, IQRes} ->
- jlib:iq_to_xml(IQ#iq{type = result, sub_el = IQRes})
- end,
- ejabberd_router:route(To, From, Res);
- #iq{} ->
- Err = jlib:make_error_reply(Packet, ?ERR_FEATURE_NOT_IMPLEMENTED),
- ejabberd_router:route(To, From, Err);
- _ ->
- ok
- end;
- <<"message">> ->
- case fxml:get_attr_s(<<"type">>, Attrs) of
- <<"error">> ->
+ case Packet of
+ #message{type = T} when T /= error ->
+ case find_authorization_response(Packet) of
+ undefined ->
ok;
- _ ->
- case find_authorization_response(Packet) of
- none ->
- ok;
- invalid ->
- Lang = fxml:get_attr_s(<<"xml:lang">>, Attrs),
- Txt = <<"Incorrect authorization response">>,
- Err = jlib:make_error_reply(
- Packet, ?ERRT_BAD_REQUEST(Lang, Txt)),
- ejabberd_router:route(To, From, Err);
- XFields ->
- handle_authorization_response(Host, From, To, Packet, XFields)
- end
+ {error, Err} ->
+ ejabberd_router:route_error(Packet, Err);
+ AuthResponse ->
+ handle_authorization_response(
+ To#jid.lserver, Packet, AuthResponse)
end;
_ ->
- ok
+ Err = xmpp:err_service_unavailable(),
+ ejabberd_router:route_error(Packet, Err)
end;
_ ->
- case fxml:get_attr_s(<<"type">>, Attrs) of
- <<"error">> ->
- ok;
- <<"result">> ->
- ok;
- _ ->
- Err = jlib:make_error_reply(Packet, ?ERR_ITEM_NOT_FOUND),
- ejabberd_router:route(To, From, Err)
- end
+ Err = xmpp:err_item_not_found(),
+ ejabberd_router:route_error(Packet, Err)
end.
+-spec command_disco_info(binary(), binary(), jid()) -> {result, disco_info()}.
command_disco_info(_Host, ?NS_COMMANDS, _From) ->
- IdentityEl = #xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"automation">>},
- {<<"type">>, <<"command-list">>}]},
- {result, [IdentityEl]};
+ {result, #disco_info{identities = [#identity{category = <<"automation">>,
+ type = <<"command-list">>}]}};
command_disco_info(_Host, ?NS_PUBSUB_GET_PENDING, _From) ->
- IdentityEl = #xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"automation">>},
- {<<"type">>, <<"command-node">>}]},
- FeaturesEl = #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_COMMANDS}]},
- {result, [IdentityEl, FeaturesEl]}.
+ {result, #disco_info{identities = [#identity{category = <<"automation">>,
+ type = <<"command-node">>}],
+ features = [?NS_COMMANDS]}}.
+-spec node_disco_info(binary(), binary(), jid()) -> {result, disco_info()} |
+ {error, stanza_error()}.
node_disco_info(Host, Node, From) ->
node_disco_info(Host, Node, From, true, true).
+-spec node_disco_info(binary(), binary(), jid(), boolean(), boolean()) ->
+ {result, disco_info()} | {error, stanza_error()}.
node_disco_info(Host, Node, _From, _Identity, _Features) ->
- Action = fun (#pubsub_node{type = Type, options = Options}) ->
- NodeType = case get_option(Options, node_type) of
- collection -> <<"collection">>;
- _ -> <<"leaf">>
- end,
- I = #xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, NodeType}]},
- F = [#xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_PUBSUB}]}
- | [#xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, feature(F)}]}
- || F <- plugin_features(Host, Type)]],
- {result, [I | F]}
- end,
+ Action =
+ fun(#pubsub_node{id = Nidx, type = Type, options = Options}) ->
+ NodeType = case get_option(Options, node_type) of
+ collection -> <<"collection">>;
+ _ -> <<"leaf">>
+ end,
+ Affs = case node_call(Host, Type, get_node_affiliations, [Nidx]) of
+ {result, As} -> As;
+ _ -> []
+ end,
+ Subs = case node_call(Host, Type, get_node_subscriptions, [Nidx]) of
+ {result, Ss} -> Ss;
+ _ -> []
+ end,
+ Meta = [{title, get_option(Options, title, <<>>)},
+ {description, get_option(Options, description, <<>>)},
+ {owner, [jid:make(LJID) || {LJID, Aff} <- Affs, Aff =:= owner]},
+ {publisher, [jid:make(LJID) || {LJID, Aff} <- Affs, Aff =:= publisher]},
+ {access_model, get_option(Options, access_model, open)},
+ {publish_model, get_option(Options, publish_model, publishers)},
+ {num_subscribers, length(Subs)}],
+ XData = #xdata{type = result,
+ fields = pubsub_meta_data:encode(Meta)},
+ Is = [#identity{category = <<"pubsub">>, type = NodeType}],
+ Fs = [?NS_PUBSUB | [feature(F) || F <- plugin_features(Host, Type)]],
+ {result, #disco_info{identities = Is, features = Fs, xdata = [XData]}}
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> {result, Result};
Other -> Other
end.
-iq_disco_info(Host, SNode, From, Lang) ->
+-spec iq_disco_info(binary(), binary(), binary(), jid(), binary())
+ -> {result, disco_info()} | {error, stanza_error()}.
+iq_disco_info(ServerHost, Host, SNode, From, Lang) ->
[Node | _] = case SNode of
- <<>> -> [<<>>];
- _ -> str:tokens(SNode, <<"!">>)
- end,
- % Node = string_to_node(RealSNode),
+ <<>> -> [<<>>];
+ _ -> str:tokens(SNode, <<"!">>)
+ end,
case Node of
<<>> ->
- {result, [#xmlel{name = <<"identity">>,
- attrs = [{<<"category">>, <<"pubsub">>},
- {<<"type">>, <<"service">>},
- {<<"name">>, translate:translate(Lang, <<"Publish-Subscribe">>)}]},
- #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_DISCO_INFO}]},
- #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_DISCO_ITEMS}]},
- #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_PUBSUB}]},
- #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_COMMANDS}]},
- #xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, ?NS_VCARD}]}]
- ++ [#xmlel{name = <<"feature">>,
- attrs = [{<<"var">>, feature(F)}]}
- || F <- features(Host, Node)]};
+ Name = mod_pubsub_opt:name(ServerHost),
+ {result,
+ #disco_info{
+ identities = [#identity{
+ category = <<"pubsub">>,
+ type = <<"service">>,
+ name = translate:translate(Lang, Name)}],
+ features = [?NS_DISCO_INFO,
+ ?NS_DISCO_ITEMS,
+ ?NS_PUBSUB,
+ ?NS_COMMANDS,
+ ?NS_VCARD |
+ [feature(F) || F <- features(Host, Node)]]}};
?NS_COMMANDS ->
command_disco_info(Host, Node, From);
?NS_PUBSUB_GET_PENDING ->
@@ -1097,541 +1006,460 @@ iq_disco_info(Host, SNode, From, Lang) ->
node_disco_info(Host, Node, From)
end.
--spec iq_disco_items(Host :: mod_pubsub:host(), Node :: <<>> | mod_pubsub:nodeId(),
- From :: jid(), Rsm :: none | rsm_in()) -> {result, [xmlel()]}.
-iq_disco_items(Host, <<>>, From, _RSM) ->
- {result,
- lists:map(fun (#pubsub_node{nodeid = {_, SubNode}, options = Options}) ->
- Attrs = case get_option(Options, title) of
- false ->
- [{<<"jid">>, Host}
- | nodeAttr(SubNode)];
- Title ->
- [{<<"jid">>, Host},
- {<<"name">>, Title}
- | nodeAttr(SubNode)]
- end,
- #xmlel{name = <<"item">>, attrs = Attrs}
- end,
- tree_action(Host, get_subnodes, [Host, <<>>, From]))};
+-spec iq_disco_items(host(), binary(), jid(), undefined | rsm_set()) ->
+ {result, disco_items()} | {error, stanza_error()}.
+iq_disco_items(Host, <<>>, _From, _RSM) ->
+ MaxNodes = mod_pubsub_opt:max_nodes_discoitems(serverhost(Host)),
+ case tree_action(Host, get_subnodes, [Host, <<>>, MaxNodes]) of
+ {error, #stanza_error{}} = Err ->
+ Err;
+ Nodes when is_list(Nodes) ->
+ Items =
+ lists:map(
+ fun(#pubsub_node{nodeid = {_, SubNode}, options = Options}) ->
+ case get_option(Options, title) of
+ false ->
+ #disco_item{jid = jid:make(Host),
+ node = SubNode};
+ Title ->
+ #disco_item{jid = jid:make(Host),
+ name = Title,
+ node = SubNode}
+ end
+ end, Nodes),
+ {result, #disco_items{items = Items}}
+ end;
iq_disco_items(Host, ?NS_COMMANDS, _From, _RSM) ->
- {result, [#xmlel{name = <<"item">>,
- attrs = [{<<"jid">>, Host},
- {<<"node">>, ?NS_PUBSUB_GET_PENDING},
- {<<"name">>, <<"Get Pending">>}]}]};
+ {result,
+ #disco_items{items = [#disco_item{jid = jid:make(Host),
+ node = ?NS_PUBSUB_GET_PENDING,
+ name = ?T("Get Pending")}]}};
iq_disco_items(_Host, ?NS_PUBSUB_GET_PENDING, _From, _RSM) ->
- {result, []};
+ {result, #disco_items{}};
iq_disco_items(Host, Item, From, RSM) ->
case str:tokens(Item, <<"!">>) of
[_Node, _ItemId] ->
- {result, []};
+ {result, #disco_items{}};
[Node] ->
- Action = fun (#pubsub_node{id = Nidx, type = Type, options = Options, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- {NodeItems, RsmOut} = case get_allowed_items_call(Host, Nidx,
- From, Type, Options, Owners, RSM)
- of
- {result, R} -> R;
- _ -> {[], none}
- end,
- Nodes = lists:map(fun (#pubsub_node{nodeid = {_, SubNode}, options = SubOptions}) ->
- Attrs = case get_option(SubOptions, title) of
- false ->
- [{<<"jid">>, Host}
- | nodeAttr(SubNode)];
- Title ->
- [{<<"jid">>, Host},
- {<<"name">>, Title}
- | nodeAttr(SubNode)]
- end,
- #xmlel{name = <<"item">>, attrs = Attrs}
- end,
- tree_call(Host, get_subnodes, [Host, Node, From])),
- Items = lists:map(fun (#pubsub_item{itemid = {RN, _}}) ->
- {result, Name} = node_call(Host, Type, get_item_name, [Host, Node, RN]),
- #xmlel{name = <<"item">>,
- attrs = [{<<"jid">>, Host}, {<<"name">>, Name}]}
- end,
- NodeItems),
- {result, Nodes ++ Items ++ jlib:rsm_encode(RsmOut)}
- end,
+ MaxNodes = mod_pubsub_opt:max_nodes_discoitems(serverhost(Host)),
+ Action = fun(#pubsub_node{id = Nidx, type = Type, options = Options, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ {NodeItems, RsmOut} = case get_allowed_items_call(
+ Host, Nidx, From, Type, Options, Owners, RSM) of
+ {result, R} -> R;
+ _ -> {[], undefined}
+ end,
+ case tree_call(Host, get_subnodes, [Host, Node, MaxNodes]) of
+ SubNodes when is_list(SubNodes) ->
+ Nodes = lists:map(
+ fun(#pubsub_node{nodeid = {_, SubNode}, options = SubOptions}) ->
+ case get_option(SubOptions, title) of
+ false ->
+ #disco_item{jid = jid:make(Host),
+ node = SubNode};
+ Title ->
+ #disco_item{jid = jid:make(Host),
+ name = Title,
+ node = SubNode}
+ end
+ end, SubNodes),
+ Items = lists:flatmap(
+ fun(#pubsub_item{itemid = {RN, _}}) ->
+ case node_call(Host, Type, get_item_name, [Host, Node, RN]) of
+ {result, Name} ->
+ [#disco_item{jid = jid:make(Host), name = Name}];
+ _ ->
+ []
+ end
+ end, NodeItems),
+ {result, #disco_items{items = Nodes ++ Items,
+ rsm = RsmOut}};
+ Error ->
+ Error
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> {result, Result};
Other -> Other
end
end.
--spec iq_sm(From :: jid(), To :: jid(), IQ :: iq_request()) -> iq_result() | iq_error().
-iq_sm(From, To, #iq{type = Type, sub_el = SubEl, xmlns = XMLNS, lang = Lang} = IQ) ->
- ServerHost = To#jid.lserver,
+-spec iq_sm(iq()) -> iq().
+iq_sm(#iq{to = To, sub_els = [SubEl]} = IQ) ->
LOwner = jid:tolower(jid:remove_resource(To)),
- Res = case XMLNS of
- ?NS_PUBSUB ->
- iq_pubsub(LOwner, ServerHost, From, Type, SubEl, Lang);
- ?NS_PUBSUB_OWNER ->
- iq_pubsub_owner(LOwner, ServerHost, From, Type, SubEl, Lang)
- end,
+ Res = case xmpp:get_ns(SubEl) of
+ ?NS_PUBSUB ->
+ iq_pubsub(LOwner, all, IQ);
+ ?NS_PUBSUB_OWNER ->
+ iq_pubsub_owner(LOwner, IQ)
+ end,
case Res of
- {result, IQRes} -> IQ#iq{type = result, sub_el = IQRes};
- {error, Error} -> IQ#iq{type = error, sub_el = [Error, SubEl]}
+ {result, IQRes} ->
+ xmpp:make_iq_result(IQ, IQRes);
+ {error, Error} ->
+ xmpp:make_error(IQ, Error)
end.
-iq_get_vcard(Lang) ->
- [#xmlel{name = <<"FN">>, attrs = [],
- children = [{xmlcdata, <<"ejabberd/mod_pubsub">>}]},
- #xmlel{name = <<"URL">>, attrs = [],
- children = [{xmlcdata, ?EJABBERD_URI}]},
- #xmlel{name = <<"DESC">>, attrs = [],
- children = [{xmlcdata,
- <<(translate:translate(Lang, <<"ejabberd Publish-Subscribe module">>))/binary,
- "\nCopyright (c) 2004-2016 ProcessOne">>}]}].
-
--spec iq_pubsub(Host :: mod_pubsub:host(), ServerHost :: binary(), From :: jid(),
- IQType :: 'get' | 'set', SubEl :: xmlel(), Lang :: binary()) ->
- {result, [xmlel()]} | {error, xmlel()}.
-
-iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang) ->
- iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, all, plugins(Host)).
-
--spec iq_pubsub(Host :: mod_pubsub:host(), ServerHost :: binary(), From :: jid(),
- IQType :: 'get' | 'set', SubEl :: xmlel(), Lang :: binary(),
- Access :: atom(), Plugins :: [binary(),...]) ->
- {result, [xmlel()]} | {error, xmlel()}.
-
-iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
- #xmlel{children = SubEls} = SubEl,
- case fxml:remove_cdata(SubEls) of
- [#xmlel{name = Name, attrs = Attrs, children = Els} | Rest] ->
- Node = fxml:get_attr_s(<<"node">>, Attrs),
- case {IQType, Name} of
- {set, <<"create">>} ->
- Config = case Rest of
- [#xmlel{name = <<"configure">>, children = C}] -> C;
- _ -> []
- end,
- Type = case fxml:get_attr_s(<<"type">>, Attrs) of
- <<>> -> hd(Plugins);
- T -> T
- end,
- case lists:member(Type, Plugins) of
- false ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"create-nodes">>)};
- true ->
- create_node(Host, ServerHost, Node, From, Type, Access, Config)
- end;
- {set, <<"publish">>} ->
- case fxml:remove_cdata(Els) of
- [#xmlel{name = <<"item">>, attrs = ItemAttrs,
- children = Payload}] ->
- ItemId = fxml:get_attr_s(<<"id">>, ItemAttrs),
- PubOpts = case [C || #xmlel{name = <<"publish-options">>,
- children = [C]} <- Rest] of
- [XEl] ->
- case jlib:parse_xdata_submit(XEl) of
- invalid -> [];
- Form -> Form
- end;
- _ -> []
- end,
- publish_item(Host, ServerHost, Node, From, ItemId, Payload, PubOpts, Access);
- [] ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"item-required">>)};
- _ ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"invalid-payload">>)}
+-spec iq_get_vcard(binary(), binary()) -> vcard_temp().
+iq_get_vcard(ServerHost, Lang) ->
+ case mod_pubsub_opt:vcard(ServerHost) of
+ undefined ->
+ Desc = misc:get_descr(Lang, ?T("ejabberd Publish-Subscribe module")),
+ #vcard_temp{fn = <<"ejabberd/mod_pubsub">>,
+ url = ejabberd_config:get_uri(),
+ desc = Desc};
+ VCard ->
+ VCard
+ end.
+
+-spec iq_pubsub(binary() | ljid(), atom(), iq()) ->
+ {result, pubsub()} | {error, stanza_error()}.
+iq_pubsub(Host, Access, #iq{from = From, type = IQType, lang = Lang,
+ sub_els = [SubEl]}) ->
+ case {IQType, SubEl} of
+ {set, #pubsub{create = Node, configure = Configure,
+ _ = undefined}} when is_binary(Node) ->
+ ServerHost = serverhost(Host),
+ Plugins = config(ServerHost, plugins),
+ Config = case Configure of
+ {_, XData} -> decode_node_config(XData, Host, Lang);
+ undefined -> []
+ end,
+ Type = hd(Plugins),
+ case Config of
+ {error, _} = Err ->
+ Err;
+ _ ->
+ create_node(Host, ServerHost, Node, From, Type, Access, Config)
+ end;
+ {set, #pubsub{publish = #ps_publish{node = Node, items = Items},
+ publish_options = XData, configure = _, _ = undefined}} ->
+ ServerHost = serverhost(Host),
+ case Items of
+ [#ps_item{id = ItemId, sub_els = Payload}] ->
+ case decode_publish_options(XData, Lang) of
+ {error, _} = Err ->
+ Err;
+ PubOpts ->
+ publish_item(Host, ServerHost, Node, From, ItemId,
+ Payload, PubOpts, Access)
end;
- {set, <<"retract">>} ->
- ForceNotify = case fxml:get_attr_s(<<"notify">>, Attrs) of
- <<"1">> -> true;
- <<"true">> -> true;
- _ -> false
- end,
- case fxml:remove_cdata(Els) of
- [#xmlel{name = <<"item">>, attrs = ItemAttrs}] ->
- ItemId = fxml:get_attr_s(<<"id">>, ItemAttrs),
- delete_item(Host, Node, From, ItemId, ForceNotify);
- _ ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"item-required">>)}
+ [] ->
+ publish_item(Host, ServerHost, Node, From, <<>>, [], [], Access);
+ _ ->
+ {error, extended_error(xmpp:err_bad_request(), err_invalid_payload())}
+ end;
+ {set, #pubsub{retract = #ps_retract{node = Node, notify = Notify, items = Items},
+ _ = undefined}} ->
+ case Items of
+ [#ps_item{id = ItemId}] ->
+ if ItemId /= <<>> ->
+ delete_item(Host, Node, From, ItemId, Notify);
+ true ->
+ {error, extended_error(xmpp:err_bad_request(),
+ err_item_required())}
end;
- {set, <<"subscribe">>} ->
- Config = case Rest of
- [#xmlel{name = <<"options">>, children = C}] -> C;
- _ -> []
- end,
- JID = fxml:get_attr_s(<<"jid">>, Attrs),
- subscribe_node(Host, Node, From, JID, Config);
- {set, <<"unsubscribe">>} ->
- JID = fxml:get_attr_s(<<"jid">>, Attrs),
- SubId = fxml:get_attr_s(<<"subid">>, Attrs),
- unsubscribe_node(Host, Node, From, JID, SubId);
- {get, <<"items">>} ->
- MaxItems = fxml:get_attr_s(<<"max_items">>, Attrs),
- SubId = fxml:get_attr_s(<<"subid">>, Attrs),
- ItemIds = lists:foldl(fun
- (#xmlel{name = <<"item">>, attrs = ItemAttrs}, Acc) ->
- case fxml:get_attr_s(<<"id">>, ItemAttrs) of
- <<>> -> Acc;
- ItemId -> [ItemId | Acc]
- end;
- (_, Acc) ->
- Acc
- end,
- [], fxml:remove_cdata(Els)),
- get_items(Host, Node, From, SubId, MaxItems, ItemIds, jlib:rsm_decode(SubEl));
- {get, <<"subscriptions">>} ->
- get_subscriptions(Host, Node, From, Plugins);
- {get, <<"affiliations">>} ->
- get_affiliations(Host, Node, From, Plugins);
- {get, <<"options">>} ->
- SubId = fxml:get_attr_s(<<"subid">>, Attrs),
- JID = fxml:get_attr_s(<<"jid">>, Attrs),
- get_options(Host, Node, JID, SubId, Lang);
- {set, <<"options">>} ->
- SubId = fxml:get_attr_s(<<"subid">>, Attrs),
- JID = fxml:get_attr_s(<<"jid">>, Attrs),
- set_options(Host, Node, JID, SubId, Els);
+ [] ->
+ {error, extended_error(xmpp:err_bad_request(), err_item_required())};
_ ->
- {error, ?ERR_FEATURE_NOT_IMPLEMENTED}
+ {error, extended_error(xmpp:err_bad_request(), err_invalid_payload())}
end;
- Other ->
- ?INFO_MSG("Too many actions: ~p", [Other]),
- {error, ?ERR_BAD_REQUEST}
- end.
-
-
--spec iq_pubsub_owner(Host :: mod_pubsub:host(), ServerHost :: binary(), From :: jid(),
- IQType :: 'get' | 'set', SubEl :: xmlel(), Lang :: binary()) ->
- {result, [xmlel()]} | {error, xmlel()}.
-
-iq_pubsub_owner(Host, ServerHost, From, IQType, SubEl, Lang) ->
- #xmlel{children = SubEls} = SubEl,
- Action = fxml:remove_cdata(SubEls),
- case Action of
- [#xmlel{name = Name, attrs = Attrs, children = Els}] ->
- Node = fxml:get_attr_s(<<"node">>, Attrs),
- case {IQType, Name} of
- {get, <<"configure">>} ->
- get_configure(Host, ServerHost, Node, From, Lang);
- {set, <<"configure">>} ->
- set_configure(Host, Node, From, Els, Lang);
- {get, <<"default">>} ->
- get_default(Host, Node, From, Lang);
- {set, <<"delete">>} ->
- delete_node(Host, Node, From);
- {set, <<"purge">>} ->
- purge_node(Host, Node, From);
- {get, <<"subscriptions">>} ->
- get_subscriptions(Host, Node, From);
- {set, <<"subscriptions">>} ->
- set_subscriptions(Host, Node, From, fxml:remove_cdata(Els));
- {get, <<"affiliations">>} ->
- get_affiliations(Host, Node, From);
- {set, <<"affiliations">>} ->
- set_affiliations(Host, Node, From, fxml:remove_cdata(Els));
+ {set, #pubsub{subscribe = #ps_subscribe{node = Node, jid = JID},
+ options = Options, _ = undefined}} ->
+ Config = case Options of
+ #ps_options{xdata = XData, jid = undefined, node = <<>>} ->
+ decode_subscribe_options(XData, Lang);
+ #ps_options{xdata = _XData, jid = #jid{}} ->
+ Txt = ?T("Attribute 'jid' is not allowed here"),
+ {error, xmpp:err_bad_request(Txt, Lang)};
+ #ps_options{xdata = _XData} ->
+ Txt = ?T("Attribute 'node' is not allowed here"),
+ {error, xmpp:err_bad_request(Txt, Lang)};
+ _ ->
+ []
+ end,
+ case Config of
+ {error, _} = Err ->
+ Err;
_ ->
- {error, ?ERR_FEATURE_NOT_IMPLEMENTED}
+ subscribe_node(Host, Node, From, JID, Config)
end;
+ {set, #pubsub{unsubscribe = #ps_unsubscribe{node = Node, jid = JID, subid = SubId},
+ _ = undefined}} ->
+ unsubscribe_node(Host, Node, From, JID, SubId);
+ {get, #pubsub{items = #ps_items{node = Node,
+ max_items = MaxItems,
+ subid = SubId,
+ items = Items},
+ rsm = RSM, _ = undefined}} ->
+ ItemIds = [ItemId || #ps_item{id = ItemId} <- Items, ItemId /= <<>>],
+ get_items(Host, Node, From, SubId, MaxItems, ItemIds, RSM);
+ {get, #pubsub{subscriptions = {Node, _}, _ = undefined}} ->
+ Plugins = config(serverhost(Host), plugins),
+ get_subscriptions(Host, Node, From, Plugins);
+ {get, #pubsub{affiliations = {Node, _}, _ = undefined}} ->
+ Plugins = config(serverhost(Host), plugins),
+ get_affiliations(Host, Node, From, Plugins);
+ {_, #pubsub{options = #ps_options{jid = undefined}, _ = undefined}} ->
+ {error, extended_error(xmpp:err_bad_request(), err_jid_required())};
+ {_, #pubsub{options = #ps_options{node = <<>>}, _ = undefined}} ->
+ {error, extended_error(xmpp:err_bad_request(), err_nodeid_required())};
+ {get, #pubsub{options = #ps_options{node = Node, subid = SubId, jid = JID},
+ _ = undefined}} ->
+ get_options(Host, Node, JID, SubId, Lang);
+ {set, #pubsub{options = #ps_options{node = Node, subid = SubId,
+ jid = JID, xdata = XData},
+ _ = undefined}} ->
+ case decode_subscribe_options(XData, Lang) of
+ {error, _} = Err ->
+ Err;
+ Config ->
+ set_options(Host, Node, JID, SubId, Config)
+ end;
+ {set, #pubsub{}} ->
+ {error, xmpp:err_bad_request()};
_ ->
- ?INFO_MSG("Too many actions: ~p", [Action]),
- {error, ?ERR_BAD_REQUEST}
+ {error, xmpp:err_feature_not_implemented()}
end.
-iq_command(Host, ServerHost, From, IQ, Access, Plugins) ->
- case adhoc:parse_request(IQ) of
- Req when is_record(Req, adhoc_request) ->
- case adhoc_request(Host, ServerHost, From, Req, Access, Plugins) of
- Resp when is_record(Resp, adhoc_response) ->
- {result, [adhoc:produce_response(Req, Resp)]};
- Error ->
- Error
+-spec iq_pubsub_owner(binary() | ljid(), iq()) -> {result, pubsub_owner() | undefined} |
+ {error, stanza_error()}.
+iq_pubsub_owner(Host, #iq{type = IQType, from = From,
+ lang = Lang, sub_els = [SubEl]}) ->
+ case {IQType, SubEl} of
+ {get, #pubsub_owner{configure = {Node, undefined}, _ = undefined}} ->
+ ServerHost = serverhost(Host),
+ get_configure(Host, ServerHost, Node, From, Lang);
+ {set, #pubsub_owner{configure = {Node, XData}, _ = undefined}} ->
+ case XData of
+ undefined ->
+ {error, xmpp:err_bad_request(?T("No data form found"), Lang)};
+ #xdata{type = cancel} ->
+ {result, #pubsub_owner{}};
+ #xdata{type = submit} ->
+ case decode_node_config(XData, Host, Lang) of
+ {error, _} = Err ->
+ Err;
+ Config ->
+ set_configure(Host, Node, From, Config, Lang)
+ end;
+ #xdata{} ->
+ {error, xmpp:err_bad_request(?T("Incorrect data form"), Lang)}
end;
- Err -> Err
+ {get, #pubsub_owner{default = {Node, undefined}, _ = undefined}} ->
+ get_default(Host, Node, From, Lang);
+ {set, #pubsub_owner{delete = {Node, _}, _ = undefined}} ->
+ delete_node(Host, Node, From);
+ {set, #pubsub_owner{purge = Node, _ = undefined}} when Node /= undefined ->
+ purge_node(Host, Node, From);
+ {get, #pubsub_owner{subscriptions = {Node, []}, _ = undefined}} ->
+ get_subscriptions(Host, Node, From);
+ {set, #pubsub_owner{subscriptions = {Node, Subs}, _ = undefined}} ->
+ set_subscriptions(Host, Node, From, Subs);
+ {get, #pubsub_owner{affiliations = {Node, []}, _ = undefined}} ->
+ get_affiliations(Host, Node, From);
+ {set, #pubsub_owner{affiliations = {Node, Affs}, _ = undefined}} ->
+ set_affiliations(Host, Node, From, Affs);
+ {_, #pubsub_owner{}} ->
+ {error, xmpp:err_bad_request()};
+ _ ->
+ {error, xmpp:err_feature_not_implemented()}
end.
-%% @doc <p>Processes an Ad Hoc Command.</p>
+-spec adhoc_request(binary(), binary(), jid(), adhoc_command(),
+ atom(), [binary()]) -> adhoc_command() | {error, stanza_error()}.
adhoc_request(Host, _ServerHost, Owner,
- #adhoc_request{node = ?NS_PUBSUB_GET_PENDING,
- lang = Lang, action = <<"execute">>,
- xdata = false},
- _Access, Plugins) ->
+ #adhoc_command{node = ?NS_PUBSUB_GET_PENDING, lang = Lang,
+ action = execute, xdata = undefined},
+ _Access, Plugins) ->
send_pending_node_form(Host, Owner, Lang, Plugins);
adhoc_request(Host, _ServerHost, Owner,
- #adhoc_request{node = ?NS_PUBSUB_GET_PENDING,
- action = <<"execute">>, xdata = XData, lang = Lang},
- _Access, _Plugins) ->
- ParseOptions = case XData of
- #xmlel{name = <<"x">>} = XEl ->
- case jlib:parse_xdata_submit(XEl) of
- invalid ->
- Txt = <<"Incorrect data form">>,
- {error, ?ERRT_BAD_REQUEST(Lang, Txt)};
- XData2 ->
- case set_xoption(Host, XData2, []) of
- NewOpts when is_list(NewOpts) -> {result, NewOpts};
- Err -> Err
- end
- end;
- _ ->
- Txt = <<"No data form found">>,
- {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
- end,
- case ParseOptions of
- {result, XForm} ->
- case lists:keysearch(node, 1, XForm) of
- {value, {_, Node}} -> send_pending_auth_events(Host, Node, Owner);
- false -> {error, extended_error(?ERR_BAD_REQUEST, <<"bad-payload">>)}
- end;
- Error -> Error
+ #adhoc_command{node = ?NS_PUBSUB_GET_PENDING, lang = Lang,
+ action = execute, xdata = #xdata{} = XData} = Request,
+ _Access, _Plugins) ->
+ case decode_get_pending(XData, Lang) of
+ {error, _} = Err ->
+ Err;
+ Config ->
+ Node = proplists:get_value(node, Config),
+ case send_pending_auth_events(Host, Node, Owner, Lang) of
+ ok ->
+ xmpp_util:make_adhoc_response(
+ Request, #adhoc_command{status = completed});
+ Err ->
+ Err
+ end
end;
adhoc_request(_Host, _ServerHost, _Owner,
- #adhoc_request{action = <<"cancel">>}, _Access,
- _Plugins) ->
- #adhoc_response{status = canceled};
-adhoc_request(Host, ServerHost, Owner,
- #adhoc_request{action = <<>>} = R, Access, Plugins) ->
- adhoc_request(Host, ServerHost, Owner,
- R#adhoc_request{action = <<"execute">>}, Access,
- Plugins);
+ #adhoc_command{action = cancel}, _Access, _Plugins) ->
+ #adhoc_command{status = canceled};
adhoc_request(_Host, _ServerHost, _Owner, Other, _Access, _Plugins) ->
?DEBUG("Couldn't process ad hoc command:~n~p", [Other]),
- {error, ?ERR_ITEM_NOT_FOUND}.
+ {error, xmpp:err_item_not_found()}.
-%% @doc <p>Sends the process pending subscriptions XForm for Host to Owner.</p>
-send_pending_node_form(Host, Owner, _Lang, Plugins) ->
+-spec send_pending_node_form(binary(), jid(), binary(),
+ [binary()]) -> adhoc_command() | {error, stanza_error()}.
+send_pending_node_form(Host, Owner, Lang, Plugins) ->
Filter = fun (Type) ->
lists:member(<<"get-pending">>, plugin_features(Host, Type))
end,
case lists:filter(Filter, Plugins) of
[] ->
- Err = extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
- unsupported, <<"get-pending">>),
+ Err = extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('get-pending')),
{error, Err};
Ps ->
- XOpts = [#xmlel{name = <<"option">>, attrs = [],
- children = [#xmlel{name = <<"value">>,
- attrs = [],
- children = [{xmlcdata, Node}]}]}
- || Node <- get_pending_nodes(Host, Owner, Ps)],
- XForm = #xmlel{name = <<"x">>,
- attrs = [{<<"xmlns">>, ?NS_XDATA},
- {<<"type">>, <<"form">>}],
- children = [#xmlel{name = <<"field">>,
- attrs = [{<<"type">>, <<"list-single">>},
- {<<"var">>, <<"pubsub#node">>}],
- children = lists:usort(XOpts)}]},
- #adhoc_response{status = executing,
- defaultaction = <<"execute">>, elements = [XForm]}
+ case get_pending_nodes(Host, Owner, Ps) of
+ {ok, Nodes} ->
+ Form = [{node, <<>>, lists:zip(Nodes, Nodes)}],
+ XForm = #xdata{type = form,
+ fields = pubsub_get_pending:encode(Form, Lang)},
+ #adhoc_command{status = executing, action = execute,
+ xdata = XForm};
+ Err ->
+ Err
+ end
end.
+-spec get_pending_nodes(binary(), jid(), [binary()]) -> {ok, [binary()]} |
+ {error, stanza_error()}.
get_pending_nodes(Host, Owner, Plugins) ->
Tr = fun (Type) ->
case node_call(Host, Type, get_pending_nodes, [Host, Owner]) of
{result, Nodes} -> Nodes;
_ -> []
end
- end,
+ end,
Action = fun() -> {result, lists:flatmap(Tr, Plugins)} end,
case transaction(Host, Action, sync_dirty) of
- {result, Res} -> Res;
+ {result, Res} -> {ok, Res};
Err -> Err
end.
%% @doc <p>Send a subscription approval form to Owner for all pending
%% subscriptions on Host and Node.</p>
-send_pending_auth_events(Host, Node, Owner) ->
- ?DEBUG("Sending pending auth events for ~s on ~s:~s",
- [jid:to_string(Owner), Host, Node]),
- Action = fun (#pubsub_node{id = Nidx, type = Type}) ->
- case lists:member(<<"get-pending">>, plugin_features(Host, Type)) of
- true ->
- case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
- {result, owner} -> node_call(Host, Type, get_node_subscriptions, [Nidx]);
- _ -> {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
- end;
- false ->
- {error, extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
- unsupported, <<"get-pending">>)}
- end
- end,
+-spec send_pending_auth_events(binary(), binary(), jid(),
+ binary()) -> ok | {error, stanza_error()}.
+send_pending_auth_events(Host, Node, Owner, Lang) ->
+ ?DEBUG("Sending pending auth events for ~ts on ~ts:~ts",
+ [jid:encode(Owner), Host, Node]),
+ Action =
+ fun(#pubsub_node{id = Nidx, type = Type}) ->
+ case lists:member(<<"get-pending">>, plugin_features(Host, Type)) of
+ true ->
+ case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
+ {result, owner} ->
+ node_call(Host, Type, get_node_subscriptions, [Nidx]);
+ _ ->
+ {error, xmpp:err_forbidden(
+ ?T("Owner privileges required"), Lang)}
+ end;
+ false ->
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('get-pending'))}
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {N, Subs}} ->
- lists:foreach(fun
- ({J, pending, _SubId}) -> send_authorization_request(N, jid:make(J));
- ({J, pending}) -> send_authorization_request(N, jid:make(J));
- (_) -> ok
- end,
- Subs),
- #adhoc_response{};
+ lists:foreach(
+ fun({J, pending, _SubId}) -> send_authorization_request(N, jid:make(J));
+ ({J, pending}) -> send_authorization_request(N, jid:make(J));
+ (_) -> ok
+ end, Subs);
Err ->
Err
end.
%%% authorization handling
-
-send_authorization_request(#pubsub_node{nodeid = {Host, Node}, type = Type, id = Nidx, owners = O},
- Subscriber) ->
+-spec send_authorization_request(#pubsub_node{}, jid()) -> ok.
+send_authorization_request(#pubsub_node{nodeid = {Host, Node},
+ type = Type, id = Nidx, owners = O},
+ Subscriber) ->
+ %% TODO: pass lang to this function
Lang = <<"en">>,
- Stanza = #xmlel{name = <<"message">>, attrs = [],
- children =
- [#xmlel{name = <<"x">>,
- attrs =
- [{<<"xmlns">>, ?NS_XDATA},
- {<<"type">>, <<"form">>}],
- children =
- [#xmlel{name = <<"title">>, attrs = [],
- children =
- [{xmlcdata,
- translate:translate(Lang, <<"PubSub subscriber request">>)}]},
- #xmlel{name = <<"instructions">>,
- attrs = [],
- children =
- [{xmlcdata,
- translate:translate(Lang,
- <<"Choose whether to approve this entity's "
- "subscription.">>)}]},
- #xmlel{name = <<"field">>,
- attrs =
- [{<<"var">>, <<"FORM_TYPE">>},
- {<<"type">>, <<"hidden">>}],
- children =
- [#xmlel{name = <<"value">>,
- attrs = [],
- children =
- [{xmlcdata, ?NS_PUBSUB_SUB_AUTH}]}]},
- #xmlel{name = <<"field">>,
- attrs =
- [{<<"var">>, <<"pubsub#node">>},
- {<<"type">>,
- <<"text-single">>},
- {<<"label">>, translate:translate(Lang, <<"Node ID">>)}],
- children =
- [#xmlel{name = <<"value">>,
- attrs = [],
- children =
- [{xmlcdata, Node}]}]},
- #xmlel{name = <<"field">>,
- attrs =
- [{<<"var">>,
- <<"pubsub#subscriber_jid">>},
- {<<"type">>, <<"jid-single">>},
- {<<"label">>,
- translate:translate(Lang, <<"Subscriber Address">>)}],
- children =
- [#xmlel{name = <<"value">>,
- attrs = [],
- children =
- [{xmlcdata, jid:to_string(Subscriber)}]}]},
- #xmlel{name = <<"field">>,
- attrs =
- [{<<"var">>,
- <<"pubsub#allow">>},
- {<<"type">>, <<"boolean">>},
- {<<"label">>,
- translate:translate(Lang,
- <<"Allow this Jabber ID to subscribe to "
- "this pubsub node?">>)}],
- children =
- [#xmlel{name = <<"value">>,
- attrs = [],
- children =
- [{xmlcdata, <<"false">>}]}]}]}]},
- lists:foreach(fun (Owner) ->
- ejabberd_router:route(service_jid(Host), jid:make(Owner), Stanza)
- end,
- node_owners_action(Host, Type, Nidx, O)).
+ Fs = pubsub_subscribe_authorization:encode(
+ [{node, Node},
+ {subscriber_jid, Subscriber},
+ {allow, false}],
+ Lang),
+ X = #xdata{type = form,
+ title = translate:translate(
+ Lang, ?T("PubSub subscriber request")),
+ instructions = [translate:translate(
+ Lang,
+ ?T("Choose whether to approve this entity's "
+ "subscription."))],
+ fields = Fs},
+ Stanza = #message{from = service_jid(Host), sub_els = [X]},
+ lists:foreach(
+ fun (Owner) ->
+ ejabberd_router:route(xmpp:set_to(Stanza, jid:make(Owner)))
+ end, node_owners_action(Host, Type, Nidx, O)).
+-spec find_authorization_response(message()) -> undefined |
+ pubsub_subscribe_authorization:result() |
+ {error, stanza_error()}.
find_authorization_response(Packet) ->
- #xmlel{children = Els} = Packet,
- XData1 = lists:map(fun
- (#xmlel{name = <<"x">>, attrs = XAttrs} = XEl) ->
- case fxml:get_attr_s(<<"xmlns">>, XAttrs) of
- ?NS_XDATA ->
- case fxml:get_attr_s(<<"type">>, XAttrs) of
- <<"cancel">> -> none;
- _ -> jlib:parse_xdata_submit(XEl)
- end;
- _ ->
- none
- end;
- (_) ->
- none
- end,
- fxml:remove_cdata(Els)),
- XData = lists:filter(fun (E) -> E /= none end, XData1),
- case XData of
- [invalid] ->
- invalid;
- [] ->
- none;
- [XFields] when is_list(XFields) ->
- ?DEBUG("XFields: ~p", [XFields]),
- case lists:keysearch(<<"FORM_TYPE">>, 1, XFields) of
- {value, {_, [?NS_PUBSUB_SUB_AUTH]}} -> XFields;
- _ -> invalid
- end
+ case xmpp:get_subtag(Packet, #xdata{type = form}) of
+ #xdata{type = cancel} ->
+ undefined;
+ #xdata{type = submit, fields = Fs} ->
+ try pubsub_subscribe_authorization:decode(Fs) of
+ Result -> Result
+ catch _:{pubsub_subscribe_authorization, Why} ->
+ Lang = xmpp:get_lang(Packet),
+ Txt = pubsub_subscribe_authorization:format_error(Why),
+ {error, xmpp:err_bad_request(Txt, Lang)}
+ end;
+ #xdata{} ->
+ {error, xmpp:err_bad_request()};
+ false ->
+ undefined
end.
%% @doc Send a message to JID with the supplied Subscription
+-spec send_authorization_approval(binary(), jid(), binary(), subscribed | none) -> ok.
send_authorization_approval(Host, JID, SNode, Subscription) ->
- SubAttrs = case Subscription of
- %{S, SID} ->
- % [{<<"subscription">>, subscription_to_string(S)},
- % {<<"subid">>, SID}];
- S ->
- [{<<"subscription">>, subscription_to_string(S)}]
- end,
- Stanza = event_stanza(<<"subscription">>,
- [{<<"jid">>, jid:to_string(JID)}
- | nodeAttr(SNode)]
- ++ SubAttrs),
- ejabberd_router:route(service_jid(Host), JID, Stanza).
-
-handle_authorization_response(Host, From, To, Packet, XFields) ->
- Lang = fxml:get_tag_attr_s(<<"xml:lang">>, Packet),
- case {lists:keysearch(<<"pubsub#node">>, 1, XFields),
- lists:keysearch(<<"pubsub#subscriber_jid">>, 1, XFields),
- lists:keysearch(<<"pubsub#allow">>, 1, XFields)}
- of
- {{value, {_, [Node]}},
- {value, {_, [SSubscriber]}},
- {value, {_, [SAllow]}}} ->
- FromLJID = jid:tolower(jid:remove_resource(From)),
- Subscriber = jid:from_string(SSubscriber),
- Allow = case SAllow of
- <<"1">> -> true;
- <<"true">> -> true;
- _ -> false
- end,
- Action = fun (#pubsub_node{type = Type, id = Nidx, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case lists:member(FromLJID, Owners) of
- true ->
- {result, Subs} = node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]),
- update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs);
- false ->
- {error, ?ERRT_FORBIDDEN(Lang, <<"You're not an owner">>)}
- end
- end,
- case transaction(Host, Node, Action, sync_dirty) of
- {error, Error} ->
- Err = jlib:make_error_reply(Packet, Error),
- ejabberd_router:route(To, From, Err);
- {result, {_, _NewSubscription}} ->
- %% XXX: notify about subscription state change, section 12.11
- ok;
- _ ->
- Err = jlib:make_error_reply(Packet, ?ERR_INTERNAL_SERVER_ERROR),
- ejabberd_router:route(To, From, Err)
- end;
- _ ->
- Txt = <<"Incorrect data form">>,
- Err = jlib:make_error_reply(Packet, ?ERRT_NOT_ACCEPTABLE(Lang, Txt)),
- ejabberd_router:route(To, From, Err)
+ Event = #ps_event{subscription =
+ #ps_subscription{jid = JID,
+ node = SNode,
+ type = Subscription}},
+ Stanza = #message{from = service_jid(Host), to = JID, sub_els = [Event]},
+ ejabberd_router:route(Stanza).
+
+-spec handle_authorization_response(binary(), message(),
+ pubsub_subscribe_authorization:result()) -> ok.
+handle_authorization_response(Host, #message{from = From} = Packet, Response) ->
+ Node = proplists:get_value(node, Response),
+ Subscriber = proplists:get_value(subscriber_jid, Response),
+ Allow = proplists:get_value(allow, Response),
+ Lang = xmpp:get_lang(Packet),
+ FromLJID = jid:tolower(jid:remove_resource(From)),
+ Action =
+ fun(#pubsub_node{type = Type, id = Nidx, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case lists:member(FromLJID, Owners) of
+ true ->
+ case node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]) of
+ {result, Subs} ->
+ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs);
+ {error, _} = Err ->
+ Err
+ end;
+ false ->
+ {error, xmpp:err_forbidden(?T("Owner privileges required"), Lang)}
+ end
+ end,
+ case transaction(Host, Node, Action, sync_dirty) of
+ {error, Error} ->
+ ejabberd_router:route_error(Packet, Error);
+ {result, {_, _NewSubscription}} ->
+ %% XXX: notify about subscription state change, section 12.11
+ ok
end.
+-spec update_auth(binary(), binary(), _, _, jid() | error, boolean(), _) ->
+ {result, ok} | {error, stanza_error()}.
update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
Sub= lists:filter(fun
({pending, _}) -> true;
@@ -1648,69 +1476,10 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
send_authorization_approval(Host, Subscriber, Node, NewSub),
{result, ok};
_ ->
- Txt = <<"No pending subscriptions found">>,
- {error, ?ERRT_UNEXPECTED_REQUEST(?MYLANG, Txt)}
+ Txt = ?T("No pending subscriptions found"),
+ {error, xmpp:err_unexpected_request(Txt, ejabberd_option:language())}
end.
--define(XFIELD(Type, Label, Var, Val),
- #xmlel{name = <<"field">>,
- attrs = [{<<"type">>, Type},
- {<<"label">>, translate:translate(Lang, Label)},
- {<<"var">>, Var}],
- children = [#xmlel{name = <<"value">>, attrs = [],
- children = [{xmlcdata, Val}]}]}).
-
--define(BOOLXFIELD(Label, Var, Val),
- ?XFIELD(<<"boolean">>, Label, Var,
- case Val of
- true -> <<"1">>;
- _ -> <<"0">>
- end)).
-
--define(STRINGXFIELD(Label, Var, Val),
- ?XFIELD(<<"text-single">>, Label, Var, Val)).
-
--define(STRINGMXFIELD(Label, Var, Vals),
- #xmlel{name = <<"field">>,
- attrs = [{<<"type">>, <<"text-multi">>},
- {<<"label">>, translate:translate(Lang, Label)},
- {<<"var">>, Var}],
- children = [#xmlel{name = <<"value">>, attrs = [],
- children = [{xmlcdata, V}]}
- || V <- Vals]}).
-
--define(XFIELDOPT(Type, Label, Var, Val, Opts),
- #xmlel{name = <<"field">>,
- attrs = [{<<"type">>, Type},
- {<<"label">>, translate:translate(Lang, Label)},
- {<<"var">>, Var}],
- children = [#xmlel{name = <<"option">>, attrs = [],
- children = [#xmlel{name = <<"value">>,
- attrs = [],
- children = [{xmlcdata, Opt}]}]}
- || Opt <- Opts]
- ++
- [#xmlel{name = <<"value">>, attrs = [],
- children = [{xmlcdata, Val}]}]}).
-
--define(LISTXFIELD(Label, Var, Val, Opts),
- ?XFIELDOPT(<<"list-single">>, Label, Var, Val, Opts)).
-
--define(LISTMXFIELD(Label, Var, Vals, Opts),
- #xmlel{name = <<"field">>,
- attrs = [{<<"type">>, <<"list-multi">>},
- {<<"label">>, translate:translate(Lang, Label)},
- {<<"var">>, Var}],
- children = [#xmlel{name = <<"option">>, attrs = [],
- children = [#xmlel{name = <<"value">>,
- attrs = [],
- children = [{xmlcdata, Opt}]}]}
- || Opt <- Opts]
- ++
- [#xmlel{name = <<"value">>, attrs = [],
- children = [{xmlcdata, Val}]}
- || Val <- Vals]}).
-
%% @doc <p>Create new pubsub nodes</p>
%%<p>In addition to method-specific error conditions, there are several general reasons why the node creation request might fail:</p>
%%<ul>
@@ -1728,149 +1497,136 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
%%<li>nodetree create_node checks if nodeid already exists</li>
%%<li>node plugin create_node just sets default affiliation/subscription</li>
%%</ul>
--spec create_node(Host :: mod_pubsub:host(), ServerHost :: binary(),
- Node :: <<>> | mod_pubsub:nodeId(), Owner :: jid(),
- Type :: binary()) -> {result, [xmlel(),...]} | {error, xmlel()}.
+-spec create_node(host(), binary(), binary(), jid(),
+ binary()) -> {result, pubsub()} | {error, stanza_error()}.
create_node(Host, ServerHost, Node, Owner, Type) ->
create_node(Host, ServerHost, Node, Owner, Type, all, []).
--spec create_node(Host :: mod_pubsub:host(), ServerHost :: binary(),
- Node :: <<>> | mod_pubsub:nodeId(), Owner :: jid(),
- Type :: binary(), Access :: atom(), Configuration :: [xmlel()]) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
-
+-spec create_node(host(), binary(), binary(), jid(), binary(),
+ atom(), [{binary(), [binary()]}]) -> {result, pubsub()} | {error, stanza_error()}.
create_node(Host, ServerHost, <<>>, Owner, Type, Access, Configuration) ->
case lists:member(<<"instant-nodes">>, plugin_features(Host, Type)) of
true ->
- Node = randoms:get_string(),
+ Node = p1_rand:get_string(),
case create_node(Host, ServerHost, Node, Owner, Type, Access, Configuration) of
{result, _} ->
- {result, [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"create">>,
- attrs = nodeAttr(Node)}]}]};
+ {result, #pubsub{create = Node}};
Error ->
Error
end;
false ->
- {error, extended_error(?ERR_NOT_ACCEPTABLE, <<"nodeid-required">>)}
+ {error, extended_error(xmpp:err_not_acceptable(), err_nodeid_required())}
end;
create_node(Host, ServerHost, Node, Owner, GivenType, Access, Configuration) ->
Type = select_type(ServerHost, Host, Node, GivenType),
- ParseOptions = case fxml:remove_cdata(Configuration) of
- [] ->
- {result, node_options(Host, Type)};
- [#xmlel{name = <<"x">>} = XEl] ->
- case jlib:parse_xdata_submit(XEl) of
- invalid ->
- Txt = <<"Incorrect data form">>,
- {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)};
- XData ->
- case set_xoption(Host, XData, node_options(Host, Type)) of
- NewOpts when is_list(NewOpts) -> {result, NewOpts};
- Err -> Err
- end
+ NodeOptions = merge_config(
+ [node_config(Node, ServerHost),
+ Configuration, node_options(Host, Type)]),
+ CreateNode =
+ fun() ->
+ Parent = case node_call(Host, Type, node_to_path, [Node]) of
+ {result, [Node]} ->
+ <<>>;
+ {result, Path} ->
+ element(2, node_call(Host, Type, path_to_node,
+ [lists:sublist(Path, length(Path)-1)]))
+ end,
+ Parents = case Parent of
+ <<>> -> [];
+ _ -> [Parent]
+ end,
+ case node_call(Host, Type, create_node_permission,
+ [Host, ServerHost, Node, Parent, Owner, Access]) of
+ {result, true} ->
+ case tree_call(Host, create_node,
+ [Host, Node, Type, Owner, NodeOptions, Parents])
+ of
+ {ok, Nidx} ->
+ case get_node_subs_by_depth(Host, Node, Owner) of
+ {result, SubsByDepth} ->
+ case node_call(Host, Type, create_node, [Nidx, Owner]) of
+ {result, Result} -> {result, {Nidx, SubsByDepth, Result}};
+ Error -> Error
+ end;
+ Error ->
+ Error
+ end;
+ {error, {virtual, Nidx}} ->
+ case node_call(Host, Type, create_node, [Nidx, Owner]) of
+ {result, Result} -> {result, {Nidx, [], Result}};
+ Error -> Error
+ end;
+ Error ->
+ Error
+ end;
+ {result, _} ->
+ Txt = ?T("You're not allowed to create nodes"),
+ {error, xmpp:err_forbidden(Txt, ejabberd_option:language())};
+ Err ->
+ Err
+ end
+ end,
+ Reply = #pubsub{create = Node},
+ case transaction(Host, CreateNode, transaction) of
+ {result, {Nidx, SubsByDepth, {Result, broadcast}}} ->
+ broadcast_created_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth),
+ ejabberd_hooks:run(pubsub_create_node, ServerHost,
+ [ServerHost, Host, Node, Nidx, NodeOptions]),
+ case Result of
+ default -> {result, Reply};
+ _ -> {result, Result}
end;
- _ ->
- ?INFO_MSG("Node ~p; bad configuration: ~p", [Node, Configuration]),
- Txt = <<"No data form found">>,
- {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)}
- end,
- case ParseOptions of
- {result, NodeOptions} ->
- CreateNode = fun () ->
- Parent = case node_call(Host, Type, node_to_path, [Node]) of
- {result, [Node]} ->
- <<>>;
- {result, Path} ->
- element(2, node_call(Host, Type, path_to_node, [lists:sublist(Path, length(Path)-1)]))
- end,
- Parents = case Parent of
- <<>> -> [];
- _ -> [Parent]
- end,
- case node_call(Host, Type, create_node_permission,
- [Host, ServerHost, Node, Parent, Owner, Access])
- of
- {result, true} ->
- case tree_call(Host, create_node,
- [Host, Node, Type, Owner, NodeOptions, Parents])
- of
- {ok, Nidx} ->
- SubsByDepth = get_node_subs_by_depth(Host, Node, Owner),
- case node_call(Host, Type, create_node, [Nidx, Owner]) of
- {result, Result} -> {result, {Nidx, SubsByDepth, Result}};
- Error -> Error
- end;
- {error, {virtual, Nidx}} ->
- case node_call(Host, Type, create_node, [Nidx, Owner]) of
- {result, Result} -> {result, {Nidx, [], Result}};
- Error -> Error
- end;
- Error ->
- Error
- end;
- _ ->
- Txt1 = <<"You're not allowed to create nodes">>,
- {error, ?ERRT_FORBIDDEN(?MYLANG, Txt1)}
- end
- end,
- Reply = [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"create">>,
- attrs = nodeAttr(Node)}]}],
- case transaction(Host, CreateNode, transaction) of
- {result, {Nidx, SubsByDepth, {Result, broadcast}}} ->
- broadcast_created_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth),
- ejabberd_hooks:run(pubsub_create_node, ServerHost,
- [ServerHost, Host, Node, Nidx, NodeOptions]),
- case Result of
- default -> {result, Reply};
- _ -> {result, Result}
- end;
- {result, {Nidx, _SubsByDepth, Result}} ->
- ejabberd_hooks:run(pubsub_create_node, ServerHost,
- [ServerHost, Host, Node, Nidx, NodeOptions]),
- case Result of
- default -> {result, Reply};
- _ -> {result, Result}
- end;
- Error ->
- %% in case we change transaction to sync_dirty...
- %% node_call(Host, Type, delete_node, [Host, Node]),
- %% tree_call(Host, delete_node, [Host, Node]),
- Error
+ {result, {Nidx, _SubsByDepth, Result}} ->
+ ejabberd_hooks:run(pubsub_create_node, ServerHost,
+ [ServerHost, Host, Node, Nidx, NodeOptions]),
+ case Result of
+ default -> {result, Reply};
+ _ -> {result, Result}
end;
Error ->
+ %% in case we change transaction to sync_dirty...
+ %% node_call(Host, Type, delete_node, [Host, Node]),
+ %% tree_call(Host, delete_node, [Host, Node]),
Error
end.
-%% @doc <p>Delete specified node and all childs.</p>
+%% @doc <p>Delete specified node and all children.</p>
%%<p>There are several reasons why the node deletion request might fail:</p>
%%<ul>
%%<li>The requesting entity does not have sufficient privileges to delete the node.</li>
%%<li>The node is the root collection node, which cannot be deleted.</li>
%%<li>The specified node does not exist.</li>
%%</ul>
--spec delete_node(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- Owner :: jid()) -> {result, [xmlel(),...]} | {error, xmlel()}.
+-spec delete_node(host(), binary(), jid()) -> {result, pubsub_owner()} | {error, stanza_error()}.
delete_node(_Host, <<>>, _Owner) ->
- {error, ?ERRT_NOT_ALLOWED(?MYLANG, <<"No node specified">>)};
+ {error, xmpp:err_not_allowed(?T("No node specified"), ejabberd_option:language())};
delete_node(Host, Node, Owner) ->
- Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
- case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
- {result, owner} ->
- SubsByDepth = get_node_subs_by_depth(Host, Node, service_jid(Host)),
- Removed = tree_call(Host, delete_node, [Host, Node]),
- case node_call(Host, Type, delete_node, [Removed]) of
- {result, Res} -> {result, {SubsByDepth, Res}};
- Error -> Error
- end;
- _ ->
- {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
- end
- end,
- Reply = [],
+ Action =
+ fun(#pubsub_node{type = Type, id = Nidx}) ->
+ case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
+ {result, owner} ->
+ case get_node_subs_by_depth(Host, Node, service_jid(Host)) of
+ {result, SubsByDepth} ->
+ case tree_call(Host, delete_node, [Host, Node]) of
+ Removed when is_list(Removed) ->
+ case node_call(Host, Type, delete_node, [Removed]) of
+ {result, Res} -> {result, {SubsByDepth, Res}};
+ Error -> Error
+ end;
+ Error ->
+ Error
+ end;
+ Error ->
+ Error
+ end;
+ {result, _} ->
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_forbidden(?T("Owner privileges required"), Lang)};
+ Error ->
+ Error
+ end
+ end,
+ Reply = undefined,
ServerHost = serverhost(Host),
case transaction(Host, Node, Action, transaction) of
{result, {_, {SubsByDepth, {Result, broadcast, Removed}}}} ->
@@ -1879,6 +1635,7 @@ delete_node(Host, Node, Owner) ->
RNidx = RNode#pubsub_node.id,
RType = RNode#pubsub_node.type,
ROptions = RNode#pubsub_node.options,
+ unset_cached_item(RH, RNidx),
broadcast_removed_node(RH, RN, RNidx, RType, ROptions, SubsByDepth),
ejabberd_hooks:run(pubsub_delete_node,
ServerHost,
@@ -1893,6 +1650,7 @@ delete_node(Host, Node, Owner) ->
lists:foreach(fun ({RNode, _RSubs}) ->
{RH, RN} = RNode#pubsub_node.nodeid,
RNidx = RNode#pubsub_node.id,
+ unset_cached_item(RH, RNidx),
ejabberd_hooks:run(pubsub_delete_node,
ServerHost,
[ServerHost, RH, RN, RNidx])
@@ -1904,6 +1662,7 @@ delete_node(Host, Node, Owner) ->
end;
{result, {TNode, {_, Result}}} ->
Nidx = TNode#pubsub_node.id,
+ unset_cached_item(Host, Nidx),
ejabberd_hooks:run(pubsub_delete_node, ServerHost,
[ServerHost, Host, Node, Nidx]),
case Result of
@@ -1929,16 +1688,15 @@ delete_node(Host, Node, Owner) ->
%%<li>The node does not support subscriptions.</li>
%%<li>The node does not exist.</li>
%%</ul>
--spec subscribe_node(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- From :: jid(), JID :: binary(), Configuration :: [xmlel()]) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
+-spec subscribe_node(host(), binary(), jid(), jid(), [{binary(), [binary()]}]) ->
+ {result, pubsub()} | {error, stanza_error()}.
subscribe_node(Host, Node, From, JID, Configuration) ->
SubModule = subscription_plugin(Host),
SubOpts = case SubModule:parse_options_xform(Configuration) of
{result, GoodSubOpts} -> GoodSubOpts;
_ -> invalid
end,
- Subscriber = string_to_ljid(JID),
+ Subscriber = jid:tolower(JID),
Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx, owners = O}) ->
Features = plugin_features(Host, Type),
SubscribeFeature = lists:member(<<"subscribe">>, Features),
@@ -1964,24 +1722,24 @@ subscribe_node(Host, Node, From, JID, Configuration) ->
true
end,
if not SubscribeFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"subscribe">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('subscribe'))};
not SubscribeConfig ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"subscribe">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('subscribe'))};
HasOptions andalso not OptionsFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"subscription-options">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('subscription-options'))};
SubOpts == invalid ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"invalid-options">>)};
+ {error, extended_error(xmpp:err_bad_request(),
+ err_invalid_options())};
not CanSubscribe ->
%% fallback to closest XEP compatible result, assume we are not allowed to subscribe
- {error,
- extended_error(?ERR_NOT_ALLOWED, <<"closed-node">>)};
+ {error, extended_error(xmpp:err_not_allowed(),
+ err_closed_node())};
true ->
Owners = node_owners_call(Host, Type, Nidx, O),
- {PS, RG} = get_presence_and_roster_permissions(Host, Subscriber,
+ {PS, RG} = get_presence_and_roster_permissions(Host, JID,
Owners, AccessModel, AllowedGroups),
node_call(Host, Type, subscribe_node,
[Nidx, From, Subscriber, AccessModel,
@@ -1989,19 +1747,13 @@ subscribe_node(Host, Node, From, JID, Configuration) ->
end
end,
Reply = fun (Subscription) ->
- SubAttrs = case Subscription of
- {subscribed, SubId} ->
- [{<<"subscription">>, subscription_to_string(subscribed)},
- {<<"subid">>, SubId}, {<<"node">>, Node}];
- Other ->
- [{<<"subscription">>, subscription_to_string(Other)},
- {<<"node">>, Node}]
- end,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"subscription">>,
- attrs = [{<<"jid">>, jid:to_string(Subscriber)}
- | SubAttrs]}]}]
+ Sub = case Subscription of
+ {subscribed, SubId} ->
+ #ps_subscription{jid = JID, type = subscribed, subid = SubId};
+ Other ->
+ #ps_subscription{jid = JID, type = Other}
+ end,
+ #pubsub{subscription = Sub#ps_subscription{node = Node}}
end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {TNode, {Result, subscribed, SubId, send_last}}} ->
@@ -2021,10 +1773,10 @@ subscribe_node(Host, Node, From, JID, Configuration) ->
{result, {_TNode, {Result, subscribed, _SubId}}} ->
{result, Result};
{result, {TNode, {default, pending, _SubId}}} ->
- send_authorization_request(TNode, Subscriber),
+ send_authorization_request(TNode, JID),
{result, Reply(pending)};
{result, {TNode, {Result, pending}}} ->
- send_authorization_request(TNode, Subscriber),
+ send_authorization_request(TNode, JID),
{result, Result};
{result, {_, Result}} ->
{result, Result};
@@ -2040,14 +1792,10 @@ subscribe_node(Host, Node, From, JID, Configuration) ->
%%<li>The node does not exist.</li>
%%<li>The request specifies a subscription ID that is not valid or current.</li>
%%</ul>
--spec unsubscribe_node(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- From :: jid(), JID :: binary() | ljid(),
- SubId :: mod_pubsub:subId()) ->
- {result, []} | {error, xmlel()}.
-
-unsubscribe_node(Host, Node, From, JID, SubId) when is_binary(JID) ->
- unsubscribe_node(Host, Node, From, string_to_ljid(JID), SubId);
-unsubscribe_node(Host, Node, From, Subscriber, SubId) ->
+-spec unsubscribe_node(host(), binary(), jid(), jid(), binary()) ->
+ {result, undefined} | {error, stanza_error()}.
+unsubscribe_node(Host, Node, From, JID, SubId) ->
+ Subscriber = jid:tolower(JID),
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
node_call(Host, Type, unsubscribe_node, [Nidx, From, Subscriber, SubId])
end,
@@ -2055,9 +1803,8 @@ unsubscribe_node(Host, Node, From, Subscriber, SubId) ->
{result, {_, default}} ->
ServerHost = serverhost(Host),
ejabberd_hooks:run(pubsub_unsubscribe_node, ServerHost,
- [ServerHost, Host, Node, Subscriber, SubId]),
- {result, []};
- % {result, {_, Result}} -> {result, Result};
+ [ServerHost, Host, Node, Subscriber, SubId]),
+ {result, undefined};
Error -> Error
end.
@@ -2072,12 +1819,8 @@ unsubscribe_node(Host, Node, From, Subscriber, SubId) ->
%%<li>The item contains more than one payload element or the namespace of the root payload element does not match the configured namespace for the node.</li>
%%<li>The request does not match the node configuration.</li>
%%</ul>
--spec publish_item(Host :: mod_pubsub:host(), ServerHost :: binary(),
- Node :: mod_pubsub:nodeId(), Publisher :: jid(),
- ItemId :: <<>> | mod_pubsub:itemId(),
- Payload :: mod_pubsub:payload()) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
-
+-spec publish_item(host(), binary(), binary(), jid(), binary(),
+ [xmlel()]) -> {result, pubsub()} | {error, stanza_error()}.
publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload) ->
publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, [], all).
publish_item(Host, ServerHost, Node, Publisher, <<>>, Payload, PubOpts, Access) ->
@@ -2093,35 +1836,32 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access
PayloadCount = payload_xmlelements(Payload),
PayloadSize = byte_size(term_to_binary(Payload)) - 2,
PayloadMaxSize = get_option(Options, max_payload_size),
+ PreconditionsMet = preconditions_met(PubOpts, Options),
if not PublishFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"publish">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported(publish))};
+ not PreconditionsMet ->
+ {error, extended_error(xmpp:err_conflict(),
+ err_precondition_not_met())};
PayloadSize > PayloadMaxSize ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"payload-too-big">>)};
- (PayloadCount == 0) and (Payload == []) ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"payload-required">>)};
- (PayloadCount > 1) or (PayloadCount == 0) ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"invalid-payload">>)};
- (DeliverPayloads == false) and (PersistItems == false) and
- (PayloadSize > 0) ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"item-forbidden">>)};
- ((DeliverPayloads == true) or (PersistItems == true)) and (PayloadSize == 0) ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"item-required">>)};
+ {error, extended_error(xmpp:err_not_acceptable(),
+ err_payload_too_big())};
+ (DeliverPayloads or PersistItems) and (PayloadCount == 0) ->
+ {error, extended_error(xmpp:err_bad_request(),
+ err_item_required())};
+ (DeliverPayloads or PersistItems) and (PayloadCount > 1) ->
+ {error, extended_error(xmpp:err_bad_request(),
+ err_invalid_payload())};
+ (not (DeliverPayloads or PersistItems)) and (PayloadCount > 0) ->
+ {error, extended_error(xmpp:err_bad_request(),
+ err_item_forbidden())};
true ->
node_call(Host, Type, publish_item,
[Nidx, Publisher, PublishModel, MaxItems, ItemId, Payload, PubOpts])
end
end,
- Reply = [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"publish">>, attrs = nodeAttr(Node),
- children = [#xmlel{name = <<"item">>,
- attrs = itemAttr(ItemId)}]}]}],
+ Reply = #pubsub{publish = #ps_publish{node = Node,
+ items = [#ps_item{id = ItemId}]}},
case transaction(Host, Node, Action, sync_dirty) of
{result, {TNode, {Result, Broadcast, Removed}}} ->
Nidx = TNode#pubsub_node.id,
@@ -2131,8 +1871,6 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access
broadcast -> Payload;
PluginPayload -> PluginPayload
end,
- ejabberd_hooks:run(pubsub_publish_item, ServerHost,
- [ServerHost, Node, Publisher, service_jid(Host), ItemId, BrPayload]),
set_cached_item(Host, Nidx, ItemId, Publisher, BrPayload),
case get_option(Options, deliver_notifications) of
true ->
@@ -2141,6 +1879,8 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access
false ->
ok
end,
+ ejabberd_hooks:run(pubsub_publish_item, ServerHost,
+ [ServerHost, Node, Publisher, service_jid(Host), ItemId, BrPayload]),
case Result of
default -> {result, Reply};
_ -> {result, Result}
@@ -2163,29 +1903,20 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access
{result, Reply};
{result, {_, Result}} ->
{result, Result};
- {error, _} = Error ->
- case is_item_not_found(Error) of
+ {error, #stanza_error{reason = 'item-not-found'}} ->
+ Type = select_type(ServerHost, Host, Node),
+ case lists:member(<<"auto-create">>, plugin_features(Host, Type)) of
true ->
- Type = select_type(ServerHost, Host, Node),
- case lists:member(<<"auto-create">>, plugin_features(Host, Type)) of
- true ->
- case create_node(Host, ServerHost, Node, Publisher, Type, Access, []) of
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"create">>,
- attrs = [{<<"node">>, NewNode}]}]}]} ->
+ case create_node(Host, ServerHost, Node, Publisher, Type, Access, PubOpts) of
+ {result, #pubsub{create = NewNode}} ->
publish_item(Host, ServerHost, NewNode, Publisher, ItemId,
- Payload, PubOpts, Access);
- _ ->
- {error, ?ERR_ITEM_NOT_FOUND}
- end;
- false ->
- Txt = <<"Automatic node creation is not enabled">>,
- {error, ?ERRT_ITEM_NOT_FOUND(?MYLANG, Txt)}
+ Payload, PubOpts, Access);
+ _ ->
+ {error, xmpp:err_item_not_found()}
end;
false ->
- Error
+ Txt = ?T("Automatic node creation is not enabled"),
+ {error, xmpp:err_item_not_found(Txt, ejabberd_option:language())}
end;
Error ->
Error
@@ -2202,34 +1933,32 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access
%%<li>The node does not support persistent items.</li>
%%<li>The service does not support the deletion of items.</li>
%%</ul>
--spec delete_item(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- Publisher :: jid(), ItemId :: mod_pubsub:itemId()) ->
- {result, []} | {error, xmlel()}.
-
+-spec delete_item(host(), binary(), jid(), binary()) -> {result, undefined} |
+ {error, stanza_error()}.
delete_item(Host, Node, Publisher, ItemId) ->
delete_item(Host, Node, Publisher, ItemId, false).
delete_item(_, <<>>, _, _, _) ->
- {error, extended_error(?ERR_BAD_REQUEST, <<"node-required">>)};
+ {error, extended_error(xmpp:err_bad_request(), err_nodeid_required())};
delete_item(Host, Node, Publisher, ItemId, ForceNotify) ->
Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx}) ->
Features = plugin_features(Host, Type),
PersistentFeature = lists:member(<<"persistent-items">>, Features),
DeleteFeature = lists:member(<<"delete-items">>, Features),
PublishModel = get_option(Options, publish_model),
- if %%-> iq_pubsub just does that matchs
+ if %%-> iq_pubsub just does that matches
%% %% Request does not specify an item
%% {error, extended_error(?ERR_BAD_REQUEST, "item-required")};
not PersistentFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"persistent-items">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('persistent-items'))};
not DeleteFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"delete-items">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('delete-items'))};
true ->
node_call(Host, Type, delete_item, [Nidx, Publisher, PublishModel, ItemId])
end
end,
- Reply = [],
+ Reply = undefined,
case transaction(Host, Node, Action, sync_dirty) of
{result, {TNode, {Result, broadcast}}} ->
Nidx = TNode#pubsub_node.id,
@@ -2260,10 +1989,8 @@ delete_item(Host, Node, Publisher, ItemId, ForceNotify) ->
%%<li>The node is not configured to persist items.</li>
%%<li>The specified node does not exist.</li>
%%</ul>
--spec purge_node(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- Owner :: jid()) ->
- {result, []} | {error, xmlel()}.
-
+-spec purge_node(mod_pubsub:host(), binary(), jid()) -> {result, undefined} |
+ {error, stanza_error()}.
purge_node(Host, Node, Owner) ->
Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx}) ->
Features = plugin_features(Host, Type),
@@ -2271,19 +1998,19 @@ purge_node(Host, Node, Owner) ->
PersistentFeature = lists:member(<<"persistent-items">>, Features),
PersistentConfig = get_option(Options, persist_items),
if not PurgeFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"purge-nodes">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('purge-nodes'))};
not PersistentFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"persistent-items">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('persistent-items'))};
not PersistentConfig ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"persistent-items">>)};
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('persistent-items'))};
true -> node_call(Host, Type, purge_node, [Nidx, Owner])
end
end,
- Reply = [],
- case transaction(Host, Node, Action, sync_dirty) of
+ Reply = undefined,
+ case transaction(Host, Node, Action, transaction) of
{result, {TNode, {Result, broadcast}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
@@ -2305,84 +2032,79 @@ purge_node(Host, Node, Owner) ->
%% @doc <p>Return the items of a given node.</p>
%% <p>The number of items to return is limited by MaxItems.</p>
%% <p>The permission are not checked in this function.</p>
-%% @todo We probably need to check that the user doing the query has the right
-%% to read the items.
--spec get_items(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- From :: jid(), SubId :: mod_pubsub:subId(),
- SMaxItems :: binary(), ItemIds :: [mod_pubsub:itemId()],
- Rsm :: none | rsm_in()) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
-
-get_items(Host, Node, From, SubId, SMaxItems, ItemIds, RSM) ->
- MaxItems = if SMaxItems == <<>> ->
- case get_max_items_node(Host) of
- undefined -> ?MAXITEMS;
- Max -> Max
- end;
- true ->
- case catch jlib:binary_to_integer(SMaxItems) of
- {'EXIT', _} ->
- Txt = <<"Value of 'max_items' should be integer">>,
- {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)};
- Val -> Val
- end
- end,
- case MaxItems of
- {error, Error} ->
- {error, Error};
- _ ->
- Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx, owners = O}) ->
- Features = plugin_features(Host, Type),
- RetreiveFeature = lists:member(<<"retrieve-items">>, Features),
- PersistentFeature = lists:member(<<"persistent-items">>, Features),
- AccessModel = get_option(Options, access_model),
- AllowedGroups = get_option(Options, roster_groups_allowed, []),
- if not RetreiveFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"retrieve-items">>)};
- not PersistentFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"persistent-items">>)};
- true ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- {PS, RG} = get_presence_and_roster_permissions(Host, From, Owners,
- AccessModel, AllowedGroups),
- node_call(Host, Type, get_items,
- [Nidx, From, AccessModel, PS, RG, SubId, RSM])
- end
- end,
- case transaction(Host, Node, Action, sync_dirty) of
- {result, {_, {Items, RsmOut}}} ->
- SendItems = case ItemIds of
- [] ->
- Items;
- _ ->
- lists:filter(fun (#pubsub_item{itemid = {ItemId, _}}) ->
- lists:member(ItemId, ItemIds)
- end,
- Items)
- end,
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children =
- [#xmlel{name = <<"items">>, attrs = nodeAttr(Node),
- children = itemsEls(lists:sublist(SendItems, MaxItems))}
- | jlib:rsm_encode(RsmOut)]}]};
- Error ->
- Error
- end
+-spec get_items(host(), binary(), jid(), binary(),
+ undefined | non_neg_integer(), [binary()], undefined | rsm_set()) ->
+ {result, pubsub()} | {error, stanza_error()}.
+get_items(Host, Node, From, SubId, MaxItems, ItemIds, undefined)
+ when MaxItems =/= undefined ->
+ get_items(Host, Node, From, SubId, MaxItems, ItemIds,
+ #rsm_set{max = MaxItems, before = <<>>});
+get_items(Host, Node, From, SubId, _MaxItems, ItemIds, RSM) ->
+ Action =
+ fun(#pubsub_node{options = Options, type = Type,
+ id = Nidx, owners = O}) ->
+ Features = plugin_features(Host, Type),
+ RetreiveFeature = lists:member(<<"retrieve-items">>, Features),
+ PersistentFeature = lists:member(<<"persistent-items">>, Features),
+ AccessModel = get_option(Options, access_model),
+ AllowedGroups = get_option(Options, roster_groups_allowed, []),
+ if not RetreiveFeature ->
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('retrieve-items'))};
+ not PersistentFeature ->
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('persistent-items'))};
+ true ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ {PS, RG} = get_presence_and_roster_permissions(
+ Host, From, Owners, AccessModel, AllowedGroups),
+ case ItemIds of
+ [ItemId] ->
+ NotFound = xmpp:err_item_not_found(),
+ case node_call(Host, Type, get_item,
+ [Nidx, ItemId, From, AccessModel, PS, RG, undefined])
+ of
+ {error, NotFound} -> {result, {[], undefined}};
+ Result -> Result
+ end;
+ _ ->
+ node_call(Host, Type, get_items,
+ [Nidx, From, AccessModel, PS, RG, SubId, RSM])
+ end
+ end
+ end,
+ case transaction(Host, Node, Action, sync_dirty) of
+ {result, {TNode, {Items, RsmOut}}} ->
+ SendItems = case ItemIds of
+ [] ->
+ Items;
+ _ ->
+ lists:filter(
+ fun(#pubsub_item{itemid = {ItemId, _}}) ->
+ lists:member(ItemId, ItemIds)
+ end, Items)
+ end,
+ Options = TNode#pubsub_node.options,
+ {result, #pubsub{items = items_els(Node, Options, SendItems),
+ rsm = RsmOut}};
+ {result, {TNode, Item}} ->
+ Options = TNode#pubsub_node.options,
+ {result, #pubsub{items = items_els(Node, Options, [Item])}};
+ Error ->
+ Error
end.
+%% Seems like this function broken
get_items(Host, Node) ->
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
- node_call(Host, Type, get_items, [Nidx, service_jid(Host), none])
+ node_call(Host, Type, get_items, [Nidx, service_jid(Host), undefined])
end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, {Items, _}}} -> Items;
Error -> Error
end.
+%% This function is broken too?
get_item(Host, Node, ItemId) ->
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
node_call(Host, Type, get_item, [Nidx, ItemId])
@@ -2392,310 +2114,265 @@ get_item(Host, Node, ItemId) ->
Error -> Error
end.
+-spec get_allowed_items_call(host(), nodeIdx(), jid(),
+ binary(), nodeOptions(), [ljid()]) -> {result, [#pubsub_item{}]} |
+ {error, stanza_error()}.
get_allowed_items_call(Host, Nidx, From, Type, Options, Owners) ->
- case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners, none) of
+ case get_allowed_items_call(Host, Nidx, From, Type, Options, Owners, undefined) of
{result, {Items, _RSM}} -> {result, Items};
Error -> Error
end.
+
+-spec get_allowed_items_call(host(), nodeIdx(), jid(),
+ binary(), nodeOptions(), [ljid()],
+ undefined | rsm_set()) ->
+ {result, {[#pubsub_item{}], undefined | rsm_set()}} |
+ {error, stanza_error()}.
get_allowed_items_call(Host, Nidx, From, Type, Options, Owners, RSM) ->
AccessModel = get_option(Options, access_model),
AllowedGroups = get_option(Options, roster_groups_allowed, []),
{PS, RG} = get_presence_and_roster_permissions(Host, From, Owners, AccessModel, AllowedGroups),
node_call(Host, Type, get_items, [Nidx, From, AccessModel, PS, RG, undefined, RSM]).
-get_last_item(Host, Type, Nidx, LJID) ->
+-spec get_last_items(host(), binary(), nodeIdx(), ljid(), last | integer()) -> [#pubsub_item{}].
+get_last_items(Host, Type, Nidx, LJID, last) ->
+ % hack to handle section 6.1.7 of XEP-0060
+ get_last_items(Host, Type, Nidx, LJID, 1);
+get_last_items(Host, Type, Nidx, LJID, 1) ->
case get_cached_item(Host, Nidx) of
- undefined -> get_last_item(Host, Type, Nidx, LJID, gen_mod:db_type(serverhost(Host), ?MODULE));
- LastItem -> LastItem
- end.
-get_last_item(Host, Type, Nidx, LJID, mnesia) ->
- case node_action(Host, Type, get_items, [Nidx, LJID, none]) of
- {result, {[LastItem|_], _}} -> LastItem;
- _ -> undefined
- end;
-get_last_item(Host, Type, Nidx, LJID, sql) ->
- case node_action(Host, Type, get_last_items, [Nidx, LJID, 1]) of
- {result, [LastItem]} -> LastItem;
- _ -> undefined
- end;
-get_last_item(_Host, _Type, _Nidx, _LJID, _) ->
- undefined.
-
-get_last_items(Host, Type, Nidx, LJID, Number) ->
- get_last_items(Host, Type, Nidx, LJID, Number, gen_mod:db_type(serverhost(Host), ?MODULE)).
-get_last_items(Host, Type, Nidx, LJID, Number, mnesia) ->
- case node_action(Host, Type, get_items, [Nidx, LJID, none]) of
- {result, {Items, _}} -> lists:sublist(Items, Number);
- _ -> []
+ undefined ->
+ case node_action(Host, Type, get_last_items, [Nidx, LJID, 1]) of
+ {result, Items} -> Items;
+ _ -> []
+ end;
+ LastItem ->
+ [LastItem]
end;
-get_last_items(Host, Type, Nidx, LJID, Number, sql) ->
- case node_action(Host, Type, get_last_items, [Nidx, LJID, Number]) of
+get_last_items(Host, Type, Nidx, LJID, Count) when Count > 1 ->
+ case node_action(Host, Type, get_last_items, [Nidx, LJID, Count]) of
{result, Items} -> Items;
_ -> []
end;
-get_last_items(_Host, _Type, _Nidx, _LJID, _Number, _) ->
+get_last_items(_Host, _Type, _Nidx, _LJID, _Count) ->
[].
-%% @doc <p>Resend the items of a node to the user.</p>
-%% @todo use cache-last-item feature
-send_items(Host, Node, Nidx, Type, Options, LJID, last) ->
- case get_last_item(Host, Type, Nidx, LJID) of
+-spec get_only_item(host(), binary(), nodeIdx(), ljid()) -> [#pubsub_item{}].
+get_only_item(Host, Type, Nidx, LJID) ->
+ case get_cached_item(Host, Nidx) of
undefined ->
- ok;
+ case node_action(Host, Type, get_only_item, [Nidx, LJID]) of
+ {result, Items} when length(Items) < 2 ->
+ Items;
+ {result, Items} ->
+ [hd(lists:keysort(#pubsub_item.modification, Items))];
+ _ -> []
+ end;
LastItem ->
- Stanza = items_event_stanza(Node, Options, [LastItem]),
- dispatch_items(Host, LJID, Node, Stanza)
- end;
-send_items(Host, Node, Nidx, Type, Options, LJID, Number) when Number > 0 ->
- Stanza = items_event_stanza(Node, Options, get_last_items(Host, Type, Nidx, Number, LJID)),
- dispatch_items(Host, LJID, Node, Stanza);
-send_items(Host, Node, _Nidx, _Type, Options, LJID, _) ->
- Stanza = items_event_stanza(Node, Options, []),
- dispatch_items(Host, LJID, Node, Stanza).
-
-dispatch_items({FromU, FromS, FromR} = From, {ToU, ToS, ToR} = To,
- Node, Stanza) ->
- C2SPid = case ejabberd_sm:get_session_pid(ToU, ToS, ToR) of
- ToPid when is_pid(ToPid) -> ToPid;
- _ ->
- R = user_resource(FromU, FromS, FromR),
- case ejabberd_sm:get_session_pid(FromU, FromS, R) of
- FromPid when is_pid(FromPid) -> FromPid;
- _ -> undefined
- end
- end,
- if C2SPid == undefined -> ok;
- true ->
- ejabberd_c2s:send_filtered(C2SPid,
- {pep_message, <<Node/binary, "+notify">>},
- service_jid(From), jid:make(To),
- Stanza)
- end;
-dispatch_items(From, To, _Node, Stanza) ->
- ejabberd_router:route(service_jid(From), jid:make(To), Stanza).
+ [LastItem]
+ end.
%% @doc <p>Return the list of affiliations as an XMPP response.</p>
--spec get_affiliations(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- JID :: jid(), Plugins :: [binary()]) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
-
+-spec get_affiliations(host(), binary(), jid(), [binary()]) ->
+ {result, pubsub()} | {error, stanza_error()}.
get_affiliations(Host, Node, JID, Plugins) when is_list(Plugins) ->
- Result = lists:foldl( fun (Type, {Status, Acc}) ->
- Features = plugin_features(Host, Type),
- RetrieveFeature = lists:member(<<"retrieve-affiliations">>, Features),
- if not RetrieveFeature ->
- {{error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
- unsupported, <<"retrieve-affiliations">>)},
- Acc};
- true ->
- {result, Affs} = node_action(Host, Type,
- get_entity_affiliations,
- [Host, JID]),
- {Status, [Affs | Acc]}
- end
- end,
- {ok, []}, Plugins),
+ Result =
+ lists:foldl(
+ fun(Type, {Status, Acc}) ->
+ Features = plugin_features(Host, Type),
+ RetrieveFeature = lists:member(<<"retrieve-affiliations">>, Features),
+ if not RetrieveFeature ->
+ {{error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('retrieve-affiliations'))},
+ Acc};
+ true ->
+ case node_action(Host, Type,
+ get_entity_affiliations,
+ [Host, JID]) of
+ {result, Affs} ->
+ {Status, [Affs | Acc]};
+ {error, _} = Err ->
+ {Err, Acc}
+ end
+ end
+ end, {ok, []}, Plugins),
case Result of
{ok, Affs} ->
- Entities = lists:flatmap(fun
- ({_, none}) ->
- [];
- ({#pubsub_node{nodeid = {_, NodeId}}, Aff}) ->
- if (Node == <<>>) or (Node == NodeId) ->
- [#xmlel{name = <<"affiliation">>,
- attrs = [{<<"affiliation">>, affiliation_to_string(Aff)}
- | nodeAttr(NodeId)]}];
- true ->
- []
- end;
- (_) ->
- []
- end,
- lists:usort(lists:flatten(Affs))),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"affiliations">>, attrs = [],
- children = Entities}]}]};
+ Entities = lists:flatmap(
+ fun({_, none}) ->
+ [];
+ ({#pubsub_node{nodeid = {_, NodeId}}, Aff}) ->
+ if (Node == <<>>) or (Node == NodeId) ->
+ [#ps_affiliation{node = NodeId,
+ type = Aff}];
+ true ->
+ []
+ end;
+ (_) ->
+ []
+ end, lists:usort(lists:flatten(Affs))),
+ {result, #pubsub{affiliations = {<<>>, Entities}}};
{Error, _} ->
Error
end.
--spec get_affiliations(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- JID :: jid()) ->
- {result, [xmlel(),...]} | {error, xmlel()}.
-
+-spec get_affiliations(host(), binary(), jid()) ->
+ {result, pubsub_owner()} | {error, stanza_error()}.
get_affiliations(Host, Node, JID) ->
- Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
- Features = plugin_features(Host, Type),
- RetrieveFeature = lists:member(<<"modify-affiliations">>, Features),
- {result, Affiliation} = node_call(Host, Type, get_affiliation, [Nidx, JID]),
- if not RetrieveFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"modify-affiliations">>)};
- Affiliation /= owner ->
- {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)};
- true ->
- node_call(Host, Type, get_node_affiliations, [Nidx])
- end
- end,
+ Action =
+ fun(#pubsub_node{type = Type, id = Nidx}) ->
+ Features = plugin_features(Host, Type),
+ RetrieveFeature = lists:member(<<"modify-affiliations">>, Features),
+ {result, Affiliation} = node_call(Host, Type, get_affiliation, [Nidx, JID]),
+ if not RetrieveFeature ->
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('modify-affiliations'))};
+ Affiliation /= owner ->
+ {error, xmpp:err_forbidden(?T("Owner privileges required"), ejabberd_option:language())};
+ true ->
+ node_call(Host, Type, get_node_affiliations, [Nidx])
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, []}} ->
- {error, ?ERR_ITEM_NOT_FOUND};
+ {error, xmpp:err_item_not_found()};
{result, {_, Affs}} ->
- Entities = lists:flatmap(fun
- ({_, none}) ->
- [];
- ({AJID, Aff}) ->
- [#xmlel{name = <<"affiliation">>,
- attrs = [{<<"jid">>, jid:to_string(AJID)},
- {<<"affiliation">>, affiliation_to_string(Aff)}]}]
- end,
- Affs),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB_OWNER}],
- children = [#xmlel{name = <<"affiliations">>,
- attrs = nodeAttr(Node), children = Entities}]}]};
+ Entities = lists:flatmap(
+ fun({_, none}) ->
+ [];
+ ({AJID, Aff}) ->
+ [#ps_affiliation{jid = AJID, type = Aff}]
+ end, Affs),
+ {result, #pubsub_owner{affiliations = {Node, Entities}}};
Error ->
Error
end.
--spec set_affiliations(Host :: mod_pubsub:host(), Node :: mod_pubsub:nodeId(),
- From :: jid(), EntitiesEls :: [xmlel()]) ->
- {result, []} | {error, xmlel()}.
-
-set_affiliations(Host, Node, From, EntitiesEls) ->
+-spec set_affiliations(host(), binary(), jid(), [ps_affiliation()]) ->
+ {result, undefined} | {error, stanza_error()}.
+set_affiliations(Host, Node, From, Affs) ->
Owner = jid:tolower(jid:remove_resource(From)),
- Entities = lists:foldl(fun
- (_, error) ->
- error;
- (El, Acc) ->
- case El of
- #xmlel{name = <<"affiliation">>, attrs = Attrs} ->
- JID = jid:from_string(fxml:get_attr_s(<<"jid">>, Attrs)),
- Affiliation = string_to_affiliation(fxml:get_attr_s(<<"affiliation">>, Attrs)),
- if (JID == error) or (Affiliation == false) -> error;
- true -> [{jid:tolower(JID), Affiliation} | Acc]
- end
- end
- end,
- [], EntitiesEls),
- case Entities of
- error ->
- {error, ?ERR_BAD_REQUEST};
- _ ->
- Action = fun (#pubsub_node{type = Type, id = Nidx, owners = O} = N) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case lists:member(Owner, Owners) of
- true ->
- OwnerJID = jid:make(Owner),
- FilteredEntities = case Owners of
- [Owner] -> [E || E <- Entities, element(1, E) =/= OwnerJID];
- _ -> Entities
+ Action =
+ fun(#pubsub_node{type = Type, id = Nidx, owners = O} = N) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case lists:member(Owner, Owners) of
+ true ->
+ OwnerJID = jid:make(Owner),
+ FilteredAffs =
+ case Owners of
+ [Owner] ->
+ [Aff || Aff <- Affs,
+ Aff#ps_affiliation.jid /= OwnerJID];
+ _ ->
+ Affs
end,
- lists:foreach(fun ({JID, Affiliation}) ->
- node_call(Host, Type, set_affiliation, [Nidx, JID, Affiliation]),
- case Affiliation of
- owner ->
- NewOwner = jid:tolower(jid:remove_resource(JID)),
- NewOwners = [NewOwner | Owners],
- tree_call(Host,
+ lists:foreach(
+ fun(#ps_affiliation{jid = JID, type = Affiliation}) ->
+ node_call(Host, Type, set_affiliation, [Nidx, JID, Affiliation]),
+ case Affiliation of
+ owner ->
+ NewOwner = jid:tolower(jid:remove_resource(JID)),
+ NewOwners = [NewOwner | Owners],
+ tree_call(Host,
set_node,
[N#pubsub_node{owners = NewOwners}]);
- none ->
- OldOwner = jid:tolower(jid:remove_resource(JID)),
- case lists:member(OldOwner, Owners) of
- true ->
- NewOwners = Owners -- [OldOwner],
- tree_call(Host,
+ none ->
+ OldOwner = jid:tolower(jid:remove_resource(JID)),
+ case lists:member(OldOwner, Owners) of
+ true ->
+ NewOwners = Owners -- [OldOwner],
+ tree_call(Host,
set_node,
[N#pubsub_node{owners = NewOwners}]);
- _ ->
- ok
- end;
- _ ->
- ok
- end
- end,
- FilteredEntities),
- {result, []};
- _ ->
- {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
- end
- end,
- case transaction(Host, Node, Action, sync_dirty) of
- {result, {_, Result}} -> {result, Result};
- Other -> Other
- end
+ _ ->
+ ok
+ end;
+ _ ->
+ ok
+ end
+ end, FilteredAffs),
+ {result, undefined};
+ _ ->
+ {error, xmpp:err_forbidden(
+ ?T("Owner privileges required"), ejabberd_option:language())}
+ end
+ end,
+ case transaction(Host, Node, Action, sync_dirty) of
+ {result, {_, Result}} -> {result, Result};
+ Other -> Other
end.
+-spec get_options(binary(), binary(), jid(), binary(), binary()) ->
+ {result, xdata()} | {error, stanza_error()}.
get_options(Host, Node, JID, SubId, Lang) ->
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
case lists:member(<<"subscription-options">>, plugin_features(Host, Type)) of
true ->
get_options_helper(Host, JID, Lang, Node, Nidx, SubId, Type);
false ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"subscription-options">>)}
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('subscription-options'))}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
- {result, {_Node, XForm}} -> {result, [XForm]};
+ {result, {_Node, XForm}} -> {result, XForm};
Error -> Error
end.
+-spec get_options_helper(binary(), jid(), binary(), binary(), _, binary(),
+ binary()) -> {result, pubsub()} | {error, stanza_error()}.
get_options_helper(Host, JID, Lang, Node, Nidx, SubId, Type) ->
- Subscriber = string_to_ljid(JID),
- {result, Subs} = node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]),
- SubIds = [Id || {Sub, Id} <- Subs, Sub == subscribed],
- case {SubId, SubIds} of
- {_, []} ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"not-subscribed">>)};
- {<<>>, [SID]} ->
- read_sub(Host, Node, Nidx, Subscriber, SID, Lang);
- {<<>>, _} ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"subid-required">>)};
- {_, _} ->
- ValidSubId = lists:member(SubId, SubIds),
- if ValidSubId ->
- read_sub(Host, Node, Nidx, Subscriber, SubId, Lang);
- true ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"invalid-subid">>)}
- end
+ Subscriber = jid:tolower(JID),
+ case node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]) of
+ {result, Subs} ->
+ SubIds = [Id || {Sub, Id} <- Subs, Sub == subscribed],
+ case {SubId, SubIds} of
+ {_, []} ->
+ {error, extended_error(xmpp:err_not_acceptable(),
+ err_not_subscribed())};
+ {<<>>, [SID]} ->
+ read_sub(Host, Node, Nidx, Subscriber, SID, Lang);
+ {<<>>, _} ->
+ {error, extended_error(xmpp:err_not_acceptable(),
+ err_subid_required())};
+ {_, _} ->
+ ValidSubId = lists:member(SubId, SubIds),
+ if ValidSubId ->
+ read_sub(Host, Node, Nidx, Subscriber, SubId, Lang);
+ true ->
+ {error, extended_error(xmpp:err_not_acceptable(),
+ err_invalid_subid())}
+ end
+ end;
+ {error, _} = Error ->
+ Error
end.
+-spec read_sub(binary(), binary(), nodeIdx(), ljid(), binary(), binary()) -> {result, pubsub()}.
read_sub(Host, Node, Nidx, Subscriber, SubId, Lang) ->
SubModule = subscription_plugin(Host),
- Children = case SubModule:get_subscription(Subscriber, Nidx, SubId) of
- {error, notfound} ->
- [];
- {result, #pubsub_subscription{options = Options}} ->
- {result, XdataEl} = SubModule:get_options_xform(Lang, Options),
- [XdataEl]
- end,
- OptionsEl = #xmlel{name = <<"options">>,
- attrs = [{<<"jid">>, jid:to_string(Subscriber)},
- {<<"subid">>, SubId}
- | nodeAttr(Node)],
- children = Children},
- PubsubEl = #xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [OptionsEl]},
- {result, PubsubEl}.
-
+ XData = case SubModule:get_subscription(Subscriber, Nidx, SubId) of
+ {error, notfound} ->
+ undefined;
+ {result, #pubsub_subscription{options = Options}} ->
+ {result, X} = SubModule:get_options_xform(Lang, Options),
+ X
+ end,
+ {result, #pubsub{options = #ps_options{jid = jid:make(Subscriber),
+ subid = SubId,
+ node = Node,
+ xdata = XData}}}.
+
+-spec set_options(binary(), binary(), jid(), binary(),
+ [{binary(), [binary()]}]) ->
+ {result, undefined} | {error, stanza_error()}.
set_options(Host, Node, JID, SubId, Configuration) ->
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
case lists:member(<<"subscription-options">>, plugin_features(Host, Type)) of
true ->
set_options_helper(Host, Configuration, JID, Nidx, SubId, Type);
false ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"subscription-options">>)}
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('subscription-options'))}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -2703,271 +2380,235 @@ set_options(Host, Node, JID, SubId, Configuration) ->
Error -> Error
end.
+-spec set_options_helper(binary(), [{binary(), [binary()]}], jid(),
+ nodeIdx(), binary(), binary()) ->
+ {result, undefined} | {error, stanza_error()}.
set_options_helper(Host, Configuration, JID, Nidx, SubId, Type) ->
SubModule = subscription_plugin(Host),
SubOpts = case SubModule:parse_options_xform(Configuration) of
{result, GoodSubOpts} -> GoodSubOpts;
_ -> invalid
end,
- Subscriber = string_to_ljid(JID),
- {result, Subs} = node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]),
- SubIds = [Id || {Sub, Id} <- Subs, Sub == subscribed],
- case {SubId, SubIds} of
- {_, []} ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"not-subscribed">>)};
- {<<>>, [SID]} ->
- write_sub(Host, Nidx, Subscriber, SID, SubOpts);
- {<<>>, _} ->
- {error,
- extended_error(?ERR_NOT_ACCEPTABLE, <<"subid-required">>)};
- {_, _} ->
- write_sub(Host, Nidx, Subscriber, SubId, SubOpts)
+ Subscriber = jid:tolower(JID),
+ case node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]) of
+ {result, Subs} ->
+ SubIds = [Id || {Sub, Id} <- Subs, Sub == subscribed],
+ case {SubId, SubIds} of
+ {_, []} ->
+ {error, extended_error(xmpp:err_not_acceptable(), err_not_subscribed())};
+ {<<>>, [SID]} ->
+ write_sub(Host, Nidx, Subscriber, SID, SubOpts);
+ {<<>>, _} ->
+ {error, extended_error(xmpp:err_not_acceptable(), err_subid_required())};
+ {_, _} ->
+ write_sub(Host, Nidx, Subscriber, SubId, SubOpts)
+ end;
+ {error, _} = Err ->
+ Err
end.
+-spec write_sub(binary(), nodeIdx(), ljid(), binary(), _) -> {result, undefined} |
+ {error, stanza_error()}.
write_sub(_Host, _Nidx, _Subscriber, _SubId, invalid) ->
- {error,
- extended_error(?ERR_BAD_REQUEST, <<"invalid-options">>)};
+ {error, extended_error(xmpp:err_bad_request(), err_invalid_options())};
write_sub(_Host, _Nidx, _Subscriber, _SubId, []) ->
- {result, []};
+ {result, undefined};
write_sub(Host, Nidx, Subscriber, SubId, Options) ->
SubModule = subscription_plugin(Host),
case SubModule:set_subscription(Subscriber, Nidx, SubId, Options) of
- {result, _} -> {result, []};
- {error, _} -> {error, extended_error(?ERR_NOT_ACCEPTABLE, <<"invalid-subid">>)}
+ {result, _} -> {result, undefined};
+ {error, _} -> {error, extended_error(xmpp:err_not_acceptable(),
+ err_invalid_subid())}
end.
-%% @spec (Host, Node, JID, Plugins) -> {error, Reason} | {result, Response}
-%% Host = host()
-%% Node = pubsubNode()
-%% JID = jid()
-%% Plugins = [Plugin::string()]
-%% Reason = stanzaError()
-%% Response = [pubsubIQResponse()]
%% @doc <p>Return the list of subscriptions as an XMPP response.</p>
+-spec get_subscriptions(host(), binary(), jid(), [binary()]) ->
+ {result, pubsub()} | {error, stanza_error()}.
get_subscriptions(Host, Node, JID, Plugins) when is_list(Plugins) ->
Result = lists:foldl(fun (Type, {Status, Acc}) ->
Features = plugin_features(Host, Type),
RetrieveFeature = lists:member(<<"retrieve-subscriptions">>, Features),
if not RetrieveFeature ->
- {{error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
- unsupported, <<"retrieve-subscriptions">>)},
+ {{error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('retrieve-subscriptions'))},
Acc};
true ->
Subscriber = jid:remove_resource(JID),
- {result, Subs} = node_action(Host, Type,
- get_entity_subscriptions,
- [Host, Subscriber]),
- {Status, [Subs | Acc]}
+ case node_action(Host, Type,
+ get_entity_subscriptions,
+ [Host, Subscriber]) of
+ {result, Subs} ->
+ {Status, [Subs | Acc]};
+ {error, _} = Err ->
+ {Err, Acc}
+ end
end
- end,
- {ok, []}, Plugins),
+ end, {ok, []}, Plugins),
case Result of
{ok, Subs} ->
Entities = lists:flatmap(fun
- ({_, none}) ->
- [];
({#pubsub_node{nodeid = {_, SubsNode}}, Sub}) ->
case Node of
<<>> ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"subscription">>, subscription_to_string(Sub)}
- | nodeAttr(SubsNode)]}];
+ [#ps_subscription{jid = jid:remove_resource(JID),
+ node = SubsNode, type = Sub}];
SubsNode ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"subscription">>, subscription_to_string(Sub)}]}];
+ [#ps_subscription{jid = jid:remove_resource(JID),
+ type = Sub}];
_ ->
[]
end;
- ({_, none, _}) ->
- [];
({#pubsub_node{nodeid = {_, SubsNode}}, Sub, SubId, SubJID}) ->
case Node of
<<>> ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(SubJID)},
- {<<"subid">>, SubId},
- {<<"subscription">>, subscription_to_string(Sub)}
- | nodeAttr(SubsNode)]}];
+ [#ps_subscription{jid = SubJID,
+ subid = SubId,
+ type = Sub,
+ node = SubsNode}];
SubsNode ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(SubJID)},
- {<<"subid">>, SubId},
- {<<"subscription">>, subscription_to_string(Sub)}]}];
+ [#ps_subscription{jid = SubJID,
+ subid = SubId,
+ type = Sub}];
_ ->
[]
end;
({#pubsub_node{nodeid = {_, SubsNode}}, Sub, SubJID}) ->
case Node of
<<>> ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(SubJID)},
- {<<"subscription">>, subscription_to_string(Sub)}
- | nodeAttr(SubsNode)]}];
+ [#ps_subscription{jid = SubJID,
+ type = Sub,
+ node = SubsNode}];
SubsNode ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(SubJID)},
- {<<"subscription">>, subscription_to_string(Sub)}]}];
+ [#ps_subscription{jid = SubJID, type = Sub}];
_ ->
[]
end
end,
lists:usort(lists:flatten(Subs))),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"subscriptions">>, attrs = [],
- children = Entities}]}]};
+ {result, #pubsub{subscriptions = {<<>>, Entities}}};
{Error, _} ->
Error
end.
+-spec get_subscriptions(host(), binary(), jid()) -> {result, pubsub_owner()} |
+ {error, stanza_error()}.
get_subscriptions(Host, Node, JID) ->
- Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
- Features = plugin_features(Host, Type),
- RetrieveFeature = lists:member(<<"manage-subscriptions">>, Features),
- {result, Affiliation} = node_call(Host, Type, get_affiliation, [Nidx, JID]),
- if not RetrieveFeature ->
- {error,
- extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"manage-subscriptions">>)};
- Affiliation /= owner ->
- {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)};
- true ->
- node_call(Host, Type, get_node_subscriptions, [Nidx])
- end
- end,
+ Action = fun(#pubsub_node{type = Type, id = Nidx}) ->
+ Features = plugin_features(Host, Type),
+ RetrieveFeature = lists:member(<<"manage-subscriptions">>, Features),
+ case node_call(Host, Type, get_affiliation, [Nidx, JID]) of
+ {result, Affiliation} ->
+ if not RetrieveFeature ->
+ {error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('manage-subscriptions'))};
+ Affiliation /= owner ->
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_forbidden(?T("Owner privileges required"), Lang)};
+ true ->
+ node_call(Host, Type, get_node_subscriptions, [Nidx])
+ end;
+ Error ->
+ Error
+ end
+ end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Subs}} ->
- Entities = lists:flatmap(fun
- ({_, none}) ->
- [];
- ({_, pending, _}) ->
- [];
- ({AJID, Sub}) ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(AJID)},
- {<<"subscription">>, subscription_to_string(Sub)}]}];
+ Entities =
+ lists:flatmap(
+ fun({_, none}) ->
+ [];
+ ({_, pending, _}) ->
+ [];
+ ({AJID, Sub}) ->
+ [#ps_subscription{jid = AJID, type = Sub}];
({AJID, Sub, SubId}) ->
- [#xmlel{name = <<"subscription">>,
- attrs =
- [{<<"jid">>, jid:to_string(AJID)},
- {<<"subscription">>, subscription_to_string(Sub)},
- {<<"subid">>, SubId}]}]
- end,
- Subs),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB_OWNER}],
- children = [#xmlel{name = <<"subscriptions">>,
- attrs = nodeAttr(Node),
- children = Entities}]}]};
+ [#ps_subscription{jid = AJID, type = Sub, subid = SubId}]
+ end, Subs),
+ {result, #pubsub_owner{subscriptions = {Node, Entities}}};
Error ->
Error
end.
-get_subscriptions_for_send_last(Host, PType, mnesia, JID, LJID, BJID) ->
- {result, Subs} = node_action(Host, PType,
- get_entity_subscriptions,
- [Host, JID]),
- [{Node, Sub, SubId, SubJID}
- || {Node, Sub, SubId, SubJID} <- Subs,
- Sub =:= subscribed, (SubJID == LJID) or (SubJID == BJID),
- match_option(Node, send_last_published_item, on_sub_and_presence)];
+-spec get_subscriptions_for_send_last(host(), binary(), atom(), jid(), ljid(), ljid()) ->
+ [{#pubsub_node{}, subId(), ljid()}].
get_subscriptions_for_send_last(Host, PType, sql, JID, LJID, BJID) ->
- case catch node_action(Host, PType,
- get_entity_subscriptions_for_send_last,
- [Host, JID])
- of
+ case node_action(Host, PType,
+ get_entity_subscriptions_for_send_last,
+ [Host, JID]) of
{result, Subs} ->
- [{Node, Sub, SubId, SubJID}
- || {Node, Sub, SubId, SubJID} <- Subs,
- Sub =:= subscribed, (SubJID == LJID) or (SubJID == BJID)];
+ [{Node, SubId, SubJID}
+ || {Node, Sub, SubId, SubJID} <- Subs,
+ Sub =:= subscribed, (SubJID == LJID) or (SubJID == BJID)];
_ ->
[]
end;
-get_subscriptions_for_send_last(_Host, _PType, _, _JID, _LJID, _BJID) ->
- [].
-
-set_subscriptions(Host, Node, From, EntitiesEls) ->
- Owner = jid:tolower(jid:remove_resource(From)),
- Entities = lists:foldl(fun
- (_, error) ->
- error;
- (El, Acc) ->
- case El of
- #xmlel{name = <<"subscription">>, attrs = Attrs} ->
- JID = jid:from_string(fxml:get_attr_s(<<"jid">>, Attrs)),
- Sub = string_to_subscription(fxml:get_attr_s(<<"subscription">>, Attrs)),
- SubId = fxml:get_attr_s(<<"subid">>, Attrs),
- if (JID == error) or (Sub == false) -> error;
- true -> [{jid:tolower(JID), Sub, SubId} | Acc]
- end
- end
- end,
- [], EntitiesEls),
- case Entities of
- error ->
- {error, ?ERR_BAD_REQUEST};
+%% sql version already filter result by on_sub_and_presence
+get_subscriptions_for_send_last(Host, PType, _, JID, LJID, BJID) ->
+ case node_action(Host, PType,
+ get_entity_subscriptions,
+ [Host, JID]) of
+ {result, Subs} ->
+ [{Node, SubId, SubJID}
+ || {Node, Sub, SubId, SubJID} <- Subs,
+ Sub =:= subscribed, (SubJID == LJID) or (SubJID == BJID),
+ match_option(Node, send_last_published_item, on_sub_and_presence)];
_ ->
- Notify = fun (JID, Sub, _SubId) ->
- Stanza = #xmlel{name = <<"message">>, attrs = [],
- children =
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children =
- [#xmlel{name = <<"subscription">>,
- attrs = [{<<"jid">>, jid:to_string(JID)},
- {<<"subscription">>, subscription_to_string(Sub)}
- | nodeAttr(Node)]}]}]},
- ejabberd_router:route(service_jid(Host), jid:make(JID), Stanza)
- end,
- Action = fun (#pubsub_node{type = Type, id = Nidx, owners = O}) ->
- Owners = node_owners_call(Host, Type, Nidx, O),
- case lists:member(Owner, Owners) of
- true ->
- Result = lists:foldl(fun ({JID, Sub, SubId}, Acc) ->
- case
- node_call(Host, Type,
- set_subscriptions,
- [Nidx, JID, Sub, SubId])
- of
- {error, Err} ->
- [{error, Err} | Acc];
- _ ->
- Notify(JID, Sub, SubId),
- Acc
- end
- end,
- [], Entities),
- case Result of
- [] -> {result, []};
- [{error, E}|_] -> {error, E}
- end;
- _ ->
- {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
- end
- end,
- case transaction(Host, Node, Action, sync_dirty) of
- {result, {_, Result}} -> {result, Result};
- Other -> Other
- end
+ []
end.
--spec get_presence_and_roster_permissions(Host :: mod_pubsub:host(),
- From :: ljid(), Owners :: [ljid(),...],
- AccessModel :: mod_pubsub:accessModel(),
- AllowedGroups :: [binary()]) ->
- {PresenceSubscription::boolean(),
- RosterGroup::boolean()}.
+-spec set_subscriptions(host(), binary(), jid(), [ps_subscription()]) ->
+ {result, undefined} | {error, stanza_error()}.
+set_subscriptions(Host, Node, From, Entities) ->
+ Owner = jid:tolower(jid:remove_resource(From)),
+ Notify = fun(#ps_subscription{jid = JID, type = Sub}) ->
+ Stanza = #message{
+ from = service_jid(Host),
+ to = JID,
+ sub_els = [#ps_event{
+ subscription = #ps_subscription{
+ jid = JID,
+ type = Sub,
+ node = Node}}]},
+ ejabberd_router:route(Stanza)
+ end,
+ Action =
+ fun(#pubsub_node{type = Type, id = Nidx, owners = O}) ->
+ Owners = node_owners_call(Host, Type, Nidx, O),
+ case lists:member(Owner, Owners) of
+ true ->
+ Result =
+ lists:foldl(
+ fun(_, {error, _} = Err) ->
+ Err;
+ (#ps_subscription{jid = JID, type = Sub,
+ subid = SubId} = Entity, _) ->
+ case node_call(Host, Type,
+ set_subscriptions,
+ [Nidx, JID, Sub, SubId]) of
+ {error, _} = Err ->
+ Err;
+ _ ->
+ Notify(Entity)
+ end
+ end, ok, Entities),
+ case Result of
+ ok -> {result, undefined};
+ {error, _} = Err -> Err
+ end;
+ _ ->
+ {error, xmpp:err_forbidden(
+ ?T("Owner privileges required"), ejabberd_option:language())}
+
+ end
+ end,
+ case transaction(Host, Node, Action, sync_dirty) of
+ {result, {_, Result}} -> {result, Result};
+ Other -> Other
+ end.
+-spec get_presence_and_roster_permissions(
+ host(), jid(), [ljid()], accessModel(),
+ [binary()]) -> {boolean(), boolean()}.
get_presence_and_roster_permissions(Host, From, Owners, AccessModel, AllowedGroups) ->
if (AccessModel == presence) or (AccessModel == roster) ->
case Host of
@@ -2981,12 +2622,13 @@ get_presence_and_roster_permissions(Host, From, Owners, AccessModel, AllowedGrou
{true, true}
end.
+-spec get_roster_info(binary(), binary(), ljid() | jid(), [binary()]) -> {boolean(), boolean()}.
get_roster_info(_, _, {<<>>, <<>>, _}, _) ->
{false, false};
get_roster_info(OwnerUser, OwnerServer, {SubscriberUser, SubscriberServer, _}, AllowedGroups) ->
LJID = {SubscriberUser, SubscriberServer, <<>>},
- {Subscription, Groups} = ejabberd_hooks:run_fold(roster_get_jid_info,
- OwnerServer, {none, []},
+ {Subscription, _Ask, Groups} = ejabberd_hooks:run_fold(roster_get_jid_info,
+ OwnerServer, {none, none, []},
[OwnerUser, OwnerServer, LJID]),
PresenceSubscription = Subscription == both orelse
Subscription == from orelse
@@ -2999,67 +2641,44 @@ get_roster_info(OwnerUser, OwnerServer, {SubscriberUser, SubscriberServer, _}, A
get_roster_info(OwnerUser, OwnerServer, JID, AllowedGroups) ->
get_roster_info(OwnerUser, OwnerServer, jid:tolower(JID), AllowedGroups).
-string_to_affiliation(<<"owner">>) -> owner;
-string_to_affiliation(<<"publisher">>) -> publisher;
-string_to_affiliation(<<"publish-only">>) -> publish_only;
-string_to_affiliation(<<"member">>) -> member;
-string_to_affiliation(<<"outcast">>) -> outcast;
-string_to_affiliation(<<"none">>) -> none;
-string_to_affiliation(_) -> false.
-
-string_to_subscription(<<"subscribed">>) -> subscribed;
-string_to_subscription(<<"pending">>) -> pending;
-string_to_subscription(<<"unconfigured">>) -> unconfigured;
-string_to_subscription(<<"none">>) -> none;
-string_to_subscription(_) -> false.
-
-affiliation_to_string(owner) -> <<"owner">>;
-affiliation_to_string(publisher) -> <<"publisher">>;
-affiliation_to_string(publish_only) -> <<"publish-only">>;
-affiliation_to_string(member) -> <<"member">>;
-affiliation_to_string(outcast) -> <<"outcast">>;
-affiliation_to_string(_) -> <<"none">>.
-
-subscription_to_string(subscribed) -> <<"subscribed">>;
-subscription_to_string(pending) -> <<"pending">>;
-subscription_to_string(unconfigured) -> <<"unconfigured">>;
-subscription_to_string(_) -> <<"none">>.
-
--spec service_jid(Host :: mod_pubsub:host()) -> jid().
+-spec preconditions_met(pubsub_publish_options:result(),
+ pubsub_node_config:result()) -> boolean().
+preconditions_met(PubOpts, NodeOpts) ->
+ lists:all(fun(Opt) -> lists:member(Opt, NodeOpts) end, PubOpts).
+
+-spec service_jid(jid() | ljid() | binary()) -> jid().
service_jid(#jid{} = Jid) -> Jid;
service_jid({U, S, R}) -> jid:make(U, S, R);
-service_jid(Host) -> jid:make(<<>>, Host, <<>>).
-
-%% @spec (LJID, NotifyType, Depth, NodeOptions, SubOptions) -> boolean()
-%% LJID = jid()
-%% NotifyType = items | nodes
-%% Depth = integer()
-%% NodeOptions = [{atom(), term()}]
-%% SubOptions = [{atom(), term()}]
+service_jid(Host) -> jid:make(Host).
+
%% @doc <p>Check if a notification must be delivered or not based on
%% node and subscription options.</p>
+-spec is_to_deliver(ljid(), items | nodes, integer(), nodeOptions(), subOptions()) -> boolean().
is_to_deliver(LJID, NotifyType, Depth, NodeOptions, SubOptions) ->
sub_to_deliver(LJID, NotifyType, Depth, SubOptions)
andalso node_to_deliver(LJID, NodeOptions).
+-spec sub_to_deliver(ljid(), items | nodes, integer(), subOptions()) -> boolean().
sub_to_deliver(_LJID, NotifyType, Depth, SubOptions) ->
lists:all(fun (Option) ->
sub_option_can_deliver(NotifyType, Depth, Option)
end,
SubOptions).
+-spec node_to_deliver(ljid(), nodeOptions()) -> boolean().
node_to_deliver(LJID, NodeOptions) ->
presence_can_deliver(LJID, get_option(NodeOptions, presence_based_delivery)).
+-spec sub_option_can_deliver(items | nodes, integer(), _) -> boolean().
sub_option_can_deliver(items, _, {subscription_type, nodes}) -> false;
sub_option_can_deliver(nodes, _, {subscription_type, items}) -> false;
sub_option_can_deliver(_, _, {subscription_depth, all}) -> true;
sub_option_can_deliver(_, Depth, {subscription_depth, D}) -> Depth =< D;
sub_option_can_deliver(_, _, {deliver, false}) -> false;
-sub_option_can_deliver(_, _, {expire, When}) -> p1_time_compat:timestamp() < When;
+sub_option_can_deliver(_, _, {expire, When}) -> erlang:timestamp() < When;
sub_option_can_deliver(_, _, _) -> true.
--spec presence_can_deliver(Entity :: ljid(), _ :: boolean()) -> boolean().
+-spec presence_can_deliver(ljid(), boolean()) -> boolean().
presence_can_deliver(_, false) ->
true;
presence_can_deliver({User, Server, Resource}, true) ->
@@ -3080,10 +2699,7 @@ presence_can_deliver({User, Server, Resource}, true) ->
false, Ss)
end.
--spec state_can_deliver(Entity::ljid(),
- SubOptions :: mod_pubsub:subOptions() | []) ->
- [ljid()].
-
+-spec state_can_deliver(ljid(), subOptions()) -> [ljid()].
state_can_deliver({U, S, R}, []) -> [{U, S, R}];
state_can_deliver({U, S, R}, SubOptions) ->
case lists:keysearch(show_values, 1, SubOptions) of
@@ -3103,10 +2719,7 @@ state_can_deliver({U, S, R}, SubOptions) ->
[], Resources)
end.
--spec get_resource_state(Entity :: ljid(), ShowValues :: [binary()],
- JIDs :: [ljid()]) ->
- [ljid()].
-
+-spec get_resource_state(ljid(), [binary()], [ljid()]) -> [ljid()].
get_resource_state({U, S, R}, ShowValues, JIDs) ->
case ejabberd_sm:get_session_pid(U, S, R) of
none ->
@@ -3114,8 +2727,9 @@ get_resource_state({U, S, R}, ShowValues, JIDs) ->
lists:append([{U, S, R}], JIDs);
Pid ->
Show = case ejabberd_c2s:get_presence(Pid) of
- {_, _, <<"available">>, _} -> <<"online">>;
- {_, _, State, _} -> State
+ #presence{type = unavailable} -> <<"unavailable">>;
+ #presence{show = undefined} -> <<"online">>;
+ #presence{show = Sh} -> atom_to_binary(Sh, latin1)
end,
case lists:member(Show, ShowValues) of
%% If yes, item can be delivered
@@ -3125,78 +2739,66 @@ get_resource_state({U, S, R}, ShowValues, JIDs) ->
end
end.
--spec payload_xmlelements(Payload :: mod_pubsub:payload()) ->
- Count :: non_neg_integer().
+-spec payload_xmlelements([xmlel()]) -> non_neg_integer().
payload_xmlelements(Payload) ->
payload_xmlelements(Payload, 0).
+-spec payload_xmlelements([xmlel()], non_neg_integer()) -> non_neg_integer().
payload_xmlelements([], Count) -> Count;
payload_xmlelements([#xmlel{} | Tail], Count) ->
payload_xmlelements(Tail, Count + 1);
payload_xmlelements([_ | Tail], Count) ->
payload_xmlelements(Tail, Count).
-items_event_stanza(Node, Options, Items) ->
- MoreEls = case Items of
- [LastItem] ->
- {ModifNow, ModifUSR} = LastItem#pubsub_item.modification,
- DateTime = calendar:now_to_datetime(ModifNow),
- {T_string, Tz_string} = jlib:timestamp_to_iso(DateTime, utc),
- [#xmlel{name = <<"delay">>, attrs = [{<<"xmlns">>, ?NS_DELAY},
- {<<"from">>, jid:to_string(ModifUSR)},
- {<<"stamp">>, <<T_string/binary, Tz_string/binary>>}],
- children = [{xmlcdata, <<>>}]}];
+-spec items_els(binary(), nodeOptions(), [#pubsub_item{}]) -> ps_items().
+items_els(Node, Options, Items) ->
+ Els = case get_option(Options, itemreply) of
+ publisher ->
+ [#ps_item{id = ItemId, sub_els = Payload, publisher = jid:encode(USR)}
+ || #pubsub_item{itemid = {ItemId, _}, payload = Payload, modification = {_, USR}}
+ <- Items];
_ ->
- []
+ [#ps_item{id = ItemId, sub_els = Payload}
+ || #pubsub_item{itemid = {ItemId, _}, payload = Payload}
+ <- Items]
end,
- BaseStanza = event_stanza_with_els([#xmlel{name = <<"items">>,
- attrs = nodeAttr(Node),
- children = itemsEls(Items)}],
- MoreEls),
- NotificationType = get_option(Options, notification_type, headline),
- add_message_type(BaseStanza, NotificationType).
-
-event_stanza(Els) ->
- event_stanza_with_els(Els, []).
-event_stanza_with_els(Els, MoreEls) ->
- #xmlel{name = <<"message">>, attrs = [],
- children = [#xmlel{name = <<"event">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB_EVENT}],
- children = Els}
- | MoreEls]}.
-
-event_stanza(Event, EvAttr) ->
- event_stanza_with_els([#xmlel{name = Event, attrs = EvAttr}], []).
+ #ps_items{node = Node, items = Els}.
%%%%%% broadcast functions
+-spec broadcast_publish_item(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), binary(), jid(), [xmlel()], _) ->
+ {result, boolean()}.
broadcast_publish_item(Host, Node, Nidx, Type, NodeOptions, ItemId, From, Payload, Removed) ->
case get_collection_subscriptions(Host, Node) of
- SubsByDepth when is_list(SubsByDepth) ->
- Content = case get_option(NodeOptions, deliver_payloads) of
- true -> Payload;
- false -> []
- end,
- Attrs = case get_option(NodeOptions, itemreply, none) of
- owner -> itemAttr(ItemId); %% owner not supported
- publisher -> itemAttr(ItemId, {<<"publisher">>, jid:to_string(From)});
- none -> itemAttr(ItemId)
- end,
- Stanza = event_stanza(
- [#xmlel{name = <<"items">>, attrs = nodeAttr(Node),
- children = [#xmlel{name = <<"item">>, attrs = Attrs,
- children = Content}]}]),
+ {result, SubsByDepth} ->
+ ItemPublisher = case get_option(NodeOptions, itemreply) of
+ publisher -> jid:encode(From);
+ _ -> <<>>
+ end,
+ ItemPayload = case get_option(NodeOptions, deliver_payloads) of
+ true -> Payload;
+ false -> []
+ end,
+ ItemsEls = #ps_items{node = Node,
+ items = [#ps_item{id = ItemId,
+ publisher = ItemPublisher,
+ sub_els = ItemPayload}]},
+ Stanza = #message{ sub_els = [#ps_event{items = ItemsEls}]},
broadcast_stanza(Host, From, Node, Nidx, Type,
- NodeOptions, SubsByDepth, items, Stanza, true),
+ NodeOptions, SubsByDepth, items, Stanza, true),
case Removed of
[] ->
ok;
_ ->
case get_option(NodeOptions, notify_retract) of
true ->
- RetractStanza = event_stanza(
- [#xmlel{name = <<"items">>, attrs = nodeAttr(Node),
- children = [#xmlel{name = <<"retract">>, attrs = itemAttr(RId)} || RId <- Removed]}]),
+ RetractStanza = #message{
+ sub_els =
+ [#ps_event{
+ items = #ps_items{
+ node = Node,
+ retract = Removed}}]},
broadcast_stanza(Host, Node, Nidx, Type,
NodeOptions, SubsByDepth,
items, RetractStanza, true);
@@ -3209,18 +2811,26 @@ broadcast_publish_item(Host, Node, Nidx, Type, NodeOptions, ItemId, From, Payloa
{result, false}
end.
+-spec broadcast_retract_items(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), [itemId()]) -> {result, boolean()}.
broadcast_retract_items(Host, Node, Nidx, Type, NodeOptions, ItemIds) ->
broadcast_retract_items(Host, Node, Nidx, Type, NodeOptions, ItemIds, false).
+
+-spec broadcast_retract_items(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), [itemId()], boolean()) -> {result, boolean()}.
broadcast_retract_items(_Host, _Node, _Nidx, _Type, _NodeOptions, [], _ForceNotify) ->
{result, false};
broadcast_retract_items(Host, Node, Nidx, Type, NodeOptions, ItemIds, ForceNotify) ->
case (get_option(NodeOptions, notify_retract) or ForceNotify) of
true ->
case get_collection_subscriptions(Host, Node) of
- SubsByDepth when is_list(SubsByDepth) ->
- Stanza = event_stanza(
- [#xmlel{name = <<"items">>, attrs = nodeAttr(Node),
- children = [#xmlel{name = <<"retract">>, attrs = itemAttr(ItemId)} || ItemId <- ItemIds]}]),
+ {result, SubsByDepth} ->
+ Stanza = #message{
+ sub_els =
+ [#ps_event{
+ items = #ps_items{
+ node = Node,
+ retract = ItemIds}}]},
broadcast_stanza(Host, Node, Nidx, Type,
NodeOptions, SubsByDepth, items, Stanza, true),
{result, true};
@@ -3231,13 +2841,13 @@ broadcast_retract_items(Host, Node, Nidx, Type, NodeOptions, ItemIds, ForceNotif
{result, false}
end.
+-spec broadcast_purge_node(host(), binary(), nodeIdx(), binary(), nodeOptions()) -> {result, boolean()}.
broadcast_purge_node(Host, Node, Nidx, Type, NodeOptions) ->
case get_option(NodeOptions, notify_retract) of
true ->
case get_collection_subscriptions(Host, Node) of
- SubsByDepth when is_list(SubsByDepth) ->
- Stanza = event_stanza(
- [#xmlel{name = <<"purge">>, attrs = nodeAttr(Node)}]),
+ {result, SubsByDepth} ->
+ Stanza = #message{sub_els = [#ps_event{purge = Node}]},
broadcast_stanza(Host, Node, Nidx, Type,
NodeOptions, SubsByDepth, nodes, Stanza, false),
{result, true};
@@ -3248,6 +2858,8 @@ broadcast_purge_node(Host, Node, Nidx, Type, NodeOptions) ->
{result, false}
end.
+-spec broadcast_removed_node(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), subs_by_depth()) -> {result, boolean()}.
broadcast_removed_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth) ->
case get_option(NodeOptions, notify_delete) of
true ->
@@ -3255,8 +2867,7 @@ broadcast_removed_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth) ->
[] ->
{result, false};
_ ->
- Stanza = event_stanza(
- [#xmlel{name = <<"delete">>, attrs = nodeAttr(Node)}]),
+ Stanza = #message{sub_els = [#ps_event{delete = {Node, <<>>}}]},
broadcast_stanza(Host, Node, Nidx, Type,
NodeOptions, SubsByDepth, nodes, Stanza, false),
{result, true}
@@ -3265,27 +2876,33 @@ broadcast_removed_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth) ->
{result, false}
end.
+-spec broadcast_created_node(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), subs_by_depth()) -> {result, boolean()}.
broadcast_created_node(_, _, _, _, _, []) ->
{result, false};
broadcast_created_node(Host, Node, Nidx, Type, NodeOptions, SubsByDepth) ->
- Stanza = event_stanza([#xmlel{name = <<"create">>, attrs = nodeAttr(Node)}]),
+ Stanza = #message{sub_els = [#ps_event{create = Node}]},
broadcast_stanza(Host, Node, Nidx, Type, NodeOptions, SubsByDepth, nodes, Stanza, true),
{result, true}.
+-spec broadcast_config_notification(host(), binary(), nodeIdx(), binary(),
+ nodeOptions(), binary()) -> {result, boolean()}.
broadcast_config_notification(Host, Node, Nidx, Type, NodeOptions, Lang) ->
case get_option(NodeOptions, notify_config) of
true ->
case get_collection_subscriptions(Host, Node) of
- SubsByDepth when is_list(SubsByDepth) ->
+ {result, SubsByDepth} ->
Content = case get_option(NodeOptions, deliver_payloads) of
true ->
- [#xmlel{name = <<"x">>, attrs = [{<<"xmlns">>, ?NS_XDATA}, {<<"type">>, <<"result">>}],
- children = get_configure_xfields(Type, NodeOptions, Lang, [])}];
+ #xdata{type = result,
+ fields = get_configure_xfields(
+ Type, NodeOptions, Lang, [])};
false ->
- []
+ undefined
end,
- Stanza = event_stanza(
- [#xmlel{name = <<"configuration">>, attrs = nodeAttr(Node), children = Content}]),
+ Stanza = #message{
+ sub_els = [#ps_event{
+ configuration = {Node, Content}}]},
broadcast_stanza(Host, Node, Nidx, Type,
NodeOptions, SubsByDepth, nodes, Stanza, false),
{result, true};
@@ -3296,26 +2913,48 @@ broadcast_config_notification(Host, Node, Nidx, Type, NodeOptions, Lang) ->
{result, false}
end.
+-spec get_collection_subscriptions(host(), nodeId()) -> {result, subs_by_depth()} |
+ {error, stanza_error()}.
get_collection_subscriptions(Host, Node) ->
- Action = fun() ->
- {result, get_node_subs_by_depth(Host, Node, service_jid(Host))}
- end,
- case transaction(Host, Action, sync_dirty) of
- {result, CollSubs} -> CollSubs;
- _ -> []
- end.
+ Action = fun() -> get_node_subs_by_depth(Host, Node, service_jid(Host)) end,
+ transaction(Host, Action, sync_dirty).
+-spec get_node_subs_by_depth(host(), nodeId(), jid()) -> {result, subs_by_depth()} |
+ {error, stanza_error()}.
get_node_subs_by_depth(Host, Node, From) ->
- ParentTree = tree_call(Host, get_parentnodes_tree, [Host, Node, From]),
- [{Depth, [{N, get_node_subs(Host, N)} || N <- Nodes]} || {Depth, Nodes} <- ParentTree].
+ case tree_call(Host, get_parentnodes_tree, [Host, Node, From]) of
+ ParentTree when is_list(ParentTree) ->
+ {result,
+ lists:filtermap(
+ fun({Depth, Nodes}) ->
+ case lists:filtermap(
+ fun(N) ->
+ case get_node_subs(Host, N) of
+ {result, Result} -> {true, {N, Result}};
+ _ -> false
+ end
+ end, Nodes) of
+ [] -> false;
+ Subs -> {true, {Depth, Subs}}
+ end
+ end, ParentTree)};
+ Error ->
+ Error
+ end.
+-spec get_node_subs(host(), #pubsub_node{}) -> {result, [{ljid(), subId(), subOptions()}]} |
+ {error, stanza_error()}.
get_node_subs(Host, #pubsub_node{type = Type, id = Nidx}) ->
WithOptions = lists:member(<<"subscription-options">>, plugin_features(Host, Type)),
case node_call(Host, Type, get_node_subscriptions, [Nidx]) of
- {result, Subs} -> get_options_for_subs(Host, Nidx, Subs, WithOptions);
+ {result, Subs} -> {result, get_options_for_subs(Host, Nidx, Subs, WithOptions)};
Other -> Other
end.
+-spec get_options_for_subs(host(), nodeIdx(),
+ [{ljid(), subscription(), subId()}],
+ boolean()) ->
+ [{ljid(), subId(), subOptions()}].
get_options_for_subs(_Host, _Nidx, Subs, false) ->
lists:foldl(fun({JID, subscribed, SubID}, Acc) ->
[{JID, SubID, []} | Acc];
@@ -3333,11 +2972,15 @@ get_options_for_subs(Host, Nidx, Subs, true) ->
Acc
end, [], Subs).
+-spec broadcast_stanza(host(), nodeId(), nodeIdx(), binary(),
+ nodeOptions(), subs_by_depth(),
+ items | nodes, stanza(), boolean()) -> ok.
broadcast_stanza(Host, _Node, _Nidx, _Type, NodeOptions, SubsByDepth, NotifyType, BaseStanza, SHIM) ->
NotificationType = get_option(NodeOptions, notification_type, headline),
- BroadcastAll = get_option(NodeOptions, broadcast_all_resources), %% XXX this is not standard, but usefull
- From = service_jid(Host),
- Stanza = add_message_type(BaseStanza, NotificationType),
+ BroadcastAll = get_option(NodeOptions, broadcast_all_resources), %% XXX this is not standard, but useful
+ Stanza = add_message_type(
+ xmpp:set_from(BaseStanza, service_jid(Host)),
+ NotificationType),
%% Handles explicit subscriptions
SubIDsByJID = subscribed_nodes_by_jid(NotifyType, SubsByDepth),
lists:foreach(fun ({LJID, _NodeName, SubIDs}) ->
@@ -3359,94 +3002,268 @@ broadcast_stanza(Host, _Node, _Nidx, _Type, NodeOptions, SubsByDepth, NotifyType
add_shim_headers(Stanza, subid_shim(SubIDs))
end,
lists:foreach(fun(To) ->
- ejabberd_router:route(From, jid:make(To), StanzaToSend)
+ ejabberd_router:route(
+ xmpp:set_to(StanzaToSend, jid:make(To)))
end, LJIDs)
end, SubIDsByJID).
+-spec broadcast_stanza(host(), jid(), nodeId(), nodeIdx(), binary(),
+ nodeOptions(), subs_by_depth(), items | nodes,
+ stanza(), boolean()) -> ok.
broadcast_stanza({LUser, LServer, LResource}, Publisher, Node, Nidx, Type, NodeOptions, SubsByDepth, NotifyType, BaseStanza, SHIM) ->
broadcast_stanza({LUser, LServer, <<>>}, Node, Nidx, Type, NodeOptions, SubsByDepth, NotifyType, BaseStanza, SHIM),
%% Handles implicit presence subscriptions
SenderResource = user_resource(LUser, LServer, LResource),
- case ejabberd_sm:get_session_pid(LUser, LServer, SenderResource) of
- C2SPid when is_pid(C2SPid) ->
- NotificationType = get_option(NodeOptions, notification_type, headline),
- Stanza = add_message_type(BaseStanza, NotificationType),
- %% set the from address on the notification to the bare JID of the account owner
- %% Also, add "replyto" if entity has presence subscription to the account owner
- %% See XEP-0163 1.1 section 4.3.1
- ejabberd_c2s:broadcast(C2SPid,
- {pep_message, <<((Node))/binary, "+notify">>},
- _Sender = jid:make(LUser, LServer, <<"">>),
- _StanzaToSend = add_extended_headers(Stanza,
- _ReplyTo = extended_headers([jid:to_string(Publisher)])));
- _ ->
- ?DEBUG("~p@~p has no session; can't deliver ~p to contacts", [LUser, LServer, BaseStanza])
- end;
+ NotificationType = get_option(NodeOptions, notification_type, headline),
+ %% set the from address on the notification to the bare JID of the account owner
+ %% Also, add "replyto" if entity has presence subscription to the account owner
+ %% See XEP-0163 1.1 section 4.3.1
+ Owner = jid:make(LUser, LServer),
+ FromBareJid = xmpp:set_from(BaseStanza, Owner),
+ Stanza = add_extended_headers(
+ add_message_type(FromBareJid, NotificationType),
+ extended_headers([Publisher])),
+ Pred = fun(To) -> delivery_permitted(Owner, To, NodeOptions) end,
+ ejabberd_sm:route(jid:make(LUser, LServer, SenderResource),
+ {pep_message, <<((Node))/binary, "+notify">>, Stanza, Pred}),
+ ejabberd_router:route(xmpp:set_to(Stanza, jid:make(LUser, LServer)));
broadcast_stanza(Host, _Publisher, Node, Nidx, Type, NodeOptions, SubsByDepth, NotifyType, BaseStanza, SHIM) ->
broadcast_stanza(Host, Node, Nidx, Type, NodeOptions, SubsByDepth, NotifyType, BaseStanza, SHIM).
+-spec c2s_handle_info(ejabberd_c2s:state(), term()) -> ejabberd_c2s:state().
+c2s_handle_info(#{lserver := LServer} = C2SState,
+ {pep_message, Feature, Packet, Pred}) when is_function(Pred) ->
+ [maybe_send_pep_stanza(LServer, USR, Caps, Feature, Packet)
+ || {USR, Caps} <- mod_caps:list_features(C2SState), Pred(USR)],
+ {stop, C2SState};
+c2s_handle_info(#{lserver := LServer} = C2SState,
+ {pep_message, Feature, Packet, {_, _, _} = USR}) ->
+ case mod_caps:get_user_caps(USR, C2SState) of
+ {ok, Caps} -> maybe_send_pep_stanza(LServer, USR, Caps, Feature, Packet);
+ error -> ok
+ end,
+ {stop, C2SState};
+c2s_handle_info(C2SState, _) ->
+ C2SState.
+
+-spec send_items(host(), nodeId(), nodeIdx(), binary(),
+ nodeOptions(), ljid(), last | integer()) -> ok.
+send_items(Host, Node, Nidx, Type, Options, LJID, Number) ->
+ send_items(Host, Node, Nidx, Type, Options, Host, LJID, LJID, Number).
+send_items(Host, Node, Nidx, Type, Options, Publisher, SubLJID, ToLJID, Number) ->
+ Items = case max_items(Host, Options) of
+ 1 ->
+ get_only_item(Host, Type, Nidx, SubLJID);
+ _ ->
+ get_last_items(Host, Type, Nidx, SubLJID, Number)
+ end,
+ case Items of
+ [] ->
+ ok;
+ Items ->
+ Delay = case Number of
+ last -> % handle section 6.1.7 of XEP-0060
+ [Last] = Items,
+ {Stamp, _USR} = Last#pubsub_item.modification,
+ [#delay{stamp = Stamp}];
+ _ ->
+ []
+ end,
+ Stanza = #message{
+ sub_els = [#ps_event{items = items_els(Node, Options, Items)}
+ | Delay]},
+ NotificationType = get_option(Options, notification_type, headline),
+ send_stanza(Publisher, ToLJID, Node,
+ add_message_type(Stanza, NotificationType))
+ end.
+
+-spec send_stanza(host(), ljid(), binary(), stanza()) -> ok.
+send_stanza({LUser, LServer, _} = Publisher, USR, Node, BaseStanza) ->
+ Stanza = xmpp:set_from(BaseStanza, jid:make(LUser, LServer)),
+ USRs = case USR of
+ {PUser, PServer, <<>>} ->
+ [{PUser, PServer, PRessource}
+ || PRessource <- user_resources(PUser, PServer)];
+ _ ->
+ [USR]
+ end,
+ lists:foreach(
+ fun(To) ->
+ ejabberd_sm:route(
+ jid:make(Publisher),
+ {pep_message, <<((Node))/binary, "+notify">>,
+ add_extended_headers(
+ Stanza, extended_headers([jid:make(Publisher)])),
+ To})
+ end, USRs);
+send_stanza(Host, USR, _Node, Stanza) ->
+ ejabberd_router:route(
+ xmpp:set_from_to(Stanza, service_jid(Host), jid:make(USR))).
+
+-spec maybe_send_pep_stanza(binary(), ljid(), caps(), binary(), stanza()) -> ok.
+maybe_send_pep_stanza(LServer, USR, Caps, Feature, Packet) ->
+ Features = mod_caps:get_features(LServer, Caps),
+ case lists:member(Feature, Features) of
+ true ->
+ ejabberd_router:route(xmpp:set_to(Packet, jid:make(USR)));
+ false ->
+ ok
+ end.
+
+-spec send_last_items(jid()) -> ok.
+send_last_items(JID) ->
+ ServerHost = JID#jid.lserver,
+ Host = host(ServerHost),
+ DBType = config(ServerHost, db_type),
+ LJID = jid:tolower(JID),
+ BJID = jid:remove_resource(LJID),
+ lists:foreach(
+ fun(PType) ->
+ Subs = get_subscriptions_for_send_last(Host, PType, DBType, JID, LJID, BJID),
+ lists:foreach(
+ fun({#pubsub_node{nodeid = {_, Node}, type = Type, id = Nidx,
+ options = Options}, _, SubJID})
+ when Type == PType->
+ send_items(Host, Node, Nidx, PType, Options, Host, SubJID, LJID, 1);
+ (_) ->
+ ok
+ end,
+ lists:usort(Subs))
+ end, config(ServerHost, plugins)).
+% pep_from_offline hack can not work anymore, as sender c2s does not
+% exists when sender is offline, so we can't get match receiver caps
+% does it make sens to send PEP from an offline contact anyway ?
+% case config(ServerHost, ignore_pep_from_offline) of
+% false ->
+% Roster = ejabberd_hooks:run_fold(roster_get, ServerHost, [],
+% [{JID#jid.luser, ServerHost}]),
+% lists:foreach(
+% fun(#roster{jid = {U, S, R}, subscription = Sub})
+% when Sub == both orelse Sub == from,
+% S == ServerHost ->
+% case user_resources(U, S) of
+% [] -> send_last_pep(jid:make(U, S, R), JID);
+% _ -> ok %% this is already handled by presence probe
+% end;
+% (_) ->
+% ok %% we can not do anything in any cases
+% end, Roster);
+% true ->
+% ok
+% end.
+-spec send_last_pep(jid(), jid()) -> ok.
+send_last_pep(From, To) ->
+ ServerHost = From#jid.lserver,
+ Host = host(ServerHost),
+ Publisher = jid:tolower(From),
+ Owner = jid:remove_resource(Publisher),
+ case tree_action(Host, get_nodes, [Owner, infinity]) of
+ Nodes when is_list(Nodes) ->
+ lists:foreach(
+ fun(#pubsub_node{nodeid = {_, Node}, type = Type, id = Nidx, options = Options}) ->
+ case match_option(Options, send_last_published_item, on_sub_and_presence) of
+ true ->
+ case delivery_permitted(From, To, Options) of
+ true ->
+ LJID = jid:tolower(To),
+ send_items(Owner, Node, Nidx, Type, Options,
+ Publisher, LJID, LJID, 1);
+ false ->
+ ok
+ end;
+ _ ->
+ ok
+ end
+ end, Nodes);
+ _ ->
+ ok
+ end.
+
+-spec subscribed_nodes_by_jid(items | nodes, subs_by_depth()) -> [{ljid(), binary(), subId()}].
subscribed_nodes_by_jid(NotifyType, SubsByDepth) ->
NodesToDeliver = fun (Depth, Node, Subs, Acc) ->
- NodeName = case Node#pubsub_node.nodeid of
- {_, N} -> N;
- Other -> Other
- end,
- NodeOptions = Node#pubsub_node.options,
- lists:foldl(fun({LJID, SubID, SubOptions}, {JIDs, Recipients}) ->
- case is_to_deliver(LJID, NotifyType, Depth, NodeOptions, SubOptions) of
- true ->
- case state_can_deliver(LJID, SubOptions) of
- [] -> {JIDs, Recipients};
- JIDsToDeliver ->
- lists:foldl(
- fun(JIDToDeliver, {JIDsAcc, RecipientsAcc}) ->
- case lists:member(JIDToDeliver, JIDs) of
- %% check if the JIDs co-accumulator contains the Subscription Jid,
- false ->
- %% - if not,
- %% - add the Jid to JIDs list co-accumulator ;
- %% - create a tuple of the Jid, Nidx, and SubID (as list),
- %% and add the tuple to the Recipients list co-accumulator
- {[JIDToDeliver | JIDsAcc],
- [{JIDToDeliver, NodeName, [SubID]}
- | RecipientsAcc]};
- true ->
- %% - if the JIDs co-accumulator contains the Jid
- %% get the tuple containing the Jid from the Recipient list co-accumulator
- {_, {JIDToDeliver, NodeName1, SubIDs}} =
- lists:keysearch(JIDToDeliver, 1, RecipientsAcc),
- %% delete the tuple from the Recipients list
- % v1 : Recipients1 = lists:keydelete(LJID, 1, Recipients),
- % v2 : Recipients1 = lists:keyreplace(LJID, 1, Recipients, {LJID, Nidx1, [SubID | SubIDs]}),
- %% add the SubID to the SubIDs list in the tuple,
- %% and add the tuple back to the Recipients list co-accumulator
- % v1.1 : {JIDs, lists:append(Recipients1, [{LJID, Nidx1, lists:append(SubIDs, [SubID])}])}
- % v1.2 : {JIDs, [{LJID, Nidx1, [SubID | SubIDs]} | Recipients1]}
- % v2: {JIDs, Recipients1}
- {JIDsAcc,
- lists:keyreplace(JIDToDeliver, 1,
- RecipientsAcc,
- {JIDToDeliver, NodeName1,
- [SubID | SubIDs]})}
- end
- end, {JIDs, Recipients}, JIDsToDeliver)
- end;
- false ->
- {JIDs, Recipients}
- end
- end, Acc, Subs)
- end,
+ NodeName = case Node#pubsub_node.nodeid of
+ {_, N} -> N;
+ Other -> Other
+ end,
+ NodeOptions = Node#pubsub_node.options,
+ lists:foldl(fun({LJID, SubID, SubOptions}, {JIDs, Recipients}) ->
+ case is_to_deliver(LJID, NotifyType, Depth, NodeOptions, SubOptions) of
+ true ->
+ case state_can_deliver(LJID, SubOptions) of
+ [] -> {JIDs, Recipients};
+ [LJID] -> {JIDs, [{LJID, NodeName, [SubID]} | Recipients]};
+ JIDsToDeliver ->
+ lists:foldl(
+ fun(JIDToDeliver, {JIDsAcc, RecipientsAcc}) ->
+ case lists:member(JIDToDeliver, JIDs) of
+ %% check if the JIDs co-accumulator contains the Subscription Jid,
+ false ->
+ %% - if not,
+ %% - add the Jid to JIDs list co-accumulator ;
+ %% - create a tuple of the Jid, Nidx, and SubID (as list),
+ %% and add the tuple to the Recipients list co-accumulator
+ {[JIDToDeliver | JIDsAcc],
+ [{JIDToDeliver, NodeName, [SubID]}
+ | RecipientsAcc]};
+ true ->
+ %% - if the JIDs co-accumulator contains the Jid
+ %% get the tuple containing the Jid from the Recipient list co-accumulator
+ {_, {JIDToDeliver, NodeName1, SubIDs}} =
+ lists:keysearch(JIDToDeliver, 1, RecipientsAcc),
+ %% delete the tuple from the Recipients list
+ % v1 : Recipients1 = lists:keydelete(LJID, 1, Recipients),
+ % v2 : Recipients1 = lists:keyreplace(LJID, 1, Recipients, {LJID, Nidx1, [SubID | SubIDs]}),
+ %% add the SubID to the SubIDs list in the tuple,
+ %% and add the tuple back to the Recipients list co-accumulator
+ % v1.1 : {JIDs, lists:append(Recipients1, [{LJID, Nidx1, lists:append(SubIDs, [SubID])}])}
+ % v1.2 : {JIDs, [{LJID, Nidx1, [SubID | SubIDs]} | Recipients1]}
+ % v2: {JIDs, Recipients1}
+ {JIDsAcc,
+ lists:keyreplace(JIDToDeliver, 1,
+ RecipientsAcc,
+ {JIDToDeliver, NodeName1,
+ [SubID | SubIDs]})}
+ end
+ end, {JIDs, Recipients}, JIDsToDeliver)
+ end;
+ false ->
+ {JIDs, Recipients}
+ end
+ end, Acc, Subs)
+ end,
DepthsToDeliver = fun({Depth, SubsByNode}, Acc1) ->
- lists:foldl(fun({Node, Subs}, Acc2) ->
- NodesToDeliver(Depth, Node, Subs, Acc2)
- end, Acc1, SubsByNode)
- end,
+ lists:foldl(fun({Node, Subs}, Acc2) ->
+ NodesToDeliver(Depth, Node, Subs, Acc2)
+ end, Acc1, SubsByNode)
+ end,
{_, JIDSubs} = lists:foldl(DepthsToDeliver, {[], []}, SubsByDepth),
JIDSubs.
+-spec delivery_permitted(jid() | ljid(), jid() | ljid(), nodeOptions()) -> boolean().
+delivery_permitted(From, To, Options) ->
+ LFrom = jid:tolower(From),
+ LTo = jid:tolower(To),
+ RecipientIsOwner = jid:remove_resource(LFrom) == jid:remove_resource(LTo),
+ %% TODO: Fix the 'whitelist'/'authorize' cases for last PEP notifications.
+ %% Currently, only node owners receive those.
+ case get_option(Options, access_model) of
+ open -> true;
+ presence -> true;
+ whitelist -> RecipientIsOwner;
+ authorize -> RecipientIsOwner;
+ roster ->
+ Grps = get_option(Options, roster_groups_allowed, []),
+ {LUser, LServer, _} = LFrom,
+ {_, IsInGrp} = get_roster_info(LUser, LServer, LTo, Grps),
+ IsInGrp
+ end.
+
+-spec user_resources(binary(), binary()) -> [binary()].
user_resources(User, Server) ->
ejabberd_sm:get_user_resources(User, Server).
+-spec user_resource(binary(), binary(), binary()) -> binary().
user_resource(User, Server, <<>>) ->
case user_resources(User, Server) of
[R | _] -> R;
@@ -3456,26 +3273,21 @@ user_resource(_, _, Resource) ->
Resource.
%%%%%%% Configuration handling
-
+-spec get_configure(host(), binary(), binary(), jid(),
+ binary()) -> {error, stanza_error()} | {result, pubsub_owner()}.
get_configure(Host, ServerHost, Node, From, Lang) ->
Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx}) ->
case node_call(Host, Type, get_affiliation, [Nidx, From]) of
{result, owner} ->
Groups = ejabberd_hooks:run_fold(roster_groups, ServerHost, [], [ServerHost]),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB_OWNER}],
- children =
- [#xmlel{name = <<"configure">>,
- attrs = nodeAttr(Node),
- children =
- [#xmlel{name = <<"x">>,
- attrs = [{<<"xmlns">>, ?NS_XDATA},
- {<<"type">>, <<"form">>}],
- children =
- get_configure_xfields(Type, Options, Lang, Groups)}]}]}]};
- _ ->
- {error, ?ERRT_FORBIDDEN(Lang, <<"You're not an owner">>)}
+ Fs = get_configure_xfields(Type, Options, Lang, Groups),
+ {result, #pubsub_owner{
+ configure =
+ {Node, #xdata{type = form, fields = Fs}}}};
+ {result, _} ->
+ {error, xmpp:err_forbidden(?T("Owner privileges required"), Lang)};
+ Error ->
+ Error
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -3483,20 +3295,14 @@ get_configure(Host, ServerHost, Node, From, Lang) ->
Other -> Other
end.
+-spec get_default(host(), binary(), jid(), binary()) -> {result, pubsub_owner()}.
get_default(Host, Node, _From, Lang) ->
- Type = select_type(Host, Host, Node),
+ Type = select_type(serverhost(Host), Host, Node),
Options = node_options(Host, Type),
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB_OWNER}],
- children =
- [#xmlel{name = <<"default">>, attrs = [],
- children =
- [#xmlel{name = <<"x">>,
- attrs = [{<<"xmlns">>, ?NS_XDATA},
- {<<"type">>, <<"form">>}],
- children = get_configure_xfields(Type, Options, Lang, [])}]}]}]}.
+ Fs = get_configure_xfields(Type, Options, Lang, []),
+ {result, #pubsub_owner{default = {<<>>, #xdata{type = form, fields = Fs}}}}.
+-spec match_option(#pubsub_node{} | [{atom(), any()}], atom(), any()) -> boolean().
match_option(Node, Var, Val) when is_record(Node, pubsub_node) ->
match_option(Node#pubsub_node.options, Var, Val);
match_option(Options, Var, Val) when is_list(Options) ->
@@ -3504,21 +3310,26 @@ match_option(Options, Var, Val) when is_list(Options) ->
match_option(_, _, _) ->
false.
+-spec get_option([{atom(), any()}], atom()) -> any().
get_option([], _) -> false;
get_option(Options, Var) -> get_option(Options, Var, false).
+-spec get_option([{atom(), any()}], atom(), any()) -> any().
get_option(Options, Var, Def) ->
case lists:keysearch(Var, 1, Options) of
{value, {_Val, Ret}} -> Ret;
_ -> Def
end.
+-spec node_options(host(), binary()) -> [{atom(), any()}].
node_options(Host, Type) ->
- case config(Host, default_node_config) of
- undefined -> node_plugin_options(Host, Type);
- [] -> node_plugin_options(Host, Type);
- Config -> Config
+ DefaultOpts = node_plugin_options(Host, Type),
+ case config(Host, plugins) of
+ [Type|_] -> config(Host, default_node_config, DefaultOpts);
+ _ -> DefaultOpts
end.
+
+-spec node_plugin_options(host(), binary()) -> [{atom(), any()}].
node_plugin_options(Host, Type) ->
Module = plugin(Host, Type),
case catch Module:options() of
@@ -3528,38 +3339,39 @@ node_plugin_options(Host, Type) ->
Result ->
Result
end.
-filter_node_options(Options, BaseOptions) ->
- lists:foldl(fun({Key, Val}, Acc) ->
- DefaultValue = proplists:get_value(Key, Options, Val),
- [{Key, DefaultValue}|Acc]
- end, [], BaseOptions).
+-spec node_owners_action(host(), binary(), nodeIdx(), [ljid()]) -> [ljid()].
node_owners_action(Host, Type, Nidx, []) ->
- case gen_mod:db_type(serverhost(Host), ?MODULE) of
- sql ->
- case node_action(Host, Type, get_node_affiliations, [Nidx]) of
- {result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
- _ -> []
- end;
- _ ->
- []
+ case node_action(Host, Type, get_node_affiliations, [Nidx]) of
+ {result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
+ _ -> []
end;
node_owners_action(_Host, _Type, _Nidx, Owners) ->
Owners.
+-spec node_owners_call(host(), binary(), nodeIdx(), [ljid()]) -> [ljid()].
node_owners_call(Host, Type, Nidx, []) ->
- case gen_mod:db_type(serverhost(Host), ?MODULE) of
- sql ->
- case node_call(Host, Type, get_node_affiliations, [Nidx]) of
- {result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
- _ -> []
- end;
- _ ->
- []
+ case node_call(Host, Type, get_node_affiliations, [Nidx]) of
+ {result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
+ _ -> []
end;
node_owners_call(_Host, _Type, _Nidx, Owners) ->
Owners.
+node_config(Node, ServerHost) ->
+ Opts = mod_pubsub_opt:force_node_config(ServerHost),
+ node_config(Node, ServerHost, Opts).
+
+node_config(Node, ServerHost, [{RE, Opts}|NodeOpts]) ->
+ case re:run(Node, RE) of
+ {match, _} ->
+ Opts;
+ nomatch ->
+ node_config(Node, ServerHost, NodeOpts)
+ end;
+node_config(_, _, []) ->
+ [].
+
%% @spec (Host, Options) -> MaxItems
%% Host = host()
%% Options = [Option]
@@ -3568,9 +3380,7 @@ node_owners_call(_Host, _Type, _Nidx, Owners) ->
%% @doc <p>Return the maximum number of items for a given node.</p>
%% <p>Unlimited means that there is no limit in the number of items that can
%% be stored.</p>
-%% @todo In practice, the current data structure means that we cannot manage
-%% millions of items on a given node. This should be addressed in a new
-%% version.
+-spec max_items(host(), [{atom(), any()}]) -> non_neg_integer().
max_items(Host, Options) ->
case get_option(Options, persist_items) of
true ->
@@ -3591,83 +3401,18 @@ max_items(Host, Options) ->
end
end.
--define(BOOL_CONFIG_FIELD(Label, Var),
- ?BOOLXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (get_option(Options, Var)))).
-
--define(STRING_CONFIG_FIELD(Label, Var),
- ?STRINGXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (get_option(Options, Var, <<>>)))).
-
--define(INTEGER_CONFIG_FIELD(Label, Var),
- ?STRINGXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (jlib:integer_to_binary(get_option(Options, Var))))).
-
--define(JLIST_CONFIG_FIELD(Label, Var, Opts),
- ?LISTXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (jid:to_string(get_option(Options, Var))),
- [jid:to_string(O) || O <- Opts])).
-
--define(ALIST_CONFIG_FIELD(Label, Var, Opts),
- ?LISTXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (atom_to_binary(get_option(Options, Var), latin1)),
- [atom_to_binary(O, latin1) || O <- Opts])).
-
--define(LISTM_CONFIG_FIELD(Label, Var, Opts),
- ?LISTMXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- (get_option(Options, Var)), Opts)).
-
--define(NLIST_CONFIG_FIELD(Label, Var),
- ?STRINGMXFIELD(Label,
- <<"pubsub#", (atom_to_binary(Var, latin1))/binary>>,
- get_option(Options, Var, []))).
-
+-spec get_configure_xfields(_, pubsub_node_config:result(),
+ binary(), [binary()]) -> [xdata_field()].
get_configure_xfields(_Type, Options, Lang, Groups) ->
- [?XFIELD(<<"hidden">>, <<>>, <<"FORM_TYPE">>, (?NS_PUBSUB_NODE_CONFIG)),
- ?BOOL_CONFIG_FIELD(<<"Deliver payloads with event notifications">>,
- deliver_payloads),
- ?BOOL_CONFIG_FIELD(<<"Deliver event notifications">>,
- deliver_notifications),
- ?BOOL_CONFIG_FIELD(<<"Notify subscribers when the node configuration changes">>,
- notify_config),
- ?BOOL_CONFIG_FIELD(<<"Notify subscribers when the node is deleted">>,
- notify_delete),
- ?BOOL_CONFIG_FIELD(<<"Notify subscribers when items are removed from the node">>,
- notify_retract),
- ?BOOL_CONFIG_FIELD(<<"Persist items to storage">>,
- persist_items),
- ?STRING_CONFIG_FIELD(<<"A friendly name for the node">>,
- title),
- ?INTEGER_CONFIG_FIELD(<<"Max # of items to persist">>,
- max_items),
- ?BOOL_CONFIG_FIELD(<<"Whether to allow subscriptions">>,
- subscribe),
- ?ALIST_CONFIG_FIELD(<<"Specify the access model">>,
- access_model, [open, authorize, presence, roster, whitelist]),
- ?LISTM_CONFIG_FIELD(<<"Roster groups allowed to subscribe">>,
- roster_groups_allowed, Groups),
- ?ALIST_CONFIG_FIELD(<<"Specify the publisher model">>,
- publish_model, [publishers, subscribers, open]),
- ?BOOL_CONFIG_FIELD(<<"Purge all items when the relevant publisher goes offline">>,
- purge_offline),
- ?ALIST_CONFIG_FIELD(<<"Specify the event message type">>,
- notification_type, [headline, normal]),
- ?INTEGER_CONFIG_FIELD(<<"Max payload size in bytes">>,
- max_payload_size),
- ?ALIST_CONFIG_FIELD(<<"When to send the last published item">>,
- send_last_published_item, [never, on_sub, on_sub_and_presence]),
- ?BOOL_CONFIG_FIELD(<<"Only deliver notifications to available users">>,
- presence_based_delivery),
- ?NLIST_CONFIG_FIELD(<<"The collections with which a node is affiliated">>,
- collection),
- ?ALIST_CONFIG_FIELD(<<"Whether owners or publisher should receive replies to items">>,
- itemreply, [none, owner, publisher])].
+ pubsub_node_config:encode(
+ lists:filtermap(
+ fun({roster_groups_allowed, Value}) ->
+ {true, {roster_groups_allowed, Value, Groups}};
+ ({sql, _}) -> false;
+ ({rsm, _}) -> false;
+ (_) -> true
+ end, Options),
+ Lang).
%%<p>There are several reasons why the node configuration request might fail:</p>
%%<ul>
@@ -3677,210 +3422,166 @@ get_configure_xfields(_Type, Options, Lang, Groups) ->
%%<li>The node has no configuration options.</li>
%%<li>The specified node does not exist.</li>
%%</ul>
-set_configure(Host, Node, From, Els, Lang) ->
- case fxml:remove_cdata(Els) of
- [#xmlel{name = <<"x">>} = XEl] ->
- case {fxml:get_tag_attr_s(<<"xmlns">>, XEl), fxml:get_tag_attr_s(<<"type">>, XEl)} of
- {?NS_XDATA, <<"cancel">>} ->
- {result, []};
- {?NS_XDATA, <<"submit">>} ->
- Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx} = N) ->
- case node_call(Host, Type, get_affiliation, [Nidx, From]) of
- {result, owner} ->
- case jlib:parse_xdata_submit(XEl) of
- invalid ->
- Txt = <<"Incorrect data form">>,
- {error, ?ERRT_BAD_REQUEST(Lang, Txt)};
- XData ->
- OldOpts = case Options of
- [] -> node_options(Host, Type);
- _ -> Options
- end,
- case set_xoption(Host, XData, OldOpts) of
- NewOpts when is_list(NewOpts) ->
- case tree_call(Host,
- set_node,
- [N#pubsub_node{options = NewOpts}])
- of
- {result, Nidx} -> {result, ok};
- ok -> {result, ok};
- Err -> Err
- end;
- Error ->
- Error
- end
- end;
- _ ->
- Txt = <<"You're not an owner">>,
- {error, ?ERRT_FORBIDDEN(Lang, Txt)}
- end
- end,
- case transaction(Host, Node, Action, transaction) of
- {result, {TNode, ok}} ->
- Nidx = TNode#pubsub_node.id,
- Type = TNode#pubsub_node.type,
- Options = TNode#pubsub_node.options,
- broadcast_config_notification(Host, Node, Nidx, Type, Options, Lang),
- {result, []};
- Other ->
- Other
- end;
- _ ->
- Txt = <<"Incorrect data form">>,
- {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
- end;
- _ ->
- Txt = <<"No data form found">>,
- {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
+-spec set_configure(host(), binary(), jid(), [{binary(), [binary()]}],
+ binary()) -> {result, undefined} | {error, stanza_error()}.
+set_configure(_Host, <<>>, _From, _Config, _Lang) ->
+ {error, extended_error(xmpp:err_bad_request(), err_nodeid_required())};
+set_configure(Host, Node, From, Config, Lang) ->
+ Action =
+ fun(#pubsub_node{options = Options, type = Type, id = Nidx} = N) ->
+ case node_call(Host, Type, get_affiliation, [Nidx, From]) of
+ {result, owner} ->
+ OldOpts = case Options of
+ [] -> node_options(Host, Type);
+ _ -> Options
+ end,
+ NewOpts = merge_config(
+ [node_config(Node, serverhost(Host)),
+ Config, OldOpts]),
+ case tree_call(Host,
+ set_node,
+ [N#pubsub_node{options = NewOpts}]) of
+ {result, Nidx} -> {result, NewOpts};
+ ok -> {result, NewOpts};
+ Err -> Err
+ end;
+ {result, _} ->
+ {error, xmpp:err_forbidden(
+ ?T("Owner privileges required"), Lang)};
+ Error ->
+ Error
+ end
+ end,
+ case transaction(Host, Node, Action, transaction) of
+ {result, {TNode, Options}} ->
+ Nidx = TNode#pubsub_node.id,
+ Type = TNode#pubsub_node.type,
+ broadcast_config_notification(Host, Node, Nidx, Type, Options, Lang),
+ {result, undefined};
+ Other ->
+ Other
end.
-add_opt(Key, Value, Opts) ->
- [{Key, Value} | lists:keydelete(Key, 1, Opts)].
+-spec merge_config([[proplists:property()]]) -> [proplists:property()].
+merge_config(ListOfConfigs) ->
+ lists:ukeysort(1, lists:flatten(ListOfConfigs)).
+
+-spec decode_node_config(undefined | xdata(), binary(), binary()) ->
+ pubsub_node_config:result() |
+ {error, stanza_error()}.
+decode_node_config(undefined, _, _) ->
+ [];
+decode_node_config(#xdata{fields = Fs}, Host, Lang) ->
+ try
+ Config = pubsub_node_config:decode(Fs),
+ Max = get_max_items_node(Host),
+ case {check_opt_range(max_items, Config, Max),
+ check_opt_range(max_payload_size, Config, ?MAX_PAYLOAD_SIZE)} of
+ {true, true} ->
+ Config;
+ {true, false} ->
+ erlang:error(
+ {pubsub_node_config,
+ {bad_var_value, <<"pubsub#max_payload_size">>,
+ ?NS_PUBSUB_NODE_CONFIG}});
+ {false, _} ->
+ erlang:error(
+ {pubsub_node_config,
+ {bad_var_value, <<"pubsub#max_items">>,
+ ?NS_PUBSUB_NODE_CONFIG}})
+ end
+ catch _:{pubsub_node_config, Why} ->
+ Txt = pubsub_node_config:format_error(Why),
+ {error, xmpp:err_resource_constraint(Txt, Lang)}
+ end.
--define(SET_BOOL_XOPT(Opt, Val),
- BoolVal = case Val of
- <<"0">> -> false;
- <<"1">> -> true;
- <<"false">> -> false;
- <<"true">> -> true;
- _ -> error
- end,
- case BoolVal of
- error ->
- Txt = <<"Value of '~s' should be boolean">>,
- ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
- {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)};
- _ -> set_xoption(Host, Opts, add_opt(Opt, BoolVal, NewOpts))
- end).
-
--define(SET_STRING_XOPT(Opt, Val),
- set_xoption(Host, Opts, add_opt(Opt, Val, NewOpts))).
-
--define(SET_INTEGER_XOPT(Opt, Val, Min, Max),
- case catch jlib:binary_to_integer(Val) of
- IVal when is_integer(IVal), IVal >= Min ->
- if (Max =:= undefined) orelse (IVal =< Max) ->
- set_xoption(Host, Opts, add_opt(Opt, IVal, NewOpts));
- true ->
- Txt = <<"Incorrect value of '~s'">>,
- ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
- {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
- end;
- _ ->
- Txt = <<"Value of '~s' should be integer">>,
- ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
- {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
- end).
+-spec decode_subscribe_options(undefined | xdata(), binary()) ->
+ pubsub_subscribe_options:result() |
+ {error, stanza_error()}.
+decode_subscribe_options(undefined, _) ->
+ [];
+decode_subscribe_options(#xdata{fields = Fs}, Lang) ->
+ try pubsub_subscribe_options:decode(Fs)
+ catch _:{pubsub_subscribe_options, Why} ->
+ Txt = pubsub_subscribe_options:format_error(Why),
+ {error, xmpp:err_resource_constraint(Txt, Lang)}
+ end.
--define(SET_ALIST_XOPT(Opt, Val, Vals),
- case lists:member(Val, [atom_to_binary(V, latin1) || V <- Vals]) of
- true ->
- set_xoption(Host, Opts, add_opt(Opt, jlib:binary_to_atom(Val), NewOpts));
- false ->
- Txt = <<"Incorrect value of '~s'">>,
- ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
- {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
- end).
-
--define(SET_LIST_XOPT(Opt, Val),
- set_xoption(Host, Opts, add_opt(Opt, Val, NewOpts))).
-
-set_xoption(_Host, [], NewOpts) -> NewOpts;
-set_xoption(Host, [{<<"FORM_TYPE">>, _} | Opts], NewOpts) ->
- set_xoption(Host, Opts, NewOpts);
-set_xoption(Host, [{<<"pubsub#roster_groups_allowed">>, Value} | Opts], NewOpts) ->
- ?SET_LIST_XOPT(roster_groups_allowed, Value);
-set_xoption(Host, [{<<"pubsub#deliver_payloads">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(deliver_payloads, Val);
-set_xoption(Host, [{<<"pubsub#deliver_notifications">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(deliver_notifications, Val);
-set_xoption(Host, [{<<"pubsub#notify_config">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(notify_config, Val);
-set_xoption(Host, [{<<"pubsub#notify_delete">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(notify_delete, Val);
-set_xoption(Host, [{<<"pubsub#notify_retract">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(notify_retract, Val);
-set_xoption(Host, [{<<"pubsub#persist_items">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(persist_items, Val);
-set_xoption(Host, [{<<"pubsub#max_items">>, [Val]} | Opts], NewOpts) ->
- MaxItems = get_max_items_node(Host),
- ?SET_INTEGER_XOPT(max_items, Val, 0, MaxItems);
-set_xoption(Host, [{<<"pubsub#subscribe">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(subscribe, Val);
-set_xoption(Host, [{<<"pubsub#access_model">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(access_model, Val, [open, authorize, presence, roster, whitelist]);
-set_xoption(Host, [{<<"pubsub#publish_model">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(publish_model, Val, [publishers, subscribers, open]);
-set_xoption(Host, [{<<"pubsub#notification_type">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(notification_type, Val, [headline, normal]);
-set_xoption(Host, [{<<"pubsub#node_type">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(node_type, Val, [leaf, collection]);
-set_xoption(Host, [{<<"pubsub#max_payload_size">>, [Val]} | Opts], NewOpts) ->
- ?SET_INTEGER_XOPT(max_payload_size, Val, 0, (?MAX_PAYLOAD_SIZE));
-set_xoption(Host, [{<<"pubsub#send_last_published_item">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(send_last_published_item, Val, [never, on_sub, on_sub_and_presence]);
-set_xoption(Host, [{<<"pubsub#presence_based_delivery">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(presence_based_delivery, Val);
-set_xoption(Host, [{<<"pubsub#purge_offline">>, [Val]} | Opts], NewOpts) ->
- ?SET_BOOL_XOPT(purge_offline, Val);
-set_xoption(Host, [{<<"pubsub#title">>, Value} | Opts], NewOpts) ->
- ?SET_STRING_XOPT(title, Value);
-set_xoption(Host, [{<<"pubsub#type">>, Value} | Opts], NewOpts) ->
- ?SET_STRING_XOPT(type, Value);
-set_xoption(Host, [{<<"pubsub#body_xslt">>, Value} | Opts], NewOpts) ->
- ?SET_STRING_XOPT(body_xslt, Value);
-set_xoption(Host, [{<<"pubsub#collection">>, Value} | Opts], NewOpts) ->
- % NewValue = [string_to_node(V) || V <- Value],
- ?SET_LIST_XOPT(collection, Value);
-set_xoption(Host, [{<<"pubsub#node">>, [Value]} | Opts], NewOpts) ->
- % NewValue = string_to_node(Value),
- ?SET_LIST_XOPT(node, Value);
-set_xoption(Host, [{<<"pubsub#itemreply">>, [Val]} | Opts], NewOpts) ->
- ?SET_ALIST_XOPT(itemreply, Val, [none, owner, publisher]);
-set_xoption(Host, [_ | Opts], NewOpts) ->
- set_xoption(Host, Opts, NewOpts).
+-spec decode_publish_options(undefined | xdata(), binary()) ->
+ pubsub_publish_options:result() |
+ {error, stanza_error()}.
+decode_publish_options(undefined, _) ->
+ [];
+decode_publish_options(#xdata{fields = Fs}, Lang) ->
+ try pubsub_publish_options:decode(Fs)
+ catch _:{pubsub_publish_options, Why} ->
+ Txt = pubsub_publish_options:format_error(Why),
+ {error, xmpp:err_resource_constraint(Txt, Lang)}
+ end.
+
+-spec decode_get_pending(xdata(), binary()) ->
+ pubsub_get_pending:result() |
+ {error, stanza_error()}.
+decode_get_pending(#xdata{fields = Fs}, Lang) ->
+ try pubsub_get_pending:decode(Fs)
+ catch _:{pubsub_get_pending, Why} ->
+ Txt = pubsub_get_pending:format_error(Why),
+ {error, xmpp:err_resource_constraint(Txt, Lang)}
+ end.
+
+-spec check_opt_range(atom(), [proplists:property()], non_neg_integer()) -> boolean().
+check_opt_range(_Opt, _Opts, undefined) ->
+ true;
+check_opt_range(Opt, Opts, Max) ->
+ Val = proplists:get_value(Opt, Opts, Max),
+ Val =< Max.
+-spec get_max_items_node(host()) -> undefined | non_neg_integer().
get_max_items_node(Host) ->
config(Host, max_items_node, undefined).
+-spec get_max_subscriptions_node(host()) -> undefined | non_neg_integer().
get_max_subscriptions_node(Host) ->
config(Host, max_subscriptions_node, undefined).
%%%% last item cache handling
-
+-spec is_last_item_cache_enabled(host()) -> boolean().
is_last_item_cache_enabled(Host) ->
config(Host, last_item_cache, false).
+-spec set_cached_item(host(), nodeIdx(), binary(), jid(), [xmlel()]) -> ok.
set_cached_item({_, ServerHost, _}, Nidx, ItemId, Publisher, Payload) ->
set_cached_item(ServerHost, Nidx, ItemId, Publisher, Payload);
set_cached_item(Host, Nidx, ItemId, Publisher, Payload) ->
case is_last_item_cache_enabled(Host) of
- true -> mnesia:dirty_write({pubsub_last_item, Nidx, ItemId,
- {p1_time_compat:timestamp(), jid:tolower(jid:remove_resource(Publisher))},
- Payload});
- _ -> ok
+ true ->
+ Stamp = {erlang:timestamp(), jid:tolower(jid:remove_resource(Publisher))},
+ Item = #pubsub_last_item{nodeid = {Host, Nidx},
+ itemid = ItemId,
+ creation = Stamp,
+ payload = Payload},
+ mnesia:dirty_write(Item);
+ _ ->
+ ok
end.
+-spec unset_cached_item(host(), nodeIdx()) -> ok.
unset_cached_item({_, ServerHost, _}, Nidx) ->
unset_cached_item(ServerHost, Nidx);
unset_cached_item(Host, Nidx) ->
case is_last_item_cache_enabled(Host) of
- true -> mnesia:dirty_delete({pubsub_last_item, Nidx});
+ true -> mnesia:dirty_delete({pubsub_last_item, {Host, Nidx}});
_ -> ok
end.
--spec get_cached_item(Host :: mod_pubsub:host(), Nidx :: mod_pubsub:nodeIdx()) ->
- undefined | mod_pubsub:pubsubItem().
-
+-spec get_cached_item(host(), nodeIdx()) -> undefined | #pubsub_item{}.
get_cached_item({_, ServerHost, _}, Nidx) ->
get_cached_item(ServerHost, Nidx);
get_cached_item(Host, Nidx) ->
case is_last_item_cache_enabled(Host) of
true ->
- case mnesia:dirty_read({pubsub_last_item, Nidx}) of
+ case mnesia:dirty_read({pubsub_last_item, {Host, Nidx}}) of
[#pubsub_last_item{itemid = ItemId, creation = Creation, payload = Payload}] ->
- % [{pubsub_last_item, Nidx, ItemId, Creation,
- % Payload}] ->
#pubsub_item{itemid = {ItemId, Nidx},
payload = Payload, creation = Creation,
modification = Creation};
@@ -3892,37 +3593,34 @@ get_cached_item(Host, Nidx) ->
end.
%%%% plugin handling
-
+-spec host(binary()) -> binary().
host(ServerHost) ->
config(ServerHost, host, <<"pubsub.", ServerHost/binary>>).
+-spec serverhost(host()) -> binary().
serverhost({_U, ServerHost, _R})->
serverhost(ServerHost);
serverhost(Host) ->
ejabberd_router:host_of_route(Host).
+-spec tree(host()) -> atom().
tree(Host) ->
case config(Host, nodetree) of
undefined -> tree(Host, ?STDTREE);
Tree -> Tree
end.
+-spec tree(host() | atom(), binary()) -> atom().
tree(_Host, <<"virtual">>) ->
nodetree_virtual; % special case, virtual does not use any backend
tree(Host, Name) ->
- case gen_mod:db_type(serverhost(Host), ?MODULE) of
- mnesia -> jlib:binary_to_atom(<<"nodetree_", Name/binary>>);
- sql -> jlib:binary_to_atom(<<"nodetree_", Name/binary, "_sql">>);
- _ -> Name
- end.
+ submodule(Host, <<"nodetree">>, Name).
+-spec plugin(host() | atom(), binary()) -> atom().
plugin(Host, Name) ->
- case gen_mod:db_type(serverhost(Host), ?MODULE) of
- mnesia -> jlib:binary_to_atom(<<"node_", Name/binary>>);
- sql -> jlib:binary_to_atom(<<"node_", Name/binary, "_sql">>);
- _ -> Name
- end.
+ submodule(Host, <<"node">>, Name).
+-spec plugins(host()) -> [binary()].
plugins(Host) ->
case config(Host, plugins) of
undefined -> [?STDNODE];
@@ -3930,16 +3628,25 @@ plugins(Host) ->
Plugins -> Plugins
end.
+-spec subscription_plugin(host() | atom()) -> atom().
subscription_plugin(Host) ->
- case gen_mod:db_type(serverhost(Host), ?MODULE) of
- mnesia -> pubsub_subscription;
- sql -> pubsub_subscription_sql;
- _ -> none
- end.
+ submodule(Host, <<"pubsub">>, <<"subscription">>).
+-spec submodule(host() | atom(), binary(), binary()) -> atom().
+submodule(Db, Type, Name) when is_atom(Db) ->
+ case Db of
+ mnesia -> ejabberd:module_name([<<"pubsub">>, Type, Name]);
+ _ -> ejabberd:module_name([<<"pubsub">>, Type, Name, misc:atom_to_binary(Db)])
+ end;
+submodule(Host, Type, Name) ->
+ Db = mod_pubsub_opt:db_type(serverhost(Host)),
+ submodule(Db, Type, Name).
+
+-spec config(binary(), any()) -> any().
config(ServerHost, Key) ->
config(ServerHost, Key, undefined).
+-spec config(host(), any(), any()) -> any().
config({_User, Host, _Resource}, Key, Default) ->
config(Host, Key, Default);
config(ServerHost, Key, Default) ->
@@ -3948,52 +3655,57 @@ config(ServerHost, Key, Default) ->
_ -> Default
end.
-select_type(ServerHost, Host, Node, Type) ->
- SelectedType = case Host of
- {_User, _Server, _Resource} ->
- case config(ServerHost, pep_mapping) of
- undefined -> ?PEPNODE;
- Mapping -> proplists:get_value(Node, Mapping, ?PEPNODE)
- end;
- _ ->
- Type
- end,
- ConfiguredTypes = plugins(Host),
- case lists:member(SelectedType, ConfiguredTypes) of
- true -> SelectedType;
- false -> hd(ConfiguredTypes)
+-spec select_type(binary(), host(), binary(), binary()) -> binary().
+select_type(ServerHost, {_User, _Server, _Resource}, Node, _Type) ->
+ case config(ServerHost, pep_mapping) of
+ undefined -> ?PEPNODE;
+ Mapping -> proplists:get_value(Node, Mapping, ?PEPNODE)
+ end;
+select_type(ServerHost, _Host, _Node, Type) ->
+ case config(ServerHost, plugins) of
+ undefined ->
+ Type;
+ Plugins ->
+ case lists:member(Type, Plugins) of
+ true -> Type;
+ false -> hd(Plugins)
+ end
end.
+-spec select_type(binary(), host(), binary()) -> binary().
select_type(ServerHost, Host, Node) ->
select_type(ServerHost, Host, Node, hd(plugins(Host))).
+-spec feature(binary()) -> binary().
feature(<<"rsm">>) -> ?NS_RSM;
feature(Feature) -> <<(?NS_PUBSUB)/binary, "#", Feature/binary>>.
+-spec features() -> [binary()].
features() ->
[% see plugin "access-authorize", % OPTIONAL
- <<"access-open">>, % OPTIONAL this relates to access_model option in node_hometree
- <<"access-presence">>, % OPTIONAL this relates to access_model option in node_pep
- <<"access-whitelist">>, % OPTIONAL
- <<"collections">>, % RECOMMENDED
- <<"config-node">>, % RECOMMENDED
- <<"create-and-configure">>, % RECOMMENDED
- <<"item-ids">>, % RECOMMENDED
- <<"last-published">>, % RECOMMENDED
- <<"member-affiliation">>, % RECOMMENDED
- <<"presence-notifications">>, % OPTIONAL
- <<"presence-subscribe">>, % RECOMMENDED
- <<"publisher-affiliation">>, % RECOMMENDED
- <<"publish-only-affiliation">>, % OPTIONAL
- <<"retrieve-default">>,
- <<"shim">>]. % RECOMMENDED
+ <<"access-open">>, % OPTIONAL this relates to access_model option in node_hometree
+ <<"access-presence">>, % OPTIONAL this relates to access_model option in node_pep
+ <<"access-whitelist">>, % OPTIONAL
+ <<"collections">>, % RECOMMENDED
+ <<"config-node">>, % RECOMMENDED
+ <<"create-and-configure">>, % RECOMMENDED
+ <<"item-ids">>, % RECOMMENDED
+ <<"last-published">>, % RECOMMENDED
+ <<"member-affiliation">>, % RECOMMENDED
+ <<"presence-notifications">>, % OPTIONAL
+ <<"presence-subscribe">>, % RECOMMENDED
+ <<"publisher-affiliation">>, % RECOMMENDED
+ <<"publish-only-affiliation">>, % OPTIONAL
+ <<"publish-options">>, % OPTIONAL
+ <<"retrieve-default">>,
+ <<"shim">>]. % RECOMMENDED
% see plugin "retrieve-items", % RECOMMENDED
% see plugin "retrieve-subscriptions", % RECOMMENDED
% see plugin "subscribe", % REQUIRED
% see plugin "subscription-options", % OPTIONAL
% see plugin "subscription-notifications" % OPTIONAL
-
+-spec plugin_features(host(), binary()) -> [binary()].
plugin_features(Host, Type) ->
Module = plugin(Host, Type),
case catch Module:features() of
@@ -4001,6 +3713,7 @@ plugin_features(Host, Type) ->
Result -> Result
end.
+-spec features(binary(), binary()) -> [binary()].
features(Host, <<>>) ->
lists:usort(lists:foldl(fun (Plugin, Acc) ->
Acc ++ plugin_features(Host, Plugin)
@@ -4016,180 +3729,282 @@ features(Host, Node) when is_binary(Node) ->
end.
%% @doc <p>node tree plugin call.</p>
+-spec tree_call(host(), atom(), list()) -> {error, stanza_error() | {virtual, nodeIdx()}} | any().
tree_call({_User, Server, _Resource}, Function, Args) ->
tree_call(Server, Function, Args);
tree_call(Host, Function, Args) ->
Tree = tree(Host),
- ?DEBUG("tree_call apply(~s, ~s, ~p) @ ~s", [Tree, Function, Args, Host]),
- catch apply(Tree, Function, Args).
+ ?DEBUG("Tree_call apply(~ts, ~ts, ~p) @ ~ts", [Tree, Function, Args, Host]),
+ case apply(Tree, Function, Args) of
+ {error, #stanza_error{}} = Err ->
+ Err;
+ {error, {virtual, _}} = Err ->
+ Err;
+ {error, _} ->
+ ErrTxt = ?T("Database failure"),
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_internal_server_error(ErrTxt, Lang)};
+ Other ->
+ Other
+ end.
+-spec tree_action(host(), atom(), list()) -> {error, stanza_error() | {virtual, nodeIdx()}} | any().
tree_action(Host, Function, Args) ->
- ?DEBUG("tree_action ~p ~p ~p", [Host, Function, Args]),
+ ?DEBUG("Tree_action ~p ~p ~p", [Host, Function, Args]),
ServerHost = serverhost(Host),
- Fun = fun () -> tree_call(Host, Function, Args) end,
- case gen_mod:db_type(ServerHost, ?MODULE) of
- mnesia ->
- catch mnesia:sync_dirty(Fun);
- sql ->
- case catch ejabberd_sql:sql_bloc(ServerHost, Fun) of
- {atomic, Result} ->
- Result;
- {aborted, Reason} ->
- ?ERROR_MSG("transaction return internal error: ~p~n", [{aborted, Reason}]),
- {error, ?ERR_INTERNAL_SERVER_ERROR}
- end;
- Other ->
- ?ERROR_MSG("unsupported backend: ~p~n", [Other]),
- {error, ?ERR_INTERNAL_SERVER_ERROR}
- end.
+ DBType = mod_pubsub_opt:db_type(ServerHost),
+ Fun = fun () ->
+ try tree_call(Host, Function, Args)
+ catch ?EX_RULE(Class, Reason, St) when DBType == sql ->
+ StackTrace = ?EX_STACK(St),
+ ejabberd_sql:abort({exception, Class, Reason, StackTrace})
+ end
+ end,
+ Ret = case DBType of
+ mnesia ->
+ mnesia:sync_dirty(Fun);
+ sql ->
+ ejabberd_sql:sql_bloc(ServerHost, Fun);
+ _ ->
+ Fun()
+ end,
+ get_tree_action_result(Ret).
+
+-spec get_tree_action_result(any()) -> {error, stanza_error() | {virtual, nodeIdx()}} | any().
+get_tree_action_result({atomic, Result}) ->
+ Result;
+get_tree_action_result({aborted, {exception, Class, Reason, StackTrace}}) ->
+ ?ERROR_MSG("Transaction aborted:~n** ~ts",
+ [misc:format_exception(2, Class, Reason, StackTrace)]),
+ get_tree_action_result({error, db_failure});
+get_tree_action_result({aborted, Reason}) ->
+ ?ERROR_MSG("Transaction aborted: ~p~n", [Reason]),
+ get_tree_action_result({error, db_failure});
+get_tree_action_result({error, #stanza_error{}} = Err) ->
+ Err;
+get_tree_action_result({error, {virtual, _}} = Err) ->
+ Err;
+get_tree_action_result({error, _}) ->
+ ErrTxt = ?T("Database failure"),
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_internal_server_error(ErrTxt, Lang)};
+get_tree_action_result(Other) ->
+ %% This is very risky, but tree plugins design is really bad
+ Other.
%% @doc <p>node plugin call.</p>
+-spec node_call(host(), binary(), atom(), list()) -> {result, any()} | {error, stanza_error()}.
node_call(Host, Type, Function, Args) ->
- ?DEBUG("node_call ~p ~p ~p", [Type, Function, Args]),
+ ?DEBUG("Node_call ~p ~p ~p", [Type, Function, Args]),
Module = plugin(Host, Type),
- case apply(Module, Function, Args) of
- {result, Result} ->
- {result, Result};
- {error, Error} ->
- {error, Error};
- {'EXIT', {undef, Undefined}} ->
- case Type of
- ?STDNODE -> {error, {undef, Undefined}};
- _ -> node_call(Host, ?STDNODE, Function, Args)
+ case erlang:function_exported(Module, Function, length(Args)) of
+ true ->
+ case apply(Module, Function, Args) of
+ {result, Result} ->
+ {result, Result};
+ {error, #stanza_error{}} = Err ->
+ Err;
+ {error, _} ->
+ ErrTxt = ?T("Database failure"),
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_internal_server_error(ErrTxt, Lang)}
end;
- {'EXIT', Reason} ->
- {error, Reason};
- Result ->
- {result, Result} %% any other return value is forced as result
+ false when Type /= ?STDNODE ->
+ node_call(Host, ?STDNODE, Function, Args);
+ false ->
+ %% Let it crash with the stacktrace
+ apply(Module, Function, Args)
end.
+-spec node_action(host(), binary(), atom(), list()) -> {result, any()} | {error, stanza_error()}.
node_action(Host, Type, Function, Args) ->
- ?DEBUG("node_action ~p ~p ~p ~p", [Host, Type, Function, Args]),
- transaction(Host, fun () ->
- node_call(Host, Type, Function, Args)
- end,
- sync_dirty).
+ ?DEBUG("Node_action ~p ~p ~p ~p", [Host, Type, Function, Args]),
+ transaction(Host, fun() -> node_call(Host, Type, Function, Args) end, sync_dirty).
%% @doc <p>plugin transaction handling.</p>
+-spec transaction(host(), binary(), fun((#pubsub_node{}) -> _), transaction | sync_dirty) ->
+ {result, any()} | {error, stanza_error()}.
transaction(Host, Node, Action, Trans) ->
- transaction(Host, fun () ->
- case tree_call(Host, get_node, [Host, Node]) of
- N when is_record(N, pubsub_node) ->
- case Action(N) of
- {result, Result} -> {result, {N, Result}};
- {atomic, {result, Result}} -> {result, {N, Result}};
- Other -> Other
- end;
- Error ->
- Error
- end
- end,
- Trans).
+ transaction(
+ Host,
+ fun() ->
+ case tree_call(Host, get_node, [Host, Node]) of
+ N when is_record(N, pubsub_node) ->
+ case Action(N) of
+ {result, Result} -> {result, {N, Result}};
+ {atomic, {result, Result}} -> {result, {N, Result}};
+ Other -> Other
+ end;
+ Error ->
+ Error
+ end
+ end,
+ Trans).
+-spec transaction(host(), fun(), transaction | sync_dirty) ->
+ {result, any()} | {error, stanza_error()}.
transaction(Host, Fun, Trans) ->
ServerHost = serverhost(Host),
- DBType = gen_mod:db_type(ServerHost, ?MODULE),
- Retry = case DBType of
- sql -> 2;
- _ -> 1
- end,
- transaction_retry(Host, ServerHost, Fun, Trans, DBType, Retry).
-
-transaction_retry(_Host, _ServerHost, _Fun, _Trans, _DBType, 0) ->
- {error, ?ERR_INTERNAL_SERVER_ERROR};
-transaction_retry(Host, ServerHost, Fun, Trans, DBType, Count) ->
+ DBType = mod_pubsub_opt:db_type(ServerHost),
+ do_transaction(ServerHost, Fun, Trans, DBType).
+
+-spec do_transaction(binary(), fun(), transaction | sync_dirty, atom()) ->
+ {result, any()} | {error, stanza_error()}.
+do_transaction(ServerHost, Fun, Trans, DBType) ->
+ F = fun() ->
+ try Fun()
+ catch ?EX_RULE(Class, Reason, St) when (DBType == mnesia andalso
+ Trans == transaction) orelse
+ DBType == sql ->
+ StackTrace = ?EX_STACK(St),
+ Ex = {exception, Class, Reason, StackTrace},
+ case DBType of
+ mnesia -> mnesia:abort(Ex);
+ sql -> ejabberd_sql:abort(Ex)
+ end
+ end
+ end,
Res = case DBType of
- mnesia ->
- catch mnesia:Trans(Fun);
- sql ->
- SqlFun = case Trans of
- transaction -> sql_transaction;
- _ -> sql_bloc
- end,
- catch ejabberd_sql:SqlFun(ServerHost, Fun);
- _ ->
- {unsupported, DBType}
- end,
- case Res of
- {result, Result} ->
- {result, Result};
- {error, Error} ->
- {error, Error};
- {atomic, {result, Result}} ->
- {result, Result};
- {atomic, {error, Error}} ->
- {error, Error};
- {aborted, Reason} ->
- ?ERROR_MSG("transaction return internal error: ~p~n", [{aborted, Reason}]),
- {error, ?ERR_INTERNAL_SERVER_ERROR};
- {'EXIT', {timeout, _} = Reason} ->
- ?ERROR_MSG("transaction return internal error: ~p~n", [Reason]),
- transaction_retry(Host, ServerHost, Fun, Trans, DBType, Count - 1);
- {'EXIT', Reason} ->
- ?ERROR_MSG("transaction return internal error: ~p~n", [{'EXIT', Reason}]),
- {error, ?ERR_INTERNAL_SERVER_ERROR};
- Other ->
- ?ERROR_MSG("transaction return internal error: ~p~n", [Other]),
- {error, ?ERR_INTERNAL_SERVER_ERROR}
- end.
+ mnesia ->
+ mnesia:Trans(F);
+ sql ->
+ SqlFun = case Trans of
+ transaction -> sql_transaction;
+ _ -> sql_bloc
+ end,
+ ejabberd_sql:SqlFun(ServerHost, F);
+ _ ->
+ F()
+ end,
+ get_transaction_response(Res).
+
+-spec get_transaction_response(any()) -> {result, any()} | {error, stanza_error()}.
+get_transaction_response({result, _} = Result) ->
+ Result;
+get_transaction_response({error, #stanza_error{}} = Err) ->
+ Err;
+get_transaction_response({atomic, Result}) ->
+ get_transaction_response(Result);
+get_transaction_response({aborted, Err}) ->
+ get_transaction_response(Err);
+get_transaction_response({error, _}) ->
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_internal_server_error(?T("Database failure"), Lang)};
+get_transaction_response({exception, Class, Reason, StackTrace}) ->
+ ?ERROR_MSG("Transaction aborted:~n** ~ts",
+ [misc:format_exception(2, Class, Reason, StackTrace)]),
+ get_transaction_response({error, db_failure});
+get_transaction_response(Err) ->
+ ?ERROR_MSG("Transaction error: ~p", [Err]),
+ get_transaction_response({error, db_failure}).
%%%% helpers
%% Add pubsub-specific error element
-extended_error(Error, Ext) ->
- extended_error(Error, Ext, [{<<"xmlns">>, ?NS_PUBSUB_ERRORS}]).
-
-extended_error(Error, unsupported, Feature) ->
- %% Give a uniq identifier
- extended_error(Error, <<"unsupported">>,
- [{<<"xmlns">>, ?NS_PUBSUB_ERRORS},
- {<<"feature">>, Feature}]);
-extended_error(#xmlel{name = Error, attrs = Attrs, children = SubEls}, Ext, ExtAttrs) ->
- #xmlel{name = Error, attrs = Attrs,
- children = lists:reverse([#xmlel{name = Ext, attrs = ExtAttrs} | SubEls])}.
-
-is_item_not_found({error, ErrEl}) ->
- case fxml:get_subtag_with_xmlns(
- ErrEl, <<"item-not-found">>, ?NS_STANZAS) of
- #xmlel{} -> true;
- _ -> false
- end.
+-spec extended_error(stanza_error(), ps_error()) -> stanza_error().
+extended_error(StanzaErr, PubSubErr) ->
+ StanzaErr#stanza_error{sub_els = [PubSubErr]}.
-string_to_ljid(JID) ->
- case jid:from_string(JID) of
- error ->
- {<<>>, <<>>, <<>>};
- J ->
- case jid:tolower(J) of
- error -> {<<>>, <<>>, <<>>};
- J1 -> J1
- end
- end.
+-spec err_closed_node() -> ps_error().
+err_closed_node() ->
+ #ps_error{type = 'closed-node'}.
--spec uniqid() -> mod_pubsub:itemId().
-uniqid() ->
- {T1, T2, T3} = p1_time_compat:timestamp(),
- iolist_to_binary(io_lib:fwrite("~.16B~.16B~.16B", [T1, T2, T3])).
+-spec err_configuration_required() -> ps_error().
+err_configuration_required() ->
+ #ps_error{type = 'configuration-required'}.
+
+-spec err_invalid_jid() -> ps_error().
+err_invalid_jid() ->
+ #ps_error{type = 'invalid-jid'}.
+
+-spec err_invalid_options() -> ps_error().
+err_invalid_options() ->
+ #ps_error{type = 'invalid-options'}.
-nodeAttr(Node) -> [{<<"node">>, Node}].
+-spec err_invalid_payload() -> ps_error().
+err_invalid_payload() ->
+ #ps_error{type = 'invalid-payload'}.
-itemAttr([]) -> [];
-itemAttr(ItemId) -> [{<<"id">>, ItemId}].
-itemAttr(ItemId, From) -> [{<<"id">>, ItemId}, From].
+-spec err_invalid_subid() -> ps_error().
+err_invalid_subid() ->
+ #ps_error{type = 'invalid-subid'}.
-itemsEls(Items) ->
- [#xmlel{name = <<"item">>, attrs = itemAttr(ItemId), children = Payload}
- || #pubsub_item{itemid = {ItemId, _}, payload = Payload} <- Items].
+-spec err_item_forbidden() -> ps_error().
+err_item_forbidden() ->
+ #ps_error{type = 'item-forbidden'}.
--spec add_message_type(Message :: xmlel(), Type :: atom()) -> xmlel().
+-spec err_item_required() -> ps_error().
+err_item_required() ->
+ #ps_error{type = 'item-required'}.
-add_message_type(Message, normal) -> Message;
-add_message_type(#xmlel{name = <<"message">>, attrs = Attrs, children = Els}, Type) ->
- #xmlel{name = <<"message">>,
- attrs = [{<<"type">>, jlib:atom_to_binary(Type)} | Attrs],
- children = Els};
-add_message_type(XmlEl, _Type) ->
- XmlEl.
+-spec err_jid_required() -> ps_error().
+err_jid_required() ->
+ #ps_error{type = 'jid-required'}.
+
+-spec err_max_items_exceeded() -> ps_error().
+err_max_items_exceeded() ->
+ #ps_error{type = 'max-items-exceeded'}.
+
+-spec err_max_nodes_exceeded() -> ps_error().
+err_max_nodes_exceeded() ->
+ #ps_error{type = 'max-nodes-exceeded'}.
+
+-spec err_nodeid_required() -> ps_error().
+err_nodeid_required() ->
+ #ps_error{type = 'nodeid-required'}.
+
+-spec err_not_in_roster_group() -> ps_error().
+err_not_in_roster_group() ->
+ #ps_error{type = 'not-in-roster-group'}.
+
+-spec err_not_subscribed() -> ps_error().
+err_not_subscribed() ->
+ #ps_error{type = 'not-subscribed'}.
+
+-spec err_payload_too_big() -> ps_error().
+err_payload_too_big() ->
+ #ps_error{type = 'payload-too-big'}.
+
+-spec err_payload_required() -> ps_error().
+err_payload_required() ->
+ #ps_error{type = 'payload-required'}.
+
+-spec err_pending_subscription() -> ps_error().
+err_pending_subscription() ->
+ #ps_error{type = 'pending-subscription'}.
+
+-spec err_precondition_not_met() -> ps_error().
+err_precondition_not_met() ->
+ #ps_error{type = 'precondition-not-met'}.
+
+-spec err_presence_subscription_required() -> ps_error().
+err_presence_subscription_required() ->
+ #ps_error{type = 'presence-subscription-required'}.
+
+-spec err_subid_required() -> ps_error().
+err_subid_required() ->
+ #ps_error{type = 'subid-required'}.
+
+-spec err_too_many_subscriptions() -> ps_error().
+err_too_many_subscriptions() ->
+ #ps_error{type = 'too-many-subscriptions'}.
+
+-spec err_unsupported(ps_feature()) -> ps_error().
+err_unsupported(Feature) ->
+ #ps_error{type = 'unsupported', feature = Feature}.
+
+-spec err_unsupported_access_model() -> ps_error().
+err_unsupported_access_model() ->
+ #ps_error{type = 'unsupported-access-model'}.
+
+-spec uniqid() -> mod_pubsub:itemId().
+uniqid() ->
+ {T1, T2, T3} = erlang:timestamp(),
+ (str:format("~.16B~.16B~.16B", [T1, T2, T3])).
+
+-spec add_message_type(message(), message_type()) -> message().
+add_message_type(#message{} = Message, Type) ->
+ Message#message{type = Type}.
%% Place of <headers/> changed at the bottom of the stanza
%% cf. http://xmpp.org/extensions/xep-0060.html#publisher-publish-success-subid
@@ -4197,63 +4012,53 @@ add_message_type(XmlEl, _Type) ->
%% "[SHIM Headers] SHOULD be included after the event notification information
%% (i.e., as the last child of the <message/> stanza)".
-add_shim_headers(Stanza, HeaderEls) ->
- add_headers(Stanza, <<"headers">>, ?NS_SHIM, HeaderEls).
-
-add_extended_headers(Stanza, HeaderEls) ->
- add_headers(Stanza, <<"addresses">>, ?NS_ADDRESS, HeaderEls).
+-spec add_shim_headers(stanza(), [{binary(), binary()}]) -> stanza().
+add_shim_headers(Stanza, Headers) ->
+ xmpp:set_subtag(Stanza, #shim{headers = Headers}).
-add_headers(#xmlel{name = Name, attrs = Attrs, children = Els}, HeaderName, HeaderNS, HeaderEls) ->
- HeaderEl = #xmlel{name = HeaderName,
- attrs = [{<<"xmlns">>, HeaderNS}],
- children = HeaderEls},
- #xmlel{name = Name, attrs = Attrs,
- children = lists:append(Els, [HeaderEl])}.
+-spec add_extended_headers(stanza(), [address()]) -> stanza().
+add_extended_headers(Stanza, Addrs) ->
+ xmpp:set_subtag(Stanza, #addresses{list = Addrs}).
+-spec subid_shim([binary()]) -> [{binary(), binary()}].
subid_shim(SubIds) ->
- [#xmlel{name = <<"header">>,
- attrs = [{<<"name">>, <<"SubId">>}],
- children = [{xmlcdata, SubId}]}
- || SubId <- SubIds].
+ [{<<"SubId">>, SubId} || SubId <- SubIds].
%% The argument is a list of Jids because this function could be used
%% with the 'pubsub#replyto' (type=jid-multi) node configuration.
+-spec extended_headers([jid()]) -> [address()].
extended_headers(Jids) ->
- [#xmlel{name = <<"address">>,
- attrs = [{<<"type">>, <<"replyto">>}, {<<"jid">>, Jid}]}
- || Jid <- Jids].
-
-on_user_offline(_, JID, _) ->
- {User, Server, Resource} = jid:tolower(JID),
- case user_resources(User, Server) of
- [] -> purge_offline({User, Server, Resource});
- _ -> true
- end.
+ [#address{type = replyto, jid = Jid} || Jid <- Jids].
+-spec purge_offline(ljid()) -> ok.
purge_offline(LJID) ->
Host = host(element(2, LJID)),
Plugins = plugins(Host),
- Result = lists:foldl(fun (Type, {Status, Acc}) ->
- Features = plugin_features(Host, Type),
- case lists:member(<<"retrieve-affiliations">>, plugin_features(Host, Type)) of
- false ->
- {{error, extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
- unsupported, <<"retrieve-affiliations">>)},
+ Result = lists:foldl(
+ fun(Type, {Status, Acc}) ->
+ Features = plugin_features(Host, Type),
+ case lists:member(<<"retrieve-affiliations">>, plugin_features(Host, Type)) of
+ false ->
+ {{error, extended_error(xmpp:err_feature_not_implemented(),
+ err_unsupported('retrieve-affiliations'))},
Acc};
- true ->
- Items = lists:member(<<"retract-items">>, Features)
- andalso lists:member(<<"persistent-items">>, Features),
- if Items ->
- {result, Affs} = node_action(Host, Type,
- get_entity_affiliations, [Host, LJID]),
- {Status, [Affs | Acc]};
- true ->
- {Status, Acc}
- end
- end
- end,
- {ok, []}, Plugins),
+ true ->
+ Items = lists:member(<<"retract-items">>, Features)
+ andalso lists:member(<<"persistent-items">>, Features),
+ if Items ->
+ case node_action(Host, Type,
+ get_entity_affiliations, [Host, LJID]) of
+ {result, Affs} ->
+ {Status, [Affs | Acc]};
+ {error, _} = Err ->
+ {Err, Acc}
+ end;
+ true ->
+ {Status, Acc}
+ end
+ end
+ end, {ok, []}, Plugins),
case Result of
{ok, Affs} ->
lists:foreach(
@@ -4269,67 +4074,109 @@ purge_offline(LJID) ->
ok
end
end, lists:usort(lists:flatten(Affs)));
- {Error, _} ->
- ?DEBUG("on_user_offline ~p", [Error])
+ _ ->
+ ok
end.
+-spec purge_offline(host(), ljid(), #pubsub_node{}) -> ok | {error, stanza_error()}.
purge_offline(Host, LJID, Node) ->
Nidx = Node#pubsub_node.id,
Type = Node#pubsub_node.type,
Options = Node#pubsub_node.options,
- case node_action(Host, Type, get_items, [Nidx, service_jid(Host), none]) of
+ case node_action(Host, Type, get_items, [Nidx, service_jid(Host), undefined]) of
{result, {[], _}} ->
ok;
{result, {Items, _}} ->
- {User, Server, _} = LJID,
+ {User, Server, Resource} = LJID,
PublishModel = get_option(Options, publish_model),
ForceNotify = get_option(Options, notify_retract),
{_, NodeId} = Node#pubsub_node.nodeid,
- lists:foreach(fun
- (#pubsub_item{itemid = {ItemId, _}, modification = {_, {U, S, _}}})
- when (U == User) and (S == Server) ->
- case node_action(Host, Type, delete_item, [Nidx, {U, S, <<>>}, PublishModel, ItemId]) of
- {result, {_, broadcast}} ->
- broadcast_retract_items(Host, NodeId, Nidx, Type, Options, [ItemId], ForceNotify),
- case get_cached_item(Host, Nidx) of
- #pubsub_item{itemid = {ItemId, Nidx}} -> unset_cached_item(Host, Nidx);
- _ -> ok
- end;
- {result, _} ->
- ok;
- Error ->
- Error
- end;
- (_) ->
- true
- end, Items);
- Error ->
- Error
+ lists:foreach(
+ fun(#pubsub_item{itemid = {ItemId, _}, modification = {_, {U, S, R}}})
+ when (U == User) and (S == Server) and (R == Resource) ->
+ case node_action(Host, Type, delete_item, [Nidx, {U, S, <<>>}, PublishModel, ItemId]) of
+ {result, {_, broadcast}} ->
+ broadcast_retract_items(Host, NodeId, Nidx, Type, Options, [ItemId], ForceNotify),
+ case get_cached_item(Host, Nidx) of
+ #pubsub_item{itemid = {ItemId, Nidx}} -> unset_cached_item(Host, Nidx);
+ _ -> ok
+ end;
+ _ ->
+ ok
+ end;
+ (_) ->
+ true
+ end, Items);
+ {error, #stanza_error{}} = Err ->
+ Err;
+ _ ->
+ Txt = ?T("Database failure"),
+ Lang = ejabberd_option:language(),
+ {error, xmpp:err_internal_server_error(Txt, Lang)}
end.
+-spec mod_opt_type(atom()) -> econf:validator().
mod_opt_type(access_createnode) ->
- fun (A) when is_atom(A) -> A end;
-mod_opt_type(db_type) -> fun(T) -> ejabberd_config:v_db(?MODULE, T) end;
-mod_opt_type(host) -> fun iolist_to_binary/1;
+ econf:acl();
+mod_opt_type(name) ->
+ econf:binary();
mod_opt_type(ignore_pep_from_offline) ->
- fun (A) when is_boolean(A) -> A end;
-mod_opt_type(iqdisc) -> fun gen_iq_handler:check_type/1;
+ econf:bool();
mod_opt_type(last_item_cache) ->
- fun (A) when is_boolean(A) -> A end;
+ econf:bool();
mod_opt_type(max_items_node) ->
- fun (A) when is_integer(A) andalso A >= 0 -> A end;
+ econf:non_neg_int();
+mod_opt_type(max_nodes_discoitems) ->
+ econf:non_neg_int(infinity);
mod_opt_type(max_subscriptions_node) ->
- fun (A) when is_integer(A) andalso A >= 0 -> A end;
+ econf:non_neg_int();
+mod_opt_type(force_node_config) ->
+ econf:map(
+ econf:glob(),
+ econf:map(
+ econf:atom(),
+ econf:either(
+ econf:int(),
+ econf:atom()),
+ [{return, orddict}, unique]));
mod_opt_type(default_node_config) ->
- fun (A) when is_list(A) -> A end;
+ econf:map(
+ econf:atom(),
+ econf:either(
+ econf:int(),
+ econf:atom()),
+ [unique]);
mod_opt_type(nodetree) ->
- fun (A) when is_binary(A) -> A end;
+ econf:binary();
mod_opt_type(pep_mapping) ->
- fun (A) when is_list(A) -> A end;
+ econf:map(econf:binary(), econf:binary());
mod_opt_type(plugins) ->
- fun (A) when is_list(A) -> A end;
-mod_opt_type(_) ->
- [access_createnode, db_type, host,
- ignore_pep_from_offline, iqdisc, last_item_cache,
- max_items_node, nodetree, pep_mapping, plugins,
- max_subscriptions_node, default_node_config].
+ econf:list(
+ econf:enum([<<"flat">>, <<"pep">>]),
+ [unique]);
+mod_opt_type(host) ->
+ econf:host();
+mod_opt_type(hosts) ->
+ econf:hosts();
+mod_opt_type(db_type) ->
+ econf:db_type(?MODULE);
+mod_opt_type(vcard) ->
+ econf:vcard_temp().
+
+mod_options(Host) ->
+ [{access_createnode, all},
+ {db_type, ejabberd_config:default_db(Host, ?MODULE)},
+ {host, <<"pubsub.", Host/binary>>},
+ {hosts, []},
+ {name, ?T("Publish-Subscribe")},
+ {vcard, undefined},
+ {ignore_pep_from_offline, true},
+ {last_item_cache, false},
+ {max_items_node, ?MAXITEMS},
+ {max_nodes_discoitems, 100},
+ {nodetree, ?STDTREE},
+ {pep_mapping, []},
+ {plugins, [?STDNODE]},
+ {max_subscriptions_node, undefined},
+ {default_node_config, []},
+ {force_node_config, []}].