25
1
mirror of https://github.com/processone/ejabberd.git synced 2024-12-02 16:37:52 +01:00
xmpp.chapril.org-ejabberd/src/mod_pubsub.erl
Evgeny Khramtsov 2168489ddf Don't retain module option on gen_mod supervisor
When module's options were updated (e.g. by reloading ejabberd.yml)
and, later, the module's process crashed, gen_mod supervisor
restarts the process with outdated options. This is now fixed.
2019-08-04 21:46:18 +03:00

4181 lines
146 KiB
Erlang

%%%----------------------------------------------------------------------
%%% File : mod_pubsub.erl
%%% Author : Christophe Romain <christophe.romain@process-one.net>
%%% Purpose : Publish Subscribe service (XEP-0060)
%%% Created : 1 Dec 2007 by Christophe Romain <christophe.romain@process-one.net>
%%%
%%%
%%% 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
%%% published by the Free Software Foundation; either version 2 of the
%%% License, or (at your option) any later version.
%%%
%%% This program is distributed in the hope that it will be useful,
%%% but WITHOUT ANY WARRANTY; without even the implied warranty of
%%% MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
%%% General Public License for more details.
%%%
%%% You should have received a copy of the GNU General Public License along
%%% with this program; if not, write to the Free Software Foundation, Inc.,
%%% 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
%%%
%%%----------------------------------------------------------------------
%%% Support for subscription-options and multi-subscribe features was
%%% added by Brian Cully (bjc AT kublai.com). Subscriptions and options are
%%% stored in the pubsub_subscription table, with a link to them provided
%%% by the subscriptions field of pubsub_state. For information on
%%% subscription-options and mulit-subscribe see XEP-0060 sections 6.1.6,
%%% 6.2.3.1, 6.2.3.5, and 6.3. For information on subscription leases see
%%% XEP-0060 section 12.18.
-module(mod_pubsub).
-behaviour(gen_mod).
-behaviour(gen_server).
-author('christophe.romain@process-one.net').
-protocol({xep, 60, '1.14'}).
-protocol({xep, 163, '1.2'}).
-protocol({xep, 248, '0.2'}).
-include("logger.hrl").
-include("xmpp.hrl").
-include("pubsub.hrl").
-include("mod_roster.hrl").
-include("translate.hrl").
-include("ejabberd_stacktrace.hrl").
-define(STDTREE, <<"tree">>).
-define(STDNODE, <<"flat">>).
-define(PEPNODE, <<"pep">>).
%% exports for hooks
-export([presence_probe/3, caps_add/3, caps_update/3,
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,
c2s_handle_info/2]).
%% exported iq handlers
-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, 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([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/2, stop/1, init/1,
handle_call/3, handle_cast/2, handle_info/2,
terminate/2, code_change/3, depends/2, mod_opt_type/1, mod_options/1]).
-export([route/1]).
%%====================================================================
%% API
%%====================================================================
%%--------------------------------------------------------------------
%% Function: start_link() -> {ok,Pid} | ignore | {error,Error}
%% Description: Starts the server
%%--------------------------------------------------------------------
-export_type([
host/0,
hostPubsub/0,
hostPEP/0,
%%
nodeIdx/0,
nodeId/0,
itemId/0,
subId/0,
payload/0,
%%
nodeOption/0,
nodeOptions/0,
subOption/0,
subOptions/0,
pubOption/0,
pubOptions/0,
%%
affiliation/0,
subscription/0,
accessModel/0,
publishModel/0
]).
%% -type payload() defined here because the -type xmlel() is not accessible
%% from pubsub.hrl
-type(payload() :: [] | [xmlel(),...]).
-export_type([
pubsubNode/0,
pubsubState/0,
pubsubItem/0,
pubsubSubscription/0,
pubsubLastItem/0
]).
-type(pubsubNode() ::
#pubsub_node{
nodeid :: {Host::mod_pubsub:host(), Node::mod_pubsub:nodeId()},
id :: Nidx::mod_pubsub:nodeIdx(),
parents :: [Node::mod_pubsub:nodeId()],
type :: Type::binary(),
owners :: [Owner::ljid(),...],
options :: Opts::mod_pubsub:nodeOptions()
}
).
-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()}]
}
).
-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()
}
).
-type(pubsubSubscription() ::
#pubsub_subscription{
subid :: SubId::mod_pubsub:subId(),
options :: [] | mod_pubsub:subOptions()
}
).
-type(pubsubLastItem() ::
#pubsub_last_item{
nodeid :: {binary(), mod_pubsub:nodeIdx()},
itemid :: mod_pubsub:itemId(),
creation :: {erlang:timestamp(), ljid()},
payload :: mod_pubsub:payload()
}
).
-record(state,
{
server_host,
hosts,
access,
pep_mapping = [],
ignore_pep_from_offline = true,
last_item_cache = false,
max_items_node = ?MAXITEMS,
max_subscriptions_node = undefined,
default_node_config = [],
nodetree = <<"nodetree_", (?STDTREE)/binary>>,
plugins = [?STDNODE],
db_type
}).
-type(state() ::
#state{
server_host :: binary(),
hosts :: [mod_pubsub:hostPubsub()],
access :: atom(),
pep_mapping :: [{binary(), binary()}],
ignore_pep_from_offline :: boolean(),
last_item_cache :: boolean(),
max_items_node :: non_neg_integer(),
max_subscriptions_node :: non_neg_integer()|undefined,
default_node_config :: [{atom(), binary()|boolean()|integer()|atom()}],
nodetree :: binary(),
plugins :: [binary(),...],
db_type :: atom()
}
).
-type subs_by_depth() :: [{integer(), [{#pubsub_node{}, [{ljid(), subId(), subOptions()}]}]}].
start(Host, Opts) ->
gen_mod:start_child(?MODULE, Host, Opts).
stop(Host) ->
gen_mod:stop_child(?MODULE, Host).
%%====================================================================
%% gen_server callbacks
%%====================================================================
%%--------------------------------------------------------------------
%% Function: init(Args) -> {ok, State} |
%% {ok, State, Timeout} |
%% ignore |
%% {stop, Reason}
%% Description: Initiates the server
%%--------------------------------------------------------------------
-spec init([binary() | [{_,_}],...]) -> {'ok',state()}.
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),
ejabberd_hooks:add(disco_local_features, ServerHost,
?MODULE, disco_local_features, 75),
ejabberd_hooks:add(disco_local_items, ServerHost,
?MODULE, disco_local_items, 75),
ejabberd_hooks:add(presence_probe_hook, ServerHost,
?MODULE, presence_probe, 80),
ejabberd_hooks:add(roster_in_subscription, ServerHost,
?MODULE, in_subscription, 50),
ejabberd_hooks:add(roster_out_subscription, ServerHost,
?MODULE, out_subscription, 50),
ejabberd_hooks:add(remove_user, ServerHost,
?MODULE, remove_user, 50),
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),
ejabberd_hooks:add(caps_update, ServerHost,
?MODULE, caps_update, 80),
ejabberd_hooks:add(disco_sm_identity, ServerHost,
?MODULE, disco_sm_identity, 75),
ejabberd_hooks:add(disco_sm_features, ServerHost,
?MODULE, disco_sm_features, 75),
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),
gen_iq_handler:add_iq_handler(ejabberd_sm, ServerHost,
?NS_PUBSUB_OWNER, ?MODULE, iq_sm);
false ->
ok
end,
NodeTree = config(ServerHost, nodetree),
Plugins = config(ServerHost, plugins),
PepMapping = config(ServerHost, pep_mapping),
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_hosts(Opts),
Plugins = mod_pubsub_opt:plugins(Opts),
Db = mod_pubsub_opt:db_type(Opts),
lists:flatmap(
fun(Name) ->
Plugin = plugin(Db, Name),
try apply(Plugin, depends, [Host, ServerHost, Opts])
catch _:undef -> []
end
end, Plugins).
%% @doc Call the init/1 function for each plugin declared in the config file.
%% 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>See {@link node_hometree:init/1} for an example implementation.</p>
init_plugins(Host, ServerHost, Opts) ->
TreePlugin = tree(Host, mod_pubsub_opt:nodetree(Opts)),
?DEBUG("** tree plugin is ~p", [TreePlugin]),
TreePlugin:init(Host, ServerHost, Opts),
Plugins = mod_pubsub_opt:plugins(Opts),
PepMapping = mod_pubsub_opt:pep_mapping(Opts),
?DEBUG("** PEP Mapping : ~p~n", [PepMapping]),
PluginsOK = lists:foldl(
fun (Name, Acc) ->
Plugin = plugin(Host, Name),
apply(Plugin, init, [Host, ServerHost, Opts]),
?DEBUG("** init ~s plugin", [Name]),
[Name | Acc]
end,
[], Plugins),
{lists:reverse(PluginsOK), TreePlugin, PepMapping}.
terminate_plugins(Host, ServerHost, Plugins, TreePlugin) ->
lists:foreach(
fun (Name) ->
?DEBUG("** terminate ~s plugin", [Name]),
Plugin = plugin(Host, Name),
Plugin:terminate(Host, ServerHost)
end,
Plugins),
TreePlugin:terminate(Host, ServerHost),
ok.
%% -------
%% disco hooks handling functions
%%
-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 ->
[#identity{category = <<"pubsub">>, type = <<"pep">>} | Acc];
false ->
Acc
end;
disco_local_identity(Acc, _From, _To, _Node, _Lang) ->
Acc.
-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 ++ [?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.
-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) ->
[#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, [#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({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)};
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,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Result}} -> Result;
_ -> []
end.
-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) ->
{result, lists:usort(OtherItems ++
disco_items(jid:tolower(jid:remove_resource(To)), Node, From))};
disco_sm_items(Acc, _From, _To, _Node, _Lang) -> Acc.
-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, _} ->
[#disco_item{node = Node,
jid = jid:make(Host),
name = get_option(Options, title, <<>>)} | Acc];
_ ->
Acc
end
end,
NodeBloc = fun() ->
case tree_call(Host, get_nodes, [Host]) 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, [#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 and session hooks handling functions
%%
-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,
%% we'll also send the last items to remote contacts when the local user
%% connects. That's the reason to use the caps_add hook instead of the
%% presence_probe_hook for remote contacts: The latter is only called when a
%% 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).
send_last_pep(To, From);
caps_add(_From, _To, _Feature) ->
ok.
-spec caps_update(jid(), jid(), [binary()]) -> ok.
caps_update(From, To, _Features) ->
send_last_pep(To, From).
-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 resources
ok;
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.
-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
%%
-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.
-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(_, _) ->
true.
-spec unsubscribe_user(jid(), jid()) -> ok.
unsubscribe_user(Entity, Owner) ->
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) ->
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),
Host = host(LServer),
HomeTreeBase = <<"/home/", LServer/binary, "/", LUser/binary>>,
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),
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">> ->
delete_node(H, N, Entity);
({#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};
handle_call(plugins, _From, State) ->
{reply, State#state.plugins, State};
handle_call(pep_mapping, _From, State) ->
{reply, State#state.pep_mapping, State};
handle_call(nodetree, _From, State) ->
{reply, State#state.nodetree, State};
handle_call(stop, _From, State) ->
{stop, normal, ok, State};
handle_call(Request, From, State) ->
?WARNING_MSG("Unexpected call from ~p: ~p", [From, Request]),
{noreply, State}.
handle_cast(Msg, State) ->
?WARNING_MSG("Unexpected cast: ~p", [Msg]),
{noreply, State}.
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~s~n** ~s",
[xmpp:pp(Packet),
misc:format_exception(2, Class, Reason, StackTrace)])
end,
{noreply, State};
handle_info(Info, State) ->
?WARNING_MSG("Unexpected info: ~p", [Info]),
{noreply, State}.
terminate(_Reason,
#state{hosts = Hosts, server_host = ServerHost, nodetree = TreePlugin, plugins = Plugins}) ->
case lists:member(?PEPNODE, Plugins) of
true ->
ejabberd_hooks:delete(caps_add, ServerHost,
?MODULE, caps_add, 80),
ejabberd_hooks:delete(caps_update, ServerHost,
?MODULE, caps_update, 80),
ejabberd_hooks:delete(disco_sm_identity, ServerHost,
?MODULE, disco_sm_identity, 75),
ejabberd_hooks:delete(disco_sm_features, ServerHost,
?MODULE, disco_sm_features, 75),
ejabberd_hooks:delete(disco_sm_items, ServerHost,
?MODULE, disco_sm_items, 75),
gen_iq_handler:remove_iq_handler(ejabberd_sm,
ServerHost, ?NS_PUBSUB),
gen_iq_handler:remove_iq_handler(ejabberd_sm,
ServerHost, ?NS_PUBSUB_OWNER);
false ->
ok
end,
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),
ejabberd_hooks:delete(disco_local_features, ServerHost,
?MODULE, disco_local_features, 75),
ejabberd_hooks:delete(disco_local_items, ServerHost,
?MODULE, disco_local_items, 75),
ejabberd_hooks:delete(presence_probe_hook, ServerHost,
?MODULE, presence_probe, 80),
ejabberd_hooks:delete(roster_in_subscription, ServerHost,
?MODULE, in_subscription, 50),
ejabberd_hooks:delete(roster_out_subscription, ServerHost,
?MODULE, out_subscription, 50),
ejabberd_hooks:delete(remove_user, ServerHost,
?MODULE, remove_user, 50),
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}
%% Description: Convert process state when code is changed
%%--------------------------------------------------------------------
%% @private
code_change(_OldVsn, State, _Extra) -> {ok, State}.
%%--------------------------------------------------------------------
%%% Internal functions
%%--------------------------------------------------------------------
-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 Packet of
#message{type = T} when T /= error ->
case find_authorization_response(Packet) of
undefined ->
ok;
{error, Err} ->
ejabberd_router:route_error(Packet, Err);
AuthResponse ->
handle_authorization_response(
To#jid.lserver, Packet, AuthResponse)
end;
_ ->
Err = xmpp:err_service_unavailable(),
ejabberd_router:route_error(Packet, 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) ->
{result, #disco_info{identities = [#identity{category = <<"automation">>,
type = <<"command-list">>}]}};
command_disco_info(_Host, ?NS_PUBSUB_GET_PENDING, _From) ->
{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{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.
-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,
case Node of
<<>> ->
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 ->
command_disco_info(Host, Node, From);
_ ->
node_disco_info(Host, Node, From)
end.
-spec iq_disco_items(host(), binary(), jid(), undefined | rsm_set()) ->
{result, disco_items()} | {error, stanza_error()}.
iq_disco_items(Host, <<>>, From, _RSM) ->
case tree_action(Host, get_subnodes, [Host, <<>>, From]) 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,
#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, #disco_items{}};
iq_disco_items(Host, Item, From, RSM) ->
case str:tokens(Item, <<"!">>) of
[_Node, _ItemId] ->
{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;
_ -> {[], undefined}
end,
case tree_call(Host, get_subnodes, [Host, Node, From]) 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(iq()) -> iq().
iq_sm(#iq{to = To, sub_els = [SubEl]} = IQ) ->
LOwner = jid:tolower(jid:remove_resource(To)),
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} ->
xmpp:make_iq_result(IQ, IQRes);
{error, Error} ->
xmpp:make_error(IQ, Error)
end.
-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;
[] ->
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;
[] ->
{error, extended_error(xmpp:err_bad_request(), err_item_required())};
_ ->
{error, extended_error(xmpp:err_bad_request(), err_invalid_payload())}
end;
{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;
_ ->
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()};
_ ->
{error, xmpp:err_feature_not_implemented()}
end.
-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;
{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.
-spec adhoc_request(binary(), binary(), jid(), adhoc_command(),
atom(), [binary()]) -> adhoc_command() | {error, stanza_error()}.
adhoc_request(Host, _ServerHost, Owner,
#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_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_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, xmpp:err_item_not_found()}.
-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(xmpp:err_feature_not_implemented(),
err_unsupported('get-pending')),
{error, Err};
Ps ->
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,
Action = fun() -> {result, lists:flatmap(Tr, Plugins)} end,
case transaction(Host, Action, sync_dirty) of
{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>
-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 ~s on ~s:~s",
[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);
Err ->
Err
end.
%%% authorization handling
-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">>,
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) ->
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) ->
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;
(_) -> false
end,
Subs),
case Sub of
[{pending, SubId}|_] ->
NewSub = case Allow of
true -> subscribed;
false -> none
end,
node_call(Host, Type, set_subscriptions, [Nidx, Subscriber, NewSub, SubId]),
send_authorization_approval(Host, Subscriber, Node, NewSub),
{result, ok};
_ ->
Txt = ?T("No pending subscriptions found"),
{error, xmpp:err_unexpected_request(Txt, ejabberd_option:language())}
end.
%% @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>
%%<li>The service does not support node creation.</li>
%%<li>Only entities that are registered with the service are allowed to create nodes but the requesting entity is not registered.</li>
%%<li>The requesting entity does not have sufficient privileges to create nodes.</li>
%%<li>The requested Node already exists.</li>
%%<li>The request did not include a Node and "instant nodes" are not supported.</li>
%%</ul>
%%<p>ote: node creation is a particular case, error return code is evaluated at many places:</p>
%%<ul>
%%<li>iq_pubsub checks if service supports node creation (type exists)</li>
%%<li>create_node checks if instant nodes are supported</li>
%%<li>create_node asks node plugin if entity have sufficient privilege</li>
%%<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(), 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(), 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 = p1_rand:get_string(),
case create_node(Host, ServerHost, Node, Owner, Type, Access, Configuration) of
{result, _} ->
{result, #pubsub{create = Node}};
Error ->
Error
end;
false ->
{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),
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;
{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 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(), binary(), jid()) -> {result, pubsub_owner()} | {error, stanza_error()}.
delete_node(_Host, <<>>, _Owner) ->
{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} ->
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}}}} ->
lists:foreach(fun ({RNode, _RSubs}) ->
{RH, RN} = RNode#pubsub_node.nodeid,
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,
[ServerHost, RH, RN, RNidx])
end,
Removed),
case Result of
default -> {result, Reply};
_ -> {result, Result}
end;
{result, {_, {_, {Result, Removed}}}} ->
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])
end,
Removed),
case Result of
default -> {result, Reply};
_ -> {result, Result}
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
default -> {result, Reply};
_ -> {result, Result}
end;
Error ->
Error
end.
%% @see node_hometree:subscribe_node/5
%% @doc <p>Accepts or rejects subcription requests on a PubSub node.</p>
%%<p>There are several reasons why the subscription request might fail:</p>
%%<ul>
%%<li>The bare JID portions of the JIDs do not match.</li>
%%<li>The node has an access model of "presence" and the requesting entity is not subscribed to the owner's presence.</li>
%%<li>The node has an access model of "roster" and the requesting entity is not in one of the authorized roster groups.</li>
%%<li>The node has an access model of "whitelist" and the requesting entity is not on the whitelist.</li>
%%<li>The service requires payment for subscriptions to the node.</li>
%%<li>The requesting entity is anonymous and the service does not allow anonymous entities to subscribe.</li>
%%<li>The requesting entity has a pending subscription.</li>
%%<li>The requesting entity is blocked from subscribing (e.g., because having an affiliation of outcast).</li>
%%<li>The node does not support subscriptions.</li>
%%<li>The node does not exist.</li>
%%</ul>
-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 = 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),
OptionsFeature = lists:member(<<"subscription-options">>, Features),
HasOptions = not (SubOpts == []),
SubscribeConfig = get_option(Options, subscribe),
AccessModel = get_option(Options, access_model),
SendLast = get_option(Options, send_last_published_item),
AllowedGroups = get_option(Options, roster_groups_allowed, []),
CanSubscribe = case get_max_subscriptions_node(Host) of
Max when is_integer(Max) ->
case node_call(Host, Type, get_node_subscriptions, [Nidx]) of
{result, NodeSubs} ->
SubsNum = lists:foldl(
fun ({_, subscribed, _}, Acc) -> Acc+1;
(_, Acc) -> Acc
end, 0, NodeSubs),
SubsNum < Max;
_ ->
true
end;
_ ->
true
end,
if not SubscribeFeature ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('subscribe'))};
not SubscribeConfig ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('subscribe'))};
HasOptions andalso not OptionsFeature ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('subscription-options'))};
SubOpts == invalid ->
{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(xmpp:err_not_allowed(),
err_closed_node())};
true ->
Owners = node_owners_call(Host, Type, Nidx, O),
{PS, RG} = get_presence_and_roster_permissions(Host, JID,
Owners, AccessModel, AllowedGroups),
node_call(Host, Type, subscribe_node,
[Nidx, From, Subscriber, AccessModel,
SendLast, PS, RG, SubOpts])
end
end,
Reply = fun (Subscription) ->
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}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
send_items(Host, Node, Nidx, Type, Options, Subscriber, last),
ServerHost = serverhost(Host),
ejabberd_hooks:run(pubsub_subscribe_node, ServerHost,
[ServerHost, Host, Node, Subscriber, SubId]),
case Result of
default -> {result, Reply({subscribed, SubId})};
_ -> {result, Result}
end;
{result, {_TNode, {default, subscribed, SubId}}} ->
{result, Reply({subscribed, SubId})};
{result, {_TNode, {Result, subscribed, _SubId}}} ->
{result, Result};
{result, {TNode, {default, pending, _SubId}}} ->
send_authorization_request(TNode, JID),
{result, Reply(pending)};
{result, {TNode, {Result, pending}}} ->
send_authorization_request(TNode, JID),
{result, Result};
{result, {_, Result}} ->
{result, Result};
Error -> Error
end.
%% @doc <p>Unsubscribe <tt>JID</tt> from the <tt>Node</tt>.</p>
%%<p>There are several reasons why the unsubscribe request might fail:</p>
%%<ul>
%%<li>The requesting entity has multiple subscriptions to the node but does not specify a subscription ID.</li>
%%<li>The request does not specify an existing subscriber.</li>
%%<li>The requesting entity does not have sufficient privileges to unsubscribe the specified JID.</li>
%%<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(), 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,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, default}} ->
ServerHost = serverhost(Host),
ejabberd_hooks:run(pubsub_unsubscribe_node, ServerHost,
[ServerHost, Host, Node, Subscriber, SubId]),
{result, undefined};
Error -> Error
end.
%% @doc <p>Publish item to a PubSub node.</p>
%% <p>The permission to publish an item must be verified by the plugin implementation.</p>
%%<p>There are several reasons why the publish request might fail:</p>
%%<ul>
%%<li>The requesting entity does not have sufficient privileges to publish.</li>
%%<li>The node does not support item publication.</li>
%%<li>The node does not exist.</li>
%%<li>The payload size exceeds a service-defined limit.</li>
%%<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(), 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) ->
publish_item(Host, ServerHost, Node, Publisher, uniqid(), Payload, PubOpts, Access);
publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, PubOpts, Access) ->
Action = fun (#pubsub_node{options = Options, type = Type, id = Nidx}) ->
Features = plugin_features(Host, Type),
PublishFeature = lists:member(<<"publish">>, Features),
PublishModel = get_option(Options, publish_model),
DeliverPayloads = get_option(Options, deliver_payloads),
PersistItems = get_option(Options, persist_items),
MaxItems = max_items(Host, Options),
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(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(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 = #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,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
BrPayload = case Broadcast of
broadcast -> Payload;
PluginPayload -> PluginPayload
end,
set_cached_item(Host, Nidx, ItemId, Publisher, BrPayload),
case get_option(Options, deliver_notifications) of
true ->
broadcast_publish_item(Host, Node, Nidx, Type, Options, ItemId,
Publisher, BrPayload, Removed);
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}
end;
{result, {TNode, {default, Removed}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
broadcast_retract_items(Host, Node, Nidx, Type, Options, Removed),
set_cached_item(Host, Nidx, ItemId, Publisher, Payload),
{result, Reply};
{result, {TNode, {Result, Removed}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
broadcast_retract_items(Host, Node, Nidx, Type, Options, Removed),
set_cached_item(Host, Nidx, ItemId, Publisher, Payload),
{result, Result};
{result, {_, default}} ->
{result, Reply};
{result, {_, Result}} ->
{result, Result};
{error, #stanza_error{reason = 'item-not-found'}} ->
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, PubOpts) of
{result, #pubsub{create = NewNode}} ->
publish_item(Host, ServerHost, NewNode, Publisher, ItemId,
Payload, PubOpts, Access);
_ ->
{error, xmpp:err_item_not_found()}
end;
false ->
Txt = ?T("Automatic node creation is not enabled"),
{error, xmpp:err_item_not_found(Txt, ejabberd_option:language())}
end;
Error ->
Error
end.
%% @doc <p>Delete item from a PubSub node.</p>
%% <p>The permission to delete an item must be verified by the plugin implementation.</p>
%%<p>There are several reasons why the item retraction request might fail:</p>
%%<ul>
%%<li>The publisher does not have sufficient privileges to delete the requested item.</li>
%%<li>The node or item does not exist.</li>
%%<li>The request does not specify a node.</li>
%%<li>The request does not include an <item/> element or the <item/> element does not specify an ItemId.</li>
%%<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(), 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(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 matches
%% %% Request does not specify an item
%% {error, extended_error(?ERR_BAD_REQUEST, "item-required")};
not PersistentFeature ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('persistent-items'))};
not DeleteFeature ->
{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 = undefined,
case transaction(Host, Node, Action, sync_dirty) of
{result, {TNode, {Result, broadcast}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
broadcast_retract_items(Host, Node, Nidx, Type, Options, [ItemId], ForceNotify),
case get_cached_item(Host, Nidx) of
#pubsub_item{itemid = {ItemId, Nidx}} -> unset_cached_item(Host, Nidx);
_ -> ok
end,
case Result of
default -> {result, Reply};
_ -> {result, Result}
end;
{result, {_, default}} ->
{result, Reply};
{result, {_, Result}} ->
{result, Result};
Error ->
Error
end.
%% @doc <p>Delete all items of specified node owned by JID.</p>
%%<p>There are several reasons why the node purge request might fail:</p>
%%<ul>
%%<li>The node or service does not support node purging.</li>
%%<li>The requesting entity does not have sufficient privileges to purge the node.</li>
%%<li>The node is not configured to persist items.</li>
%%<li>The specified node does not exist.</li>
%%</ul>
-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),
PurgeFeature = lists:member(<<"purge-nodes">>, Features),
PersistentFeature = lists:member(<<"persistent-items">>, Features),
PersistentConfig = get_option(Options, persist_items),
if not PurgeFeature ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('purge-nodes'))};
not PersistentFeature ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('persistent-items'))};
not PersistentConfig ->
{error, extended_error(xmpp:err_feature_not_implemented(),
err_unsupported('persistent-items'))};
true -> node_call(Host, Type, purge_node, [Nidx, Owner])
end
end,
Reply = undefined,
case transaction(Host, Node, Action, transaction) of
{result, {TNode, {Result, broadcast}}} ->
Nidx = TNode#pubsub_node.id,
Type = TNode#pubsub_node.type,
Options = TNode#pubsub_node.options,
broadcast_purge_node(Host, Node, Nidx, Type, Options),
unset_cached_item(Host, Nidx),
case Result of
default -> {result, Reply};
_ -> {result, Result}
end;
{result, {_, default}} ->
{result, Reply};
{result, {_, Result}} ->
{result, Result};
Error ->
Error
end.
%% @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>
-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), 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])
end,
case transaction(Host, Node, Action, sync_dirty) of
{result, {_, Items}} -> Items;
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, 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]).
-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 ->
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, 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, _Count) ->
[].
-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 ->
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 ->
[LastItem]
end.
%% @doc <p>Return the list of affiliations as an XMPP response.</p>
-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(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) ->
[#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(), 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(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, xmpp:err_item_not_found()};
{result, {_, Affs}} ->
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(), binary(), jid(), [ps_affiliation()]) ->
{result, undefined} | {error, stanza_error()}.
set_affiliations(Host, Node, From, Affs) ->
Owner = jid:tolower(jid:remove_resource(From)),
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(#ps_affiliation{jid = JID, type = Affiliation}) ->
node_call(Host, Type, set_affiliation, [Nidx, JID, Affiliation]),
case Affiliation of