2015-12-15 16:11:29 +01:00
%%%----------------------------------------------------------------------
%%% 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>
2013-03-14 10:33:02 +01:00
%%%
%%%
2017-01-02 21:41:53 +01:00
%%% ejabberd, Copyright (C) 2002-2017 ProcessOne
2007-12-01 06:16:30 +01:00
%%%
2015-12-15 16:11:29 +01:00
%%% 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.
2007-12-01 06:16:30 +01:00
%%%
2015-12-15 16:11:29 +01:00
%%% 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.
2007-12-01 06:16:30 +01:00
%%%
2015-12-15 16:11:29 +01:00
%%% 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.
2009-06-15 15:58:40 +02:00
%%%
2015-12-15 16:11:29 +01:00
%%%----------------------------------------------------------------------
2009-06-15 15:58:40 +02:00
%%% Support for subscription-options and multi-subscribe features was
2009-08-17 19:16:43 +02:00
%%% added by Brian Cully (bjc AT kublai.com). Subscriptions and options are
2009-06-15 15:58:40 +02:00
%%% 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.
2007-12-01 06:16:30 +01:00
2003-07-08 22:11:27 +02:00
- module ( mod_pubsub ) .
- behaviour ( gen_mod ) .
2015-04-08 17:12:05 +02:00
- behaviour ( gen_server ) .
- author ( 'christophe.romain@process-one.net' ) .
2015-05-21 17:02:36 +02:00
- protocol ( { xep , 60 , '1.13-1' } ) .
- protocol ( { xep , 163 , '1.2' } ) .
2015-06-22 13:11:11 +02:00
- protocol ( { xep , 248 , '0.2' } ) .
2003-07-08 22:11:27 +02:00
- include ( " ejabberd.hrl " ) .
2013-04-08 11:12:54 +02:00
- include ( " logger.hrl " ) .
2016-08-30 08:48:08 +02:00
- include ( " xmpp.hrl " ) .
2007-12-01 06:16:30 +01:00
- include ( " pubsub.hrl " ) .
2013-03-14 10:33:02 +01:00
- define ( STDTREE , < < " tree " > > ) .
- define ( STDNODE , < < " flat " > > ) .
- define ( PEPNODE , < < " pep " > > ) .
2007-12-01 06:16:30 +01:00
%% exports for hooks
2015-07-07 00:24:06 +02:00
- export ( [ presence_probe / 3 , caps_add / 3 , caps_update / 3 ,
2015-04-08 17:12:05 +02:00
in_subscription / 6 , out_subscription / 4 ,
on_user_offline / 3 , 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 ] ) .
2013-03-14 10:33:02 +01:00
2007-12-01 06:16:30 +01:00
%% exported iq handlers
2016-08-30 08:48:08 +02:00
- 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 ] ) .
2007-12-01 06:16:30 +01:00
%% exports for console debug manual use
2015-04-08 17:12:05 +02:00
- export ( [ create_node / 5 , create_node / 7 , delete_node / 3 ,
subscribe_node / 5 , unsubscribe_node / 5 , publish_item / 6 ,
2016-03-08 18:04:29 +01:00
delete_item / 4 , delete_item / 5 , send_items / 7 , get_items / 2 , get_item / 3 ,
2015-04-08 17:12:05 +02:00
get_cached_item / 2 , get_configure / 5 , set_configure / 5 ,
tree_action / 3 , node_action / 4 , node_call / 4 ] ) .
2007-12-01 06:16:30 +01:00
%% general helpers for plugins
2016-09-13 11:30:05 +02:00
- export ( [ extended_error / 2 , service_jid / 1 ,
2015-11-30 14:57:21 +01:00
tree / 1 , tree / 2 , plugin / 2 , plugins / 1 , config / 3 ,
host / 1 , serverhost / 1 ] ) .
2007-12-01 06:16:30 +01:00
2016-08-30 08:48:08 +02:00
%% 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_presence_subscription_required / 0 ,
err_subid_required / 0 , err_too_many_subscriptions / 0 , err_unsupported / 1 ,
err_unsupported_access_model / 0 ] ) .
2007-12-01 06:16:30 +01:00
%% API and gen_server callbacks
2013-03-14 10:33:02 +01:00
- export ( [ start_link / 2 , start / 2 , stop / 1 , init / 1 ,
2015-04-08 17:12:05 +02:00
handle_call / 3 , handle_cast / 2 , handle_info / 2 ,
2016-07-06 13:58:48 +02:00
terminate / 2 , code_change / 3 , depends / 2 ] ) .
2003-07-08 22:11:27 +02:00
2015-06-01 14:38:27 +02:00
- export ( [ send_loop / 1 , mod_opt_type / 1 ] ) .
2009-04-23 00:19:41 +02:00
2005-05-23 02:30:29 +02:00
- define ( PROCNAME , ejabberd_mod_pubsub ) .
2010-01-12 16:14:47 +01:00
- define ( LOOPNAME , ejabberd_mod_pubsub_loop ) .
2003-07-08 22:11:27 +02:00
2006-02-02 06:00:27 +01:00
%%====================================================================
%% API
%%====================================================================
%%--------------------------------------------------------------------
%% Function: start_link() -> {ok,Pid} | ignore | {error,Error}
%% Description: Starts the server
%%--------------------------------------------------------------------
2013-03-14 10:33:02 +01:00
- export_type ( [
2015-04-08 17:12:05 +02:00
host / 0 ,
hostPubsub / 0 ,
hostPEP / 0 ,
%%
nodeIdx / 0 ,
nodeId / 0 ,
itemId / 0 ,
subId / 0 ,
payload / 0 ,
%%
nodeOption / 0 ,
nodeOptions / 0 ,
subOption / 0 ,
subOptions / 0 ,
2015-07-01 17:18:32 +02:00
pubOption / 0 ,
pubOptions / 0 ,
2015-04-08 17:12:05 +02:00
%%
affiliation / 0 ,
subscription / 0 ,
accessModel / 0 ,
publishModel / 0
] ) .
2013-03-14 10:33:02 +01:00
%% -type payload() defined here because the -type xmlel() is not accessible
%% from pubsub.hrl
- type ( payload ( ) : : [ ] | [ xmlel ( ) , . . . ] ) .
- export_type ( [
2015-04-08 17:12:05 +02:00
pubsubNode / 0 ,
pubsubState / 0 ,
pubsubItem / 0 ,
pubsubSubscription / 0 ,
pubsubLastItem / 0
] ) .
2013-03-14 10:33:02 +01:00
- type ( pubsubNode ( ) : :
#pubsub_node {
2015-04-08 17:12:05 +02:00
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 ( )
}
) .
2013-03-14 10:33:02 +01:00
- type ( pubsubState ( ) : :
#pubsub_state {
2015-04-08 17:12:05 +02:00
stateid : : { Entity : : ljid ( ) , Nidx : : mod_pubsub : nodeIdx ( ) } ,
items : : [ ItemId : : mod_pubsub : itemId ( ) ] ,
affiliation : : Affs : : mod_pubsub : affiliation ( ) ,
subscriptions : : [ { Sub : : mod_pubsub : subscription ( ) , SubId : : mod_pubsub : subId ( ) } ]
}
) .
2013-03-14 10:33:02 +01:00
- type ( pubsubItem ( ) : :
#pubsub_item {
2015-04-08 17:12:05 +02:00
itemid : : { ItemId : : mod_pubsub : itemId ( ) , Nidx : : mod_pubsub : nodeIdx ( ) } ,
creation : : { erlang : timestamp ( ) , ljid ( ) } ,
modification : : { erlang : timestamp ( ) , ljid ( ) } ,
payload : : mod_pubsub : payload ( )
}
) .
2013-03-14 10:33:02 +01:00
- type ( pubsubSubscription ( ) : :
#pubsub_subscription {
2015-04-08 17:12:05 +02:00
subid : : SubId : : mod_pubsub : subId ( ) ,
options : : [ ] | mod_pubsub : subOptions ( )
}
) .
2013-03-14 10:33:02 +01:00
- type ( pubsubLastItem ( ) : :
#pubsub_last_item {
2015-04-08 17:12:05 +02:00
nodeid : : mod_pubsub : nodeIdx ( ) ,
itemid : : mod_pubsub : itemId ( ) ,
creation : : { erlang : timestamp ( ) , ljid ( ) } ,
payload : : mod_pubsub : payload ( )
}
) .
2013-03-14 10:33:02 +01:00
- record ( state ,
2015-04-08 17:12:05 +02:00
{
server_host ,
host ,
access ,
pep_mapping = [ ] ,
ignore_pep_from_offline = true ,
last_item_cache = false ,
max_items_node = ? MAXITEMS ,
2015-11-10 16:28:57 +01:00
max_subscriptions_node = undefined ,
default_node_config = [ ] ,
2015-04-08 17:12:05 +02:00
nodetree = < < " nodetree_ " , ( ? STDTREE ) / binary > > ,
plugins = [ ? STDNODE ] ,
db_type
} ) .
2013-03-14 10:33:02 +01:00
- type ( state ( ) : :
#state {
2015-04-08 17:12:05 +02:00
server_host : : binary ( ) ,
host : : mod_pubsub : hostPubsub ( ) ,
access : : atom ( ) ,
pep_mapping : : [ { binary ( ) , binary ( ) } ] ,
ignore_pep_from_offline : : boolean ( ) ,
last_item_cache : : boolean ( ) ,
max_items_node : : non_neg_integer ( ) ,
2015-11-10 16:28:57 +01:00
max_subscriptions_node : : non_neg_integer ( ) | undefined ,
default_node_config : : [ { atom ( ) , binary ( ) | boolean ( ) | integer ( ) | atom ( ) } ] ,
2015-04-08 17:12:05 +02:00
nodetree : : binary ( ) ,
plugins : : [ binary ( ) , . . . ] ,
db_type : : atom ( )
}
2013-03-14 10:33:02 +01:00
2015-04-08 17:12:05 +02:00
) .
2013-03-14 10:33:02 +01:00
2006-02-02 06:00:27 +01:00
start_link ( Host , Opts ) - >
Proc = gen_mod : get_module_proc ( Host , ? PROCNAME ) ,
2015-04-08 17:12:05 +02:00
gen_server : start_link ( { local , Proc } , ? MODULE , [ Host , Opts ] , [ ] ) .
2005-06-20 05:18:13 +02:00
2006-02-02 06:00:27 +01:00
start ( Host , Opts ) - >
Proc = gen_mod : get_module_proc ( Host , ? PROCNAME ) ,
2013-03-14 10:33:02 +01:00
ChildSpec = { Proc , { ? MODULE , start_link , [ Host , Opts ] } ,
2015-04-08 17:12:05 +02:00
transient , 1000 , worker , [ ? MODULE ] } ,
2006-02-02 06:00:27 +01:00
supervisor : start_child ( ejabberd_sup , ChildSpec ) .
2003-07-08 22:11:27 +02:00
2006-02-02 06:00:27 +01:00
stop ( Host ) - >
Proc = gen_mod : get_module_proc ( Host , ? PROCNAME ) ,
gen_server : call ( Proc , stop ) ,
2007-02-22 08:39:05 +01:00
supervisor : delete_child ( ejabberd_sup , Proc ) .
2005-07-20 05:09:34 +02:00
2006-02-02 06:00:27 +01:00
%%====================================================================
%% gen_server callbacks
%%====================================================================
%%--------------------------------------------------------------------
%% Function: init(Args) -> {ok, State} |
2015-04-08 17:12:05 +02:00
%% {ok, State, Timeout} |
%% ignore |
%% {stop, Reason}
2006-02-02 06:00:27 +01:00
%% Description: Initiates the server
%%--------------------------------------------------------------------
2016-07-01 21:18:55 +02:00
- spec init ( [ binary ( ) | [ { _ , _ } ] , . . . ] ) - > { 'ok' , state ( ) } .
2013-03-14 10:33:02 +01:00
2006-04-07 15:39:48 +02:00
init ( [ ServerHost , Opts ] ) - >
2013-03-14 10:33:02 +01:00
? DEBUG ( " pubsub init ~p ~p " , [ ServerHost , Opts ] ) ,
Host = gen_mod : get_opt_host ( ServerHost , Opts , < < " pubsub.@HOST@ " > > ) ,
2016-03-13 09:38:40 +01:00
ejabberd_router : register_route ( Host , ServerHost ) ,
2013-03-14 10:33:02 +01:00
Access = gen_mod : get_opt ( access_createnode , Opts ,
2016-11-15 18:35:20 +01:00
fun acl : access_rules_validator / 1 , all ) ,
2013-03-14 10:33:02 +01:00
PepOffline = gen_mod : get_opt ( ignore_pep_from_offline , Opts ,
2015-04-08 17:12:05 +02:00
fun ( A ) when is_boolean ( A ) - > A end , true ) ,
2013-03-14 10:33:02 +01:00
IQDisc = gen_mod : get_opt ( iqdisc , Opts ,
2015-04-23 13:59:18 +02:00
fun gen_iq_handler : check_type / 1 , one_queue ) ,
2013-03-14 10:33:02 +01:00
LastItemCache = gen_mod : get_opt ( last_item_cache , Opts ,
2015-04-08 17:12:05 +02:00
fun ( A ) when is_boolean ( A ) - > A end , false ) ,
2013-03-14 10:33:02 +01:00
MaxItemsNode = gen_mod : get_opt ( max_items_node , Opts ,
2015-04-08 17:12:05 +02:00
fun ( A ) when is_integer ( A ) andalso A > = 0 - > A end , ? MAXITEMS ) ,
2015-11-10 16:28:57 +01:00
MaxSubsNode = gen_mod : get_opt ( max_subscriptions_node , Opts ,
fun ( A ) when is_integer ( A ) andalso A > = 0 - > A end , undefined ) ,
2016-11-15 18:35:20 +01:00
[ pubsub_index : init ( Host , ServerHost , Opts ) | | gen_mod : db_type ( ServerHost , ? MODULE ) == mnesia ] ,
2015-04-08 17:12:05 +02:00
{ Plugins , NodeTree , PepMapping } = init_plugins ( Host , ServerHost , Opts ) ,
2016-04-25 09:44:46 +02:00
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 , [ ] ) ,
2016-11-30 11:09:17 +01:00
ejabberd_mnesia : create ( ? MODULE , pubsub_last_item ,
2015-04-08 17:12:05 +02:00
[ { ram_copies , [ node ( ) ] } ,
{ attributes , record_info ( fields , pubsub_last_item ) } ] ) ,
2009-04-30 07:18:06 +02:00
mod_disco : register_feature ( ServerHost , ? NS_PUBSUB ) ,
2016-03-13 11:16:43 +01:00
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 ] ) ,
2015-04-08 17:12:05 +02:00
ejabberd_hooks : add ( sm_remove_connection_hook , ServerHost ,
? MODULE , on_user_offline , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( disco_local_identity , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_identity , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( disco_local_features , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_features , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( disco_local_items , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_items , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( presence_probe_hook , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , presence_probe , 80 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( roster_in_subscription , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , in_subscription , 50 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( roster_out_subscription , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , out_subscription , 50 ) ,
ejabberd_hooks : add ( remove_user , ServerHost ,
? MODULE , remove_user , 50 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : add ( anonymous_purge_hook , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , remove_user , 50 ) ,
2016-08-30 08:48:08 +02:00
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_DISCO_INFO ,
? MODULE , process_disco_info , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_DISCO_ITEMS ,
? MODULE , process_disco_items , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_PUBSUB ,
? MODULE , process_pubsub , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_PUBSUB_OWNER ,
? MODULE , process_pubsub_owner , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_VCARD ,
? MODULE , process_vcard , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_local , Host , ? NS_COMMANDS ,
? MODULE , process_commands , IQDisc ) ,
2009-04-10 23:11:35 +02:00
case lists : member ( ? PEPNODE , Plugins ) of
2015-04-08 17:12:05 +02:00
true - >
2015-07-07 00:24:06 +02:00
ejabberd_hooks : add ( caps_add , ServerHost ,
? MODULE , caps_add , 80 ) ,
2015-04-08 17:12:05 +02:00
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 , IQDisc ) ,
gen_iq_handler : add_iq_handler ( ejabberd_sm , ServerHost ,
? NS_PUBSUB_OWNER , ? MODULE , iq_sm , IQDisc ) ;
false - >
ok
2009-04-08 21:30:24 +02:00
end ,
2015-04-08 17:12:05 +02:00
pubsub_migrate : update_node_database ( Host , ServerHost ) ,
pubsub_migrate : update_state_database ( Host , ServerHost ) ,
2017-01-10 12:10:11 +01:00
pubsub_migrate : update_item_database ( Host , ServerHost ) ,
2015-04-08 17:12:05 +02:00
pubsub_migrate : update_lastitem_database ( Host , ServerHost ) ,
{ _ , State } = init_send_loop ( ServerHost ) ,
2010-01-13 11:22:55 +01:00
{ ok , State } .
2015-04-08 17:12:05 +02:00
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 } ,
2010-01-13 11:22:55 +01:00
Proc = gen_mod : get_module_proc ( ServerHost , ? LOOPNAME ) ,
2015-04-08 17:12:05 +02:00
Pid = case whereis ( Proc ) of
undefined - >
SendLoop = spawn ( ? MODULE , send_loop , [ State ] ) ,
register ( Proc , SendLoop ) ,
SendLoop ;
Loop - >
Loop
end ,
{ Pid , State } .
2016-07-06 13:58:48 +02:00
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 ] ) ,
lists : flatmap (
fun ( Name ) - >
Plugin = plugin ( ServerHost , Name ) ,
try apply ( Plugin , depends , [ Host , ServerHost , Opts ] )
catch _ : undef - > [ ]
end
end , Plugins ) .
2007-12-01 06:16:30 +01:00
%% @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>
2009-05-29 02:27:26 +02:00
%% <p>See {@link node_hometree:init/1} for an example implementation.</p>
2007-12-01 06:16:30 +01:00
init_plugins ( Host , ServerHost , Opts ) - >
2015-04-08 17:12:05 +02:00
TreePlugin = tree ( Host , gen_mod : get_opt ( nodetree , Opts ,
fun ( A ) when is_binary ( A ) - > A end ,
? STDTREE ) ) ,
2013-03-14 10:33:02 +01:00
? DEBUG ( " ** tree plugin is ~p " , [ TreePlugin ] ) ,
2007-12-01 06:16:30 +01:00
TreePlugin : init ( Host , ServerHost , Opts ) ,
2013-03-14 10:33:02 +01:00
Plugins = gen_mod : get_opt ( plugins , Opts ,
2015-04-08 17:12:05 +02:00
fun ( A ) when is_list ( A ) - > A end , [ ? STDNODE ] ) ,
2013-03-14 10:33:02 +01:00
PepMapping = gen_mod : get_opt ( pep_mapping , Opts ,
2015-04-08 17:12:05 +02:00
fun ( A ) when is_list ( A ) - > A end , [ ] ) ,
2013-03-14 10:33:02 +01:00
? DEBUG ( " ** PEP Mapping : ~p ~n " , [ PepMapping ] ) ,
2015-04-08 17:12:05 +02:00
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
end ,
[ ] , Plugins ) ,
2010-11-09 14:32:40 +01:00
{ lists : reverse ( PluginsOK ) , TreePlugin , PepMapping } .
2007-12-01 06:16:30 +01:00
2015-04-08 17:12:05 +02:00
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 ) ,
2007-12-01 06:16:30 +01:00
TreePlugin : terminate ( Host , ServerHost ) ,
ok .
2009-04-23 00:19:41 +02:00
send_loop ( State ) - >
receive
2015-04-08 17:12:05 +02:00
{ presence , JID , Pid } - >
Host = State #state.host ,
ServerHost = State #state.server_host ,
DBType = State #state.db_type ,
2015-11-24 16:44:13 +01:00
LJID = jid : tolower ( JID ) ,
BJID = jid : remove_resource ( LJID ) ,
2015-04-08 17:12:05 +02:00
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 ,
2016-07-08 15:18:39 +02:00
[ send_items ( Host , Node , Nidx , PType , Options , SubJID , last )
| | NodeRec #pubsub_node.type == PType ]
2015-04-08 17:12:05 +02:00
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
2015-11-24 16:44:13 +01:00
PeerJID = jid : make ( U , S , R ) ,
2015-04-08 17:12:05 +02:00
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 ( ) - >
2013-03-14 10:33:02 +01:00
Host = State #state.host ,
2015-11-24 16:44:13 +01:00
Owner = jid : remove_resource ( jid : tolower ( JID ) ) ,
2015-04-08 17:12:05 +02:00
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 ] ) )
2010-03-05 11:53:26 +01:00
end ) ,
2015-04-08 17:12:05 +02:00
send_loop ( State ) ;
stop - >
ok
2009-04-23 00:19:41 +02:00
end .
2007-12-01 06:16:30 +01:00
%% -------
2007-12-11 17:19:17 +01:00
%% disco hooks handling functions
2007-12-01 06:16:30 +01:00
%%
2016-08-30 08:48:08 +02:00
- spec disco_local_identity ( [ identity ( ) ] , jid ( ) , jid ( ) ,
binary ( ) , binary ( ) ) - > [ identity ( ) ] .
2013-03-14 10:33:02 +01:00
disco_local_identity ( Acc , _ From , To , < < > > , _ Lang ) - >
2016-02-26 09:33:07 +01:00
case lists : member ( ? PEPNODE , plugins ( host ( To #jid.lserver ) ) ) of
2015-04-08 17:12:05 +02:00
true - >
2016-08-30 08:48:08 +02:00
[ #identity { category = < < " pubsub " > > , type = < < " pep " > > } | Acc ] ;
2015-04-08 17:12:05 +02:00
false - >
Acc
2010-08-02 17:07:23 +02:00
end ;
2007-12-01 06:16:30 +01:00
disco_local_identity ( Acc , _ From , _ To , _ Node , _ Lang ) - >
Acc .
2016-09-08 16:08:48 +02:00
- spec disco_local_features ( { error , stanza_error ( ) } | { result , [ binary ( ) ] } | empty ,
2016-08-30 08:48:08 +02:00
jid ( ) , jid ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } | { result , [ binary ( ) ] } | empty .
2013-03-14 10:33:02 +01:00
disco_local_features ( Acc , _ From , To , < < > > , _ Lang ) - >
2016-03-03 14:43:56 +01:00
Host = host ( To #jid.lserver ) ,
2007-12-11 17:19:17 +01:00
Feats = case Acc of
2015-04-08 17:12:05 +02:00
{ result , I } - > I ;
_ - > [ ]
end ,
{ result , Feats ++ [ feature ( F ) | | F < - features ( Host , < < > > ) ] } ;
2007-12-21 01:08:59 +01:00
disco_local_features ( Acc , _ From , _ To , _ Node , _ Lang ) - >
Acc .
2007-12-11 17:19:17 +01:00
2016-09-08 16:08:48 +02:00
- spec disco_local_items ( { error , stanza_error ( ) } | { result , [ disco_item ( ) ] } | empty ,
2016-08-30 08:48:08 +02:00
jid ( ) , jid ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } | { result , [ disco_item ( ) ] } | empty .
2013-03-14 10:33:02 +01:00
disco_local_items ( Acc , _ From , _ To , < < > > , _ Lang ) - > Acc ;
disco_local_items ( Acc , _ From , _ To , _ Node , _ Lang ) - > Acc .
2016-08-30 08:48:08 +02:00
- spec disco_sm_identity ( [ identity ( ) ] , jid ( ) , jid ( ) ,
binary ( ) , binary ( ) ) - > [ identity ( ) ] .
2007-12-01 06:16:30 +01:00
disco_sm_identity ( Acc , From , To , Node , _ Lang ) - >
2015-11-24 16:44:13 +01:00
disco_identity ( jid : tolower ( jid : remove_resource ( To ) ) , Node , From )
2015-04-08 17:12:05 +02:00
++ Acc .
2010-08-02 17:07:23 +02:00
2016-08-30 08:48:08 +02:00
- spec disco_identity ( binary ( ) , binary ( ) , jid ( ) ) - > [ identity ( ) ] .
2010-08-02 17:07:23 +02:00
disco_identity ( _ Host , < < > > , _ From ) - >
2016-08-30 08:48:08 +02:00
[ #identity { category = < < " pubsub " > > , type = < < " pep " > > } ] ;
2010-08-02 17:07:23 +02:00
disco_identity ( Host , Node , From ) - >
2016-08-30 08:48:08 +02:00
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 = case get_option ( Options , title ) of
false - > < < > > ;
[ Title ] - > Title
end } ] } ;
_ - >
{ result , [ ] }
end
end ,
2010-08-02 17:07:23 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Result } } - > Result ;
_ - > [ ]
2010-08-02 17:07:23 +02:00
end .
2007-12-01 06:16:30 +01:00
2016-09-08 16:08:48 +02:00
- spec disco_sm_features ( { error , stanza_error ( ) } | { result , [ binary ( ) ] } | empty ,
2016-08-30 08:48:08 +02:00
jid ( ) , jid ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } | { result , [ binary ( ) ] } .
2010-08-02 17:07:23 +02:00
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 ,
2016-08-30 08:48:08 +02:00
OtherFeatures ++
disco_features ( jid : tolower ( jid : remove_resource ( To ) ) , Node , From ) } ;
2013-03-14 10:33:02 +01:00
disco_sm_features ( Acc , _ From , _ To , _ Node , _ Lang ) - > Acc .
2010-08-02 17:07:23 +02:00
2016-08-30 08:48:08 +02:00
- spec disco_features ( ljid ( ) , binary ( ) , jid ( ) ) - > [ binary ( ) ] .
2015-04-08 17:12:05 +02:00
disco_features ( Host , < < > > , _ From ) - >
[ ? NS_PUBSUB | [ feature ( F ) | | F < - plugin_features ( Host , < < " pep " > > ) ] ] ;
2010-08-02 17:07:23 +02:00
disco_features ( Host , Node , From ) - >
2016-08-30 08:48:08 +02:00
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 ,
2010-08-02 17:07:23 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Result } } - > Result ;
_ - > [ ]
2007-12-01 06:16:30 +01:00
end .
2016-09-08 16:08:48 +02:00
- spec disco_sm_items ( { error , stanza_error ( ) } | { result , [ disco_item ( ) ] } | empty ,
2016-08-30 08:48:08 +02:00
jid ( ) , jid ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } | { result , [ disco_item ( ) ] } .
2010-08-02 17:07:23 +02:00
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 ) - >
2015-04-08 17:12:05 +02:00
{ result , lists : usort ( OtherItems ++
2015-11-24 16:44:13 +01:00
disco_items ( jid : tolower ( jid : remove_resource ( To ) ) , Node , From ) ) } ;
2013-03-14 10:33:02 +01:00
disco_sm_items ( Acc , _ From , _ To , _ Node , _ Lang ) - > Acc .
2016-08-30 08:48:08 +02:00
- spec disco_items ( ljid ( ) , binary ( ) , jid ( ) ) - > [ disco_item ( ) ] .
2010-08-02 17:07:23 +02:00
disco_items ( Host , < < > > , From ) - >
2016-08-30 08:48:08 +02:00
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 = case get_option ( Options , title ) of
false - > < < > > ;
[ Title ] - > Title
end } | Acc ] ;
_ - >
Acc
end
end ,
2015-04-08 17:12:05 +02:00
NodeBloc = fun ( ) - >
2016-08-30 08:48:08 +02:00
{ result ,
lists : foldl ( Action , [ ] , tree_call ( Host , get_nodes , [ Host ] ) ) }
end ,
2015-04-08 17:12:05 +02:00
case transaction ( Host , NodeBloc , sync_dirty ) of
{ result , Items } - > Items ;
_ - > [ ]
2007-12-01 06:16:30 +01:00
end ;
2010-08-02 17:07:23 +02:00
disco_items ( Host , Node , From ) - >
2016-08-30 08:48:08 +02:00
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 ,
2009-04-30 07:18:06 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Result } } - > Result ;
_ - > [ ]
2007-12-01 06:16:30 +01:00
end .
2007-12-11 17:19:17 +01:00
%% -------
%% presence hooks handling functions
%%
2007-12-01 06:16:30 +01:00
2016-08-09 09:56:32 +02:00
- spec caps_add ( jid ( ) , jid ( ) , [ binary ( ) ] ) - > ok .
2015-07-07 00:24:06 +02:00
caps_add ( #jid { luser = U , lserver = S , lresource = R } , #jid { lserver = Host } = JID , _ Features )
2014-05-05 17:35:38 +02:00
when Host =/= S - >
2015-07-06 23:45:25 +02:00
%% 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
2015-07-07 00:24:06 +02:00
%% connects. That's the reason to use the caps_add hook instead of the
2015-07-06 23:45:25 +02:00
%% presence_probe_hook for remote contacts: The latter is only called when a
2015-07-07 00:24:06 +02:00
%% 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).
2014-05-05 17:35:38 +02:00
presence ( Host , { presence , U , S , [ R ] , JID } ) ;
2015-07-07 00:24:06 +02:00
caps_add ( _ From , _ To , _ Feature ) - >
2014-05-05 17:35:38 +02:00
ok .
2016-08-09 09:56:32 +02:00
- spec caps_update ( jid ( ) , jid ( ) , [ binary ( ) ] ) - > ok .
2015-07-07 00:24:06 +02:00
caps_update ( #jid { luser = U , lserver = S , lresource = R } , #jid { lserver = Host } = JID , _ Features ) - >
presence ( Host , { presence , U , S , [ R ] , JID } ) .
2016-08-09 09:56:32 +02:00
- spec presence_probe ( jid ( ) , jid ( ) , pid ( ) ) - > ok .
2014-05-05 17:35:38 +02:00
presence_probe ( #jid { luser = U , lserver = S , lresource = R } = JID , JID , Pid ) - >
presence ( S , { presence , JID , Pid } ) ,
presence ( S , { presence , U , S , [ R ] , JID } ) ;
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
2009-12-07 00:18:17 +01:00
ok ;
2014-11-18 01:13:22 +01:00
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,
2015-07-07 00:24:06 +02:00
%% those are handled via caps_add
2014-11-18 01:13:22 +01:00
ok .
2010-01-12 16:14:47 +01:00
presence ( ServerHost , Presence ) - >
2015-04-08 17:12:05 +02:00
{ SendLoop , _ } = case whereis ( gen_mod : get_module_proc ( ServerHost , ? LOOPNAME ) ) of
undefined - > init_send_loop ( ServerHost ) ;
Pid - > { Pid , undefined }
end ,
2016-08-09 09:56:32 +02:00
SendLoop ! Presence ,
ok .
2006-02-02 06:00:27 +01:00
2009-03-03 23:42:37 +01:00
%% -------
%% subscription hooks handling functions
%%
2009-03-10 11:35:57 +01:00
2016-08-09 09:56:32 +02:00
- spec out_subscription (
binary ( ) , binary ( ) , jid ( ) ,
subscribed | unsubscribed | subscribe | unsubscribe ) - > boolean ( ) .
2009-03-10 11:35:57 +01:00
out_subscription ( User , Server , JID , subscribed ) - >
2015-11-24 16:44:13 +01:00
Owner = jid : make ( User , Server , < < > > ) ,
{ PUser , PServer , PResource } = jid : tolower ( JID ) ,
2009-03-10 11:35:57 +01:00
PResources = case PResource of
2015-04-08 17:12:05 +02:00
< < > > - > user_resources ( PUser , PServer ) ;
_ - > [ PResource ]
end ,
presence ( Server , { presence , PUser , PServer , PResources , Owner } ) ,
2010-10-18 18:41:29 +02:00
true ;
2015-04-08 17:12:05 +02:00
out_subscription ( _ , _ , _ , _ ) - >
true .
2013-03-14 10:33:02 +01:00
2016-08-12 12:17:42 +02:00
- spec in_subscription ( boolean ( ) , binary ( ) , binary ( ) , jid ( ) ,
subscribe | subscribed | unsubscribe | unsubscribed ,
binary ( ) ) - > true .
2015-04-08 17:12:05 +02:00
in_subscription ( _ , User , Server , Owner , unsubscribed , _ ) - >
2015-11-24 16:44:13 +01:00
unsubscribe_user ( jid : make ( User , Server , < < > > ) , Owner ) ,
2010-10-18 18:41:29 +02:00
true ;
2015-04-08 17:12:05 +02:00
in_subscription ( _ , _ , _ , _ , _ , _ ) - >
true .
2009-03-03 23:42:37 +01:00
2010-01-12 16:14:47 +01:00
unsubscribe_user ( Entity , Owner ) - >
2013-03-14 10:33:02 +01:00
spawn ( fun ( ) - >
2015-05-26 19:19:57 +02:00
[ 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 ] ) ) ]
2015-04-08 17:12:05 +02:00
end ) .
2015-05-26 19:19:57 +02:00
unsubscribe_user ( Host , Entity , Owner ) - >
2015-11-24 16:44:13 +01:00
BJID = jid : tolower ( jid : remove_resource ( Owner ) ) ,
2015-05-26 19:19:57 +02:00
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 ) ) .
2010-01-12 16:14:47 +01:00
2007-12-18 15:45:26 +01:00
%% -------
%% user remove hook handling function
%%
2016-08-09 09:56:32 +02:00
- spec remove_user ( binary ( ) , binary ( ) ) - > ok .
2007-12-18 15:45:26 +01:00
remove_user ( User , Server ) - >
2015-11-24 16:44:13 +01:00
LUser = jid : nodeprep ( User ) ,
LServer = jid : nameprep ( Server ) ,
Entity = jid : make ( LUser , LServer , < < > > ) ,
2010-01-12 16:14:47 +01:00
Host = host ( LServer ) ,
2013-03-14 10:33:02 +01:00
HomeTreeBase = < < " /home/ " , LServer / binary , " / " , LUser / binary > > ,
spawn ( fun ( ) - >
2015-04-08 17:12:05 +02:00
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 ,
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 } ) - >
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 } , publisher } ) - >
node_action ( Host , PType ,
set_affiliation ,
[ Nidx , Entity , none ] ) ;
( _ ) - >
ok
2013-03-14 10:33:02 +01:00
end ,
2015-04-08 17:12:05 +02:00
Affs )
end ,
plugins ( Host ) )
2016-08-09 09:56:32 +02:00
end ) ,
ok .
2013-03-14 10:33:02 +01:00
2007-12-01 06:16:30 +01:00
handle_call ( server_host , _ From , State ) - >
{ reply , State #state.server_host , State } ;
handle_call ( plugins , _ From , State ) - >
{ reply , State #state.plugins , State } ;
2008-09-25 19:26:06 +02:00
handle_call ( pep_mapping , _ From , State ) - >
{ reply , State #state.pep_mapping , State } ;
2007-12-01 06:16:30 +01:00
handle_call ( nodetree , _ From , State ) - >
{ reply , State #state.nodetree , State } ;
2006-02-02 06:00:27 +01:00
handle_call ( stop , _ From , State ) - >
{ stop , normal , ok , State } .
%%--------------------------------------------------------------------
%% Function: handle_cast(Msg, State) -> {noreply, State} |
2015-04-08 17:12:05 +02:00
%% {noreply, State, Timeout} |
%% {stop, Reason, State}
2006-02-02 06:00:27 +01:00
%% Description: Handling cast messages
%%--------------------------------------------------------------------
2007-12-01 06:16:30 +01:00
%% @private
2013-03-14 10:33:02 +01:00
handle_cast ( _ Msg , State ) - > { noreply , State } .
2013-06-14 00:04:25 +02:00
%%--------------------------------------------------------------------
%% Function: handle_info(Info, State) -> {noreply, State} |
2015-04-08 17:12:05 +02:00
%% {noreply, State, Timeout} |
%% {stop, Reason, State}
2013-06-14 00:04:25 +02:00
%% Description: Handling all non call/cast messages
%%--------------------------------------------------------------------
%% @private
2016-08-30 08:48:08 +02:00
handle_info ( { route , From , To , #iq { } = IQ } ,
State ) when To #jid.lresource == < < " " > > - >
ejabberd_router : process_iq ( From , To , IQ ) ,
{ noreply , State } ;
handle_info ( { route , From , To , Packet } , State ) - >
case catch do_route ( To #jid.lserver , From , To , Packet ) of
2015-04-08 17:12:05 +02:00
{ 'EXIT' , Reason } - > ? ERROR_MSG ( " ~p " , [ Reason ] ) ;
_ - > ok
2006-02-02 06:00:27 +01:00
end ,
{ noreply , State } ;
2013-06-14 00:04:25 +02:00
handle_info ( _ Info , State ) - >
{ noreply , State } .
2006-02-02 06:00:27 +01:00
%%--------------------------------------------------------------------
%% 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.
%%--------------------------------------------------------------------
2007-12-01 06:16:30 +01:00
%% @private
2013-03-14 10:33:02 +01:00
terminate ( _ Reason ,
2015-04-08 17:12:05 +02:00
#state { host = Host , server_host = ServerHost , nodetree = TreePlugin , plugins = Plugins } ) - >
2009-04-10 23:11:35 +02:00
case lists : member ( ? PEPNODE , Plugins ) of
2015-04-08 17:12:05 +02:00
true - >
2015-07-07 00:24:06 +02:00
ejabberd_hooks : delete ( caps_add , ServerHost ,
? MODULE , caps_add , 80 ) ,
2015-04-08 17:12:05 +02:00
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
2009-04-10 09:36:17 +02:00
end ,
2015-04-08 17:12:05 +02:00
ejabberd_hooks : delete ( sm_remove_connection_hook , ServerHost ,
? MODULE , on_user_offline , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : delete ( disco_local_identity , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_identity , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : delete ( disco_local_features , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_features , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : delete ( disco_local_items , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , disco_local_items , 75 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : delete ( presence_probe_hook , ServerHost ,
2015-04-08 17:12:05 +02:00
? 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 ) ,
2013-03-14 10:33:02 +01:00
ejabberd_hooks : delete ( anonymous_purge_hook , ServerHost ,
2015-04-08 17:12:05 +02:00
? MODULE , remove_user , 50 ) ,
2016-08-30 08:48:08 +02:00
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 ) ,
2007-12-01 06:16:30 +01:00
mod_disco : unregister_feature ( ServerHost , ? NS_PUBSUB ) ,
2015-04-08 17:12:05 +02:00
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 ,
2016-03-13 15:37:39 +01:00
terminate_plugins ( Host , ServerHost , Plugins , TreePlugin ) ,
ejabberd_router : unregister_route ( Host ) .
2013-06-14 00:04:25 +02:00
2006-02-02 06:00:27 +01:00
%%--------------------------------------------------------------------
%% Func: code_change(OldVsn, State, Extra) -> {ok, NewState}
%% Description: Convert process state when code is changed
%%--------------------------------------------------------------------
2007-12-01 06:16:30 +01:00
%% @private
2013-03-14 10:33:02 +01:00
code_change ( _ OldVsn , State , _ Extra ) - > { ok , State } .
2006-02-02 06:00:27 +01:00
%%--------------------------------------------------------------------
%%% Internal functions
%%--------------------------------------------------------------------
2016-08-30 08:48:08 +02:00
- spec process_disco_info ( iq ( ) ) - > iq ( ) .
process_disco_info ( #iq { type = set , lang = Lang } = IQ ) - >
Txt = < < " 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 ( Host , Node , From , Lang ) of
{ result , IQRes } - >
xmpp : make_iq_result ( IQ , IQRes #disco_info { node = Node , xdata = Info } ) ;
{ error , Error } - >
xmpp : make_error ( IQ , Error )
end .
- spec process_disco_items ( iq ( ) ) - > iq ( ) .
process_disco_items ( #iq { type = set , lang = Lang } = IQ ) - >
Txt = < < " 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 , lang = Lang } = IQ ) - >
xmpp : make_iq_result ( IQ , iq_get_vcard ( Lang ) ) ;
process_vcard ( #iq { type = set , lang = Lang } = IQ ) - >
Txt = < < " 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 = < < " Value 'get' of 'type' attribute is not allowed " > > ,
xmpp : make_error ( IQ , xmpp : err_not_allowed ( Txt , Lang ) ) .
- spec do_route ( binary ( ) , jid ( ) , jid ( ) , stanza ( ) ) - > ok .
do_route ( Host , From , To , Packet ) - >
2003-07-08 22:11:27 +02:00
case To of
2015-04-08 17:12:05 +02:00
#jid { luser = < < > > , lresource = < < > > } - >
2016-08-30 08:48:08 +02:00
case Packet of
#message { type = T } when T / = error - >
case find_authorization_response ( Packet ) of
undefined - >
2015-04-08 17:12:05 +02:00
ok ;
2016-10-07 09:31:03 +02:00
{ error , Err } - >
ejabberd_router : route_error ( To , From , Packet , Err ) ;
AuthResponse - >
handle_authorization_response (
Host , From , To , Packet , AuthResponse )
2015-04-08 17:12:05 +02:00
end ;
_ - >
2016-08-30 08:48:08 +02:00
Err = xmpp : err_service_unavailable ( ) ,
ejabberd_router : route_error ( To , From , Packet , Err )
2015-04-08 17:12:05 +02:00
end ;
_ - >
2016-08-30 08:48:08 +02:00
Err = xmpp : err_item_not_found ( ) ,
ejabberd_router : route_error ( To , From , Packet , Err )
2003-07-08 22:11:27 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec command_disco_info ( binary ( ) , binary ( ) , jid ( ) ) - > { result , disco_info ( ) } .
2013-03-14 10:33:02 +01:00
command_disco_info ( _ Host , ? NS_COMMANDS , _ From ) - >
2016-08-30 08:48:08 +02:00
{ result , #disco_info { identities = [ #identity { category = < < " automation " > > ,
type = < < " command-list " > > } ] } } ;
2015-04-08 17:12:05 +02:00
command_disco_info ( _ Host , ? NS_PUBSUB_GET_PENDING , _ From ) - >
2016-08-30 08:48:08 +02:00
{ result , #disco_info { identities = [ #identity { category = < < " automation " > > ,
type = < < " command-node " > > } ] ,
features = [ ? NS_COMMANDS ] } } .
2009-11-04 20:54:15 +01:00
2016-08-30 08:48:08 +02:00
- spec node_disco_info ( binary ( ) , binary ( ) , jid ( ) ) - > { result , disco_info ( ) } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
node_disco_info ( Host , Node , From ) - >
node_disco_info ( Host , Node , From , true , true ) .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec node_disco_info ( binary ( ) , binary ( ) , jid ( ) , boolean ( ) , boolean ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , disco_info ( ) } | { error , stanza_error ( ) } .
2015-04-21 15:02:20 +02:00
node_disco_info ( Host , Node , _ From , _ Identity , _ Features ) - >
2016-08-30 08:48:08 +02:00
Action =
fun ( #pubsub_node { type = Type , options = Options } ) - >
NodeType = case get_option ( Options , node_type ) of
collection - > < < " collection " > > ;
_ - > < < " leaf " > >
end ,
Is = [ #identity { category = < < " pubsub " > > , type = NodeType } ] ,
Fs = [ ? NS_PUBSUB | [ feature ( F ) | | F < - plugin_features ( Host , Type ) ] ] ,
{ result , #disco_info { identities = Is , features = Fs } }
end ,
2009-04-30 07:18:06 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Result } } - > { result , Result } ;
Other - > Other
2009-04-30 07:18:06 +02:00
end .
2003-07-08 22:11:27 +02:00
2016-08-30 08:48:08 +02:00
- spec iq_disco_info ( binary ( ) , binary ( ) , jid ( ) , binary ( ) )
2016-09-08 16:08:48 +02:00
- > { result , disco_info ( ) } | { error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
iq_disco_info ( Host , SNode , From , Lang ) - >
2013-03-14 10:33:02 +01:00
[ Node | _ ] = case SNode of
2016-08-30 08:48:08 +02:00
< < > > - > [ < < > > ] ;
_ - > str : tokens ( SNode , < < " ! " > > )
end ,
2003-07-08 22:11:27 +02:00
case Node of
2015-04-08 17:12:05 +02:00
< < > > - >
2016-08-30 08:48:08 +02:00
{ result ,
#disco_info {
identities = [ #identity {
category = < < " pubsub " > > ,
type = < < " service " > > ,
name = translate : translate (
Lang , < < " Publish-Subscribe " > > ) } ] ,
features = [ ? NS_DISCO_INFO ,
? NS_DISCO_ITEMS ,
? NS_PUBSUB ,
? NS_COMMANDS ,
? NS_VCARD |
[ feature ( F ) | | F < - features ( Host , Node ) ] ] } } ;
2015-04-08 17:12:05 +02:00
? NS_COMMANDS - >
command_disco_info ( Host , Node , From ) ;
? NS_PUBSUB_GET_PENDING - >
command_disco_info ( Host , Node , From ) ;
_ - >
node_disco_info ( Host , Node , From )
2013-03-14 10:33:02 +01:00
end .
2016-08-30 08:48:08 +02:00
- spec iq_disco_items ( host ( ) , binary ( ) , jid ( ) , undefined | rsm_set ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , disco_items ( ) } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
iq_disco_items ( Host , < < > > , From , _ RSM ) - >
2016-08-30 08:48:08 +02:00
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 , tree_action ( Host , get_subnodes , [ Host , < < > > , From ] ) ) ,
{ result , #disco_items { items = Items } } ;
2015-04-08 17:12:05 +02:00
iq_disco_items ( Host , ? NS_COMMANDS , _ From , _ RSM ) - >
2016-08-30 08:48:08 +02:00
{ result ,
#disco_items { items = [ #disco_item { jid = jid : make ( Host ) ,
node = ? NS_PUBSUB_GET_PENDING ,
name = < < " Get Pending " > > } ] } } ;
2015-04-08 17:12:05 +02:00
iq_disco_items ( _ Host , ? NS_PUBSUB_GET_PENDING , _ From , _ RSM ) - >
2016-08-30 08:48:08 +02:00
{ result , #disco_items { } } ;
2015-04-08 17:12:05 +02:00
iq_disco_items ( Host , Item , From , RSM ) - >
2013-03-14 10:33:02 +01:00
case str : tokens ( Item , < < " ! " > > ) of
2015-04-08 17:12:05 +02:00
[ _ Node , _ ItemId ] - >
2016-08-30 08:48:08 +02:00
{ result , #disco_items { } } ;
2015-04-08 17:12:05 +02:00
[ 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 ;
2016-08-30 08:48:08 +02:00
_ - > { [ ] , undefined }
2015-04-08 17:12:05 +02:00
end ,
2016-08-30 08:48:08 +02:00
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 , 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 ] ) ,
#disco_item { jid = jid : make ( Host ) , name = Name }
end , NodeItems ) ,
{ result ,
#disco_items { items = Nodes ++ Items ,
rsm = RsmOut } }
2015-04-08 17:12:05 +02:00
end ,
case transaction ( Host , Node , Action , sync_dirty ) of
{ result , { _ , Result } } - > { result , Result } ;
Other - > Other
end
2003-07-08 22:11:27 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec iq_sm ( iq ( ) ) - > iq ( ) .
iq_sm ( #iq { to = To , sub_els = [ SubEl ] } = IQ ) - >
2015-11-24 16:44:13 +01:00
LOwner = jid : tolower ( jid : remove_resource ( To ) ) ,
2016-08-30 08:48:08 +02:00
Res = case xmpp : get_ns ( SubEl ) of
? NS_PUBSUB - >
iq_pubsub ( LOwner , all , IQ ) ;
? NS_PUBSUB_OWNER - >
iq_pubsub_owner ( LOwner , IQ )
end ,
2007-12-01 06:16:30 +01:00
case Res of
2016-08-30 08:48:08 +02:00
{ result , IQRes } - >
xmpp : make_iq_result ( IQ , IQRes ) ;
{ error , Error } - >
xmpp : make_error ( IQ , Error )
2007-12-01 06:16:30 +01:00
end .
2003-07-08 22:11:27 +02:00
2016-08-30 08:48:08 +02:00
- spec iq_get_vcard ( binary ( ) ) - > vcard_temp ( ) .
2007-12-01 06:16:30 +01:00
iq_get_vcard ( Lang ) - >
2016-08-30 08:48:08 +02:00
Desc = translate : translate ( Lang , < < " ejabberd Publish-Subscribe module " > > ) ,
#vcard_temp { fn = < < " ejabberd/mod_pubsub " > > ,
url = ? EJABBERD_URI ,
2016-11-23 13:51:48 +01:00
desc = < < Desc / binary , $\n , ? COPYRIGHT > > } .
2016-08-30 08:48:08 +02:00
- spec iq_pubsub ( binary ( ) | ljid ( ) , atom ( ) , iq ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub ( ) } | { error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
iq_pubsub ( Host , Access , #iq { from = From , type = IQType , lang = Lang ,
sub_els = [ SubEl ] } ) - >
case { IQType , SubEl } of
2016-09-08 14:49:27 +02:00
{ set , #pubsub { create = Node , configure = Configure ,
2016-08-30 08:48:08 +02:00
_ = undefined } } when is_binary ( Node ) - >
ServerHost = serverhost ( Host ) ,
Plugins = config ( ServerHost , plugins ) ,
2016-09-08 14:49:27 +02:00
Config = case Configure of
2016-10-07 09:31:03 +02:00
{ _ , XData } - > decode_node_config ( XData , Host , Lang ) ;
2016-09-08 14:49:27 +02:00
undefined - > [ ]
end ,
2016-08-30 08:48:08 +02:00
Type = hd ( Plugins ) ,
2016-10-07 09:31:03 +02:00
case Config of
{ error , _ } = Err - >
Err ;
_ - >
create_node ( Host , ServerHost , Node , From , Type , Access , Config )
end ;
2016-08-30 08:48:08 +02:00
{ set , #pubsub { publish = #ps_publish { node = Node , items = Items } ,
publish_options = XData , _ = undefined } } - >
ServerHost = serverhost ( Host ) ,
case Items of
[ #ps_item { id = ItemId , xml_els = Payload } ] - >
2016-10-07 09:31:03 +02:00
case decode_publish_options ( XData , Lang ) of
{ error , _ } = Err - >
Err ;
PubOpts - >
publish_item ( Host , ServerHost , Node , From , ItemId ,
Payload , PubOpts , Access )
end ;
2016-08-30 08:48:08 +02:00
[ ] - >
{ error , extended_error ( xmpp : err_bad_request ( ) , err_item_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
_ - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) , err_invalid_payload ( ) ) }
2015-04-08 17:12:05 +02:00
end ;
2016-08-30 08:48:08 +02:00
{ set , #pubsub { retract = #ps_retract { node = Node , notify = Notify , items = Items } ,
_ = undefined } } - >
case Items of
[ #ps_item { id = ItemId } ] - >
2016-09-08 14:49:27 +02:00
if ItemId / = < < > > - >
delete_item ( Host , Node , From , ItemId , Notify ) ;
true - >
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_item_required ( ) ) }
end ;
2016-08-30 08:48:08 +02:00
[ ] - >
{ error , extended_error ( xmpp : err_bad_request ( ) , err_item_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
_ - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) , err_invalid_payload ( ) ) }
2015-04-08 17:12:05 +02:00
end ;
2016-08-30 08:48:08 +02:00
{ set , #pubsub { subscribe = #ps_subscribe { node = Node , jid = JID } ,
options = Options , _ = undefined } } - >
Config = case Options of
2016-10-07 09:31:03 +02:00
#ps_options { xdata = XData } - >
decode_subscribe_options ( XData , Lang ) ;
_ - >
[ ]
2016-08-30 08:48:08 +02:00
end ,
2016-10-07 09:31:03 +02:00
case Config of
{ error , _ } = Err - >
Err ;
_ - >
subscribe_node ( Host , Node , From , JID , Config )
end ;
2016-08-30 08:48:08 +02:00
{ 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 ) ;
{ 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 } } - >
2016-10-07 09:31:03 +02:00
case decode_subscribe_options ( XData , Lang ) of
{ error , _ } = Err - >
Err ;
Config - >
set_options ( Host , Node , JID , SubId , Config )
end ;
2016-09-08 14:49:27 +02:00
{ set , #pubsub { } } - >
{ error , xmpp : err_bad_request ( ) } ;
2015-04-08 17:12:05 +02:00
_ - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_feature_not_implemented ( ) }
2003-07-08 22:11:27 +02:00
end .
2016-09-08 14:49:27 +02:00
- spec iq_pubsub_owner ( binary ( ) | ljid ( ) , iq ( ) ) - > { result , pubsub_owner ( ) | undefined } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
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 ( < < " No data form found " > > , Lang ) } ;
#xdata { type = cancel } - >
2016-09-08 14:49:27 +02:00
{ result , #pubsub_owner { } } ;
2016-08-30 08:48:08 +02:00
#xdata { type = submit } - >
2016-10-07 09:31:03 +02:00
case decode_node_config ( XData , Host , Lang ) of
{ error , _ } = Err - >
Err ;
Config - >
set_configure ( Host , Node , From , Config , Lang )
end ;
2016-08-30 08:48:08 +02:00
#xdata { } - >
{ error , xmpp : err_bad_request ( < < " Incorrect data form " > > , Lang ) }
2015-04-08 17:12:05 +02:00
end ;
2016-08-30 08:48:08 +02:00
{ 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 ( ) }
2009-07-20 10:53:56 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec adhoc_request ( binary ( ) , binary ( ) , jid ( ) , adhoc_command ( ) ,
2016-09-08 16:08:48 +02:00
atom ( ) , [ binary ( ) ] ) - > adhoc_command ( ) | { error , stanza_error ( ) } .
2009-07-20 10:53:56 +02:00
adhoc_request ( Host , _ ServerHost , Owner ,
2016-08-30 08:48:08 +02:00
#adhoc_command { node = ? NS_PUBSUB_GET_PENDING , lang = Lang ,
action = execute , xdata = undefined } ,
_ Access , Plugins ) - >
2009-07-20 10:53:56 +02:00
send_pending_node_form ( Host , Owner , Lang , Plugins ) ;
adhoc_request ( Host , _ ServerHost , Owner ,
2016-08-30 08:48:08 +02:00
#adhoc_command { node = ? NS_PUBSUB_GET_PENDING , lang = Lang ,
2016-10-07 09:31:03 +02:00
action = execute , xdata = #xdata { } = XData } = Request ,
2016-08-30 08:48:08 +02:00
_ Access , _ Plugins ) - >
2016-10-07 09:31:03 +02:00
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 { action = completed } ) ;
Err - >
Err
end
2009-07-20 10:53:56 +02:00
end ;
2013-03-14 10:33:02 +01:00
adhoc_request ( _ Host , _ ServerHost , _ Owner ,
2016-08-30 08:48:08 +02:00
#adhoc_command { action = cancel } , _ Access , _ Plugins ) - >
#adhoc_command { status = canceled } ;
2015-04-08 17:12:05 +02:00
adhoc_request ( _ Host , _ ServerHost , _ Owner , Other , _ Access , _ Plugins ) - >
2009-07-20 10:53:56 +02:00
? DEBUG ( " Couldn't process ad hoc command: ~n ~p " , [ Other ] ) ,
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_item_not_found ( ) } .
2009-07-20 10:53:56 +02:00
2016-08-30 08:48:08 +02:00
- spec send_pending_node_form ( binary ( ) , jid ( ) , binary ( ) ,
2016-09-08 16:08:48 +02:00
[ binary ( ) ] ) - > adhoc_command ( ) | { error , stanza_error ( ) } .
2009-07-20 10:53:56 +02:00
send_pending_node_form ( Host , Owner , _ Lang , Plugins ) - >
2015-04-08 17:12:05 +02:00
Filter = fun ( Type ) - >
lists : member ( < < " get-pending " > > , plugin_features ( Host , Type ) )
end ,
2009-07-20 10:53:56 +02:00
case lists : filter ( Filter , Plugins ) of
2015-04-08 17:12:05 +02:00
[ ] - >
2016-08-30 08:48:08 +02:00
Err = extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'get-pending' ) ) ,
2016-04-05 12:09:44 +02:00
{ error , Err } ;
2015-04-08 17:12:05 +02:00
Ps - >
2016-08-30 08:48:08 +02:00
case get_pending_nodes ( Host , Owner , Ps ) of
{ ok , Nodes } - >
XForm = #xdata { type = form ,
2016-10-07 09:31:03 +02:00
fields = pubsub_get_pending : encode (
[ { node , Nodes } ] ) } ,
2016-08-30 08:48:08 +02:00
#adhoc_command { status = executing , action = execute ,
xdata = XForm } ;
Err - >
Err
end
2009-07-20 10:53:56 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_pending_nodes ( binary ( ) , jid ( ) , [ binary ( ) ] ) - > { ok , [ binary ( ) ] } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2009-07-20 10:53:56 +02:00
get_pending_nodes ( Host , Owner , Plugins ) - >
2013-03-14 10:33:02 +01:00
Tr = fun ( Type ) - >
2015-04-08 17:12:05 +02:00
case node_call ( Host , Type , get_pending_nodes , [ Host , Owner ] ) of
{ result , Nodes } - > Nodes ;
_ - > [ ]
end
2016-08-30 08:48:08 +02:00
end ,
2015-04-08 17:12:05 +02:00
Action = fun ( ) - > { result , lists : flatmap ( Tr , Plugins ) } end ,
case transaction ( Host , Action , sync_dirty ) of
2016-08-30 08:48:08 +02:00
{ result , Res } - > { ok , Res } ;
2015-04-08 17:12:05 +02:00
Err - > Err
2009-07-20 10:53:56 +02:00
end .
%% @doc <p>Send a subscription approval form to Owner for all pending
%% subscriptions on Host and Node.</p>
2016-08-30 08:48:08 +02:00
- spec send_pending_auth_events ( binary ( ) , binary ( ) , jid ( ) ,
2016-09-08 16:08:48 +02:00
binary ( ) ) - > adhoc_command ( ) | { error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
send_pending_auth_events ( Host , Node , Owner , Lang ) - >
2015-04-08 17:12:05 +02:00
? DEBUG ( " Sending pending auth events for ~s on ~s : ~s " ,
2016-08-30 08:48:08 +02:00
[ 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 , xmpp : err_forbidden (
< < " Owner privileges required " > > , Lang ) }
end ;
false - >
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'get-pending' ) ) }
end
end ,
2009-07-20 10:53:56 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { N , Subs } } - >
2016-08-30 08:48:08 +02:00
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_command { } ;
2015-04-08 17:12:05 +02:00
Err - >
Err
2009-07-20 10:53:56 +02:00
end .
2007-12-01 06:16:30 +01:00
%%% authorization handling
2016-08-30 08:48:08 +02:00
- 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
2013-03-14 10:33:02 +01:00
Lang = < < " en " > > ,
2016-10-07 09:31:03 +02:00
Fs = pubsub_subscribe_authorization : encode (
[ { node , Node } ,
{ subscriber_jid , Subscriber } ,
{ allow , false } ] ,
fun ( T ) - > translate : translate ( Lang , T ) end ) ,
2016-08-30 08:48:08 +02:00
X = #xdata { type = form ,
title = translate : translate (
Lang , < < " PubSub subscriber request " > > ) ,
instructions = [ translate : translate (
Lang ,
< < " Choose whether to approve this entity's "
" subscription. " > > ) ] ,
fields = Fs } ,
Stanza = #message { sub_els = [ X ] } ,
lists : foreach (
fun ( Owner ) - >
ejabberd_router : route ( service_jid ( Host ) , jid : make ( Owner ) , Stanza )
end , node_owners_action ( Host , Type , Nidx , O ) ) .
2007-12-01 06:16:30 +01:00
2016-10-07 09:31:03 +02:00
- spec find_authorization_response ( message ( ) ) - > undefined |
pubsub_subscribe_authorization : result ( ) |
{ error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
find_authorization_response ( Packet ) - >
2016-08-30 08:48:08 +02:00
case xmpp : get_subtag ( Packet , #xdata { } ) of
2016-10-07 09:31:03 +02:00
#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 ) }
2016-08-30 08:48:08 +02:00
end ;
2016-10-07 09:31:03 +02:00
#xdata { } - >
{ error , xmpp : err_bad_request ( ) } ;
false - >
2016-08-30 08:48:08 +02:00
undefined
2007-12-01 06:16:30 +01:00
end .
2015-04-08 17:12:05 +02:00
2008-12-08 20:50:50 +01:00
%% @doc Send a message to JID with the supplied Subscription
2016-08-30 08:48:08 +02:00
- spec send_authorization_approval ( binary ( ) , jid ( ) , binary ( ) , subscribed | none ) - > ok .
2009-04-08 17:53:46 +02:00
send_authorization_approval ( Host , JID , SNode , Subscription ) - >
2016-08-30 08:48:08 +02:00
Event = #ps_event { subscription =
#ps_subscription { jid = JID ,
node = SNode ,
type = Subscription } } ,
Stanza = #message { sub_els = [ Event ] } ,
2009-12-04 17:31:25 +01:00
ejabberd_router : route ( service_jid ( Host ) , JID , Stanza ) .
2008-12-08 20:50:50 +01:00
2016-10-07 09:31:03 +02:00
- spec handle_authorization_response ( binary ( ) , jid ( ) , jid ( ) , message ( ) ,
pubsub_subscribe_authorization : result ( ) ) - > ok .
handle_authorization_response ( Host , From , To , Packet , Response ) - >
Node = proplists : get_value ( node , Response ) ,
Subscriber = proplists : get_value ( subscriber_jid , Response ) ,
Allow = proplists : get_value ( allow , Response ) ,
2016-08-30 08:48:08 +02:00
Lang = xmpp : get_lang ( Packet ) ,
2016-10-07 09:31:03 +02:00
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 - >
{ result , Subs } = node_call ( Host , Type , get_subscriptions , [ Nidx , Subscriber ] ) ,
update_auth ( Host , Node , Type , Nidx , Subscriber , Allow , Subs ) ;
false - >
{ error , xmpp : err_forbidden ( < < " Owner privileges required " > > , Lang ) }
end
end ,
case transaction ( Host , Node , Action , sync_dirty ) of
{ error , Error } - >
ejabberd_router : route_error ( To , From , Packet , Error ) ;
{ result , { _ , _ NewSubscription } } - >
%% XXX: notify about subscription state change, section 12.11
ok ;
2015-04-08 17:12:05 +02:00
_ - >
2016-10-07 09:31:03 +02:00
Err = xmpp : err_internal_server_error ( ) ,
2016-08-30 08:48:08 +02:00
ejabberd_router : route_error ( To , From , Packet , Err )
2007-12-01 06:16:30 +01:00
end .
2003-07-08 22:11:27 +02:00
2016-08-30 08:48:08 +02:00
- spec update_auth ( binary ( ) , binary ( ) , _ , _ , jid ( ) | error , boolean ( ) , _ ) - >
2016-09-08 16:08:48 +02:00
{ result , ok } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
update_auth ( Host , Node , Type , Nidx , Subscriber , Allow , Subs ) - >
Sub = lists : filter ( fun
( { pending , _ } ) - > true ;
( _ ) - > false
end ,
Subs ) ,
case Sub of
2016-04-05 12:09:44 +02:00
[ { pending , SubId } | _ ] - >
2015-04-08 17:12:05 +02:00
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 } ;
_ - >
2016-04-05 12:09:44 +02:00
Txt = < < " No pending subscriptions found " > > ,
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_unexpected_request ( Txt , ? MYLANG ) }
2009-05-29 04:14:07 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @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>
2008-04-01 12:11:39 +02:00
%%<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>
2015-04-08 17:12:05 +02:00
%%<li>The requested Node already exists.</li>
%%<li>The request did not include a Node and "instant nodes" are not supported.</li>
2007-12-01 06:16:30 +01:00
%%</ul>
%%<p>ote: node creation is a particular case, error return code is evaluated at many places:</p>
%%<ul>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec create_node ( host ( ) , binary ( ) , binary ( ) , jid ( ) ,
2016-09-08 16:08:48 +02:00
binary ( ) ) - > { result , pubsub ( ) } | { error , stanza_error ( ) } .
2016-03-07 14:47:42 +01:00
create_node ( Host , ServerHost , Node , Owner , Type ) - >
create_node ( Host , ServerHost , Node , Owner , Type , all , [ ] ) .
2016-08-30 08:48:08 +02:00
- spec create_node ( host ( ) , binary ( ) , binary ( ) , jid ( ) , binary ( ) ,
2016-09-08 16:08:48 +02:00
atom ( ) , [ { binary ( ) , [ binary ( ) ] } ] ) - > { result , pubsub ( ) } | { error , stanza_error ( ) } .
2009-10-20 17:03:07 +02:00
create_node ( Host , ServerHost , < < > > , Owner , Type , Access , Configuration ) - >
2015-04-08 17:12:05 +02:00
case lists : member ( < < " instant-nodes " > > , plugin_features ( Host , Type ) ) of
true - >
Node = randoms : get_string ( ) ,
case create_node ( Host , ServerHost , Node , Owner , Type , Access , Configuration ) of
{ result , _ } - >
2016-08-30 08:48:08 +02:00
{ result , #pubsub { create = Node } } ;
2015-04-08 17:12:05 +02:00
Error - >
Error
end ;
false - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) , err_nodeid_required ( ) ) }
2007-12-01 06:16:30 +01:00
end ;
create_node ( Host , ServerHost , Node , Owner , GivenType , Access , Configuration ) - >
2008-09-25 19:26:06 +02:00
Type = select_type ( ServerHost , Host , Node , GivenType ) ,
2016-10-07 09:31:03 +02:00
NodeOptions = merge_config ( 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 } - >
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
2016-08-30 08:48:08 +02:00
end ;
2016-10-07 09:31:03 +02:00
{ error , { virtual , Nidx } } - >
case node_call ( Host , Type , create_node , [ Nidx , Owner ] ) of
{ result , Result } - > { result , { Nidx , [ ] , Result } } ;
Error - > Error
end ;
Error - >
Error
end ;
_ - >
Txt = < < " You're not allowed to create nodes " > > ,
{ error , xmpp : err_forbidden ( Txt , ? MYLANG ) }
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 }
2007-12-01 06:16:30 +01:00
end ;
Error - >
2016-10-07 09:31:03 +02:00
%% in case we change transaction to sync_dirty...
%% node_call(Host, Type, delete_node, [Host, Node]),
%% tree_call(Host, delete_node, [Host, Node]),
2007-12-01 06:16:30 +01:00
Error
2003-07-08 22:11:27 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @doc <p>Delete specified node and all childs.</p>
%%<p>There are several reasons why the node deletion request might fail:</p>
%%<ul>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-09-08 16:08:48 +02:00
- spec delete_node ( host ( ) , binary ( ) , jid ( ) ) - > { result , pubsub_owner ( ) } | { error , stanza_error ( ) } .
2009-11-05 18:36:15 +01:00
delete_node ( _ Host , < < > > , _ Owner ) - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_not_allowed ( < < " No node specified " > > , ? MYLANG ) } ;
2007-12-01 06:16:30 +01:00
delete_node ( Host , Node , Owner ) - >
2015-04-08 17:12:05 +02:00
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 ;
_ - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_forbidden ( < < " Owner privileges required " > > , ? MYLANG ) }
2015-04-08 17:12:05 +02:00
end
end ,
2016-09-08 14:49:27 +02:00
Reply = undefined ,
2015-04-08 17:12:05 +02:00
ServerHost = serverhost ( Host ) ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , transaction ) of
2015-04-08 17:12:05 +02:00
{ 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 ,
broadcast_removed_node ( RH , RN , RNidx , RType , ROptions , SubsByDepth ) ,
ejabberd_hooks : run ( pubsub_delete_node ,
ServerHost ,
[ ServerHost , RH , RN , RNidx ] )
end ,
Removed ) ,
2007-12-01 06:16:30 +01:00
case Result of
default - > { result , Reply } ;
_ - > { result , Result }
end ;
2015-04-08 17:12:05 +02:00
{ result , { _ , { _ , { Result , Removed } } } } - >
lists : foreach ( fun ( { RNode , _ RSubs } ) - >
{ RH , RN } = RNode #pubsub_node.nodeid ,
RNidx = RNode #pubsub_node.id ,
ejabberd_hooks : run ( pubsub_delete_node ,
ServerHost ,
[ ServerHost , RH , RN , RNidx ] )
end ,
Removed ) ,
2007-12-01 06:16:30 +01:00
case Result of
default - > { result , Reply } ;
_ - > { result , Result }
end ;
2011-11-29 14:13:13 +01:00
{ result , { TNode , { _ , Result } } } - >
2015-04-08 17:12:05 +02:00
Nidx = TNode #pubsub_node.id ,
ejabberd_hooks : run ( pubsub_delete_node , ServerHost ,
[ ServerHost , Host , Node , Nidx ] ) ,
case Result of
default - > { result , Reply } ;
_ - > { result , Result }
end ;
2009-04-30 07:18:06 +02:00
Error - >
Error
2003-07-08 22:11:27 +02:00
end .
2009-05-29 02:27:26 +02:00
%% @see node_hometree:subscribe_node/5
2008-04-01 12:11:39 +02:00
%% @doc <p>Accepts or rejects subcription requests on a PubSub node.</p>
2007-12-01 06:16:30 +01:00
%%<p>There are several reasons why the subscription request might fail:</p>
%%<ul>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec subscribe_node ( host ( ) , binary ( ) , jid ( ) , binary ( ) , [ { binary ( ) , [ binary ( ) ] } ] ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub ( ) } | { error , stanza_error ( ) } .
2009-05-29 04:14:07 +02:00
subscribe_node ( Host , Node , From , JID , Configuration ) - >
2015-04-08 17:12:05 +02:00
SubModule = subscription_plugin ( Host ) ,
SubOpts = case SubModule : parse_options_xform ( Configuration ) of
{ result , GoodSubOpts } - > GoodSubOpts ;
_ - > invalid
end ,
2016-08-30 08:48:08 +02:00
Subscriber = jid : tolower ( JID ) ,
2015-04-08 17:12:05 +02:00
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 , [ ] ) ,
2015-11-10 16:28:57 +01:00
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 ,
2015-04-08 17:12:05 +02:00
if not SubscribeFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'subscribe' ) ) } ;
2015-04-08 17:12:05 +02:00
not SubscribeConfig - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'subscribe' ) ) } ;
2015-04-08 17:12:05 +02:00
HasOptions andalso not OptionsFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'subscription-options' ) ) } ;
2015-04-08 17:12:05 +02:00
SubOpts == invalid - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_invalid_options ( ) ) } ;
2015-11-10 16:28:57 +01:00
not CanSubscribe - >
%% fallback to closest XEP compatible result, assume we are not allowed to subscribe
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_allowed ( ) ,
err_closed_node ( ) ) } ;
2015-04-08 17:12:05 +02:00
true - >
Owners = node_owners_call ( Host , Type , Nidx , O ) ,
{ PS , RG } = get_presence_and_roster_permissions ( Host , Subscriber ,
Owners , AccessModel , AllowedGroups ) ,
node_call ( Host , Type , subscribe_node ,
[ Nidx , From , Subscriber , AccessModel ,
SendLast , PS , RG , SubOpts ] )
end
end ,
2013-03-14 10:33:02 +01:00
Reply = fun ( Subscription ) - >
2016-08-30 08:48:08 +02:00
Sub = case Subscription of
{ subscribed , SubId } - >
#ps_subscription { type = subscribed , subid = SubId } ;
Other - >
#ps_subscription { type = Other }
end ,
#pubsub { subscription = Sub #ps_subscription { jid = Subscriber , node = Node } }
2015-04-08 17:12:05 +02:00
end ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ 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 ) ,
2016-04-28 15:57:55 +02:00
ServerHost = serverhost ( Host ) ,
ejabberd_hooks : run ( pubsub_subscribe_node , ServerHost ,
[ ServerHost , Host , Node , Subscriber , SubId ] ) ,
2015-04-08 17:12:05 +02:00
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 , Subscriber ) ,
{ result , Reply ( pending ) } ;
{ result , { TNode , { Result , pending } } } - >
send_authorization_request ( TNode , Subscriber ) ,
{ result , Result } ;
{ result , { _ , Result } } - >
{ result , Result } ;
Error - > Error
2003-07-08 22:11:27 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @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>
2008-04-01 12:11:39 +02:00
%%<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>
2016-08-30 08:48:08 +02:00
- spec unsubscribe_node ( host ( ) , binary ( ) , jid ( ) , jid ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , undefined } | { error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
unsubscribe_node ( Host , Node , From , JID , SubId ) - >
Subscriber = jid : tolower ( JID ) ,
2015-04-08 17:12:05 +02:00
Action = fun ( #pubsub_node { type = Type , id = Nidx } ) - >
node_call ( Host , Type , unsubscribe_node , [ Nidx , From , Subscriber , SubId ] )
end ,
2009-04-30 07:18:06 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2016-04-28 15:57:55 +02:00
{ result , { _ , default } } - >
ServerHost = serverhost ( Host ) ,
ejabberd_hooks : run ( pubsub_unsubscribe_node , ServerHost ,
2016-08-30 08:48:08 +02:00
[ ServerHost , Host , Node , Subscriber , SubId ] ) ,
{ result , undefined } ;
2015-04-08 17:12:05 +02:00
Error - > Error
2007-12-01 06:16:30 +01:00
end .
2003-07-08 22:11:27 +02:00
2007-12-01 06:16:30 +01:00
%% @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>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec publish_item ( host ( ) , binary ( ) , binary ( ) , jid ( ) , binary ( ) ,
2016-09-08 16:08:48 +02:00
[ xmlel ( ) ] ) - > { result , pubsub ( ) } | { error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
publish_item ( Host , ServerHost , Node , Publisher , ItemId , Payload ) - >
2015-07-01 17:18:32 +02:00
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 ) - >
2015-04-08 17:12:05 +02:00
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 ) ,
if not PublishFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( publish ) ) } ;
2015-04-08 17:12:05 +02:00
PayloadSize > PayloadMaxSize - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) ,
err_payload_too_big ( ) ) } ;
2015-04-08 17:12:05 +02:00
( PayloadCount == 0 ) and ( Payload == [ ] ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_payload_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
( PayloadCount > 1 ) or ( PayloadCount == 0 ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_invalid_payload ( ) ) } ;
2015-04-08 17:12:05 +02:00
( DeliverPayloads == false ) and ( PersistItems == false ) and
( PayloadSize > 0 ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_item_forbidden ( ) ) } ;
2015-04-08 17:12:05 +02:00
( ( DeliverPayloads == true ) or ( PersistItems == true ) ) and ( PayloadSize == 0 ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) ,
err_item_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
true - >
node_call ( Host , Type , publish_item ,
2015-07-01 17:18:32 +02:00
[ Nidx , Publisher , PublishModel , MaxItems , ItemId , Payload , PubOpts ] )
2015-04-08 17:12:05 +02:00
end
end ,
2016-08-30 08:48:08 +02:00
Reply = #pubsub { publish = #ps_publish { node = Node ,
items = [ #ps_item { id = ItemId } ] } } ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2009-10-13 18:43:02 +02:00
{ result , { TNode , { Result , Broadcast , Removed } } } - >
2015-04-08 17:12:05 +02:00
Nidx = TNode #pubsub_node.id ,
2009-04-30 07:18:06 +02:00
Type = TNode #pubsub_node.type ,
Options = TNode #pubsub_node.options ,
2015-04-08 17:12:05 +02:00
BrPayload = case Broadcast of
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 ) ,
2011-08-31 16:11:01 +02:00
case get_option ( Options , deliver_notifications ) of
2015-04-08 17:12:05 +02:00
true - >
broadcast_publish_item ( Host , Node , Nidx , Type , Options , ItemId ,
Publisher , BrPayload , Removed ) ;
false - >
ok
end ,
2009-04-30 07:18:06 +02:00
case Result of
default - > { result , Reply } ;
_ - > { result , Result }
end ;
{ result , { TNode , { default , Removed } } } - >
2015-04-08 17:12:05 +02:00
Nidx = TNode #pubsub_node.id ,
2009-04-30 07:18:06 +02:00
Type = TNode #pubsub_node.type ,
Options = TNode #pubsub_node.options ,
2015-04-08 17:12:05 +02:00
broadcast_retract_items ( Host , Node , Nidx , Type , Options , Removed ) ,
set_cached_item ( Host , Nidx , ItemId , Publisher , Payload ) ,
2009-04-30 07:18:06 +02:00
{ result , Reply } ;
{ result , { TNode , { Result , Removed } } } - >
2015-04-08 17:12:05 +02:00
Nidx = TNode #pubsub_node.id ,
2009-04-30 07:18:06 +02:00
Type = TNode #pubsub_node.type ,
Options = TNode #pubsub_node.options ,
2015-04-08 17:12:05 +02:00
broadcast_retract_items ( Host , Node , Nidx , Type , Options , Removed ) ,
set_cached_item ( Host , Nidx , ItemId , Publisher , Payload ) ,
2009-04-30 07:18:06 +02:00
{ result , Result } ;
{ result , { _ , default } } - >
{ result , Reply } ;
{ result , { _ , Result } } - >
{ result , Result } ;
2016-09-08 16:08:48 +02:00
{ error , #stanza_error { reason = 'item-not-found' } } - >
2016-08-30 08:48:08 +02:00
Type = select_type ( ServerHost , Host , Node ) ,
case lists : member ( < < " auto-create " > > , plugin_features ( Host , Type ) ) of
2007-12-01 06:16:30 +01:00
true - >
2016-08-30 08:48:08 +02:00
case create_node ( Host , ServerHost , Node , Publisher , Type , Access , [ ] ) of
{ result , #pubsub { create = NewNode } } - >
2015-07-01 17:18:32 +02:00
publish_item ( Host , ServerHost , NewNode , Publisher , ItemId ,
2016-08-30 08:48:08 +02:00
Payload , PubOpts , Access ) ;
_ - >
{ error , xmpp : err_item_not_found ( ) }
2007-12-01 06:16:30 +01:00
end ;
false - >
2016-08-30 08:48:08 +02:00
Txt = < < " Automatic node creation is not enabled " > > ,
{ error , xmpp : err_item_not_found ( Txt , ? MYLANG ) }
2003-07-08 22:11:27 +02:00
end ;
2009-04-30 07:18:06 +02:00
Error - >
Error
2003-07-08 22:11:27 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @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>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec delete_item ( host ( ) , binary ( ) , jid ( ) , binary ( ) ) - > { result , undefined } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
delete_item ( Host , Node , Publisher , ItemId ) - >
delete_item ( Host , Node , Publisher , ItemId , false ) .
2015-04-08 17:12:05 +02:00
delete_item ( _ , < < > > , _ , _ , _ ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) , err_nodeid_required ( ) ) } ;
2007-12-01 06:16:30 +01:00
delete_item ( Host , Node , Publisher , ItemId , ForceNotify ) - >
2015-04-08 17:12:05 +02:00
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
%% %% Request does not specify an item
%% {error, extended_error(?ERR_BAD_REQUEST, "item-required")};
not PersistentFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'persistent-items' ) ) } ;
2015-04-08 17:12:05 +02:00
not DeleteFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'delete-items' ) ) } ;
2015-04-08 17:12:05 +02:00
true - >
node_call ( Host , Type , delete_item , [ Nidx , Publisher , PublishModel , ItemId ] )
end
end ,
2016-08-30 08:48:08 +02:00
Reply = undefined ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ 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
2003-07-08 22:11:27 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @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>
2008-04-01 12:11:39 +02:00
%%<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>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec purge_node ( mod_pubsub : host ( ) , binary ( ) , jid ( ) ) - > { result , undefined } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
purge_node ( Host , Node , Owner ) - >
2015-04-08 17:12:05 +02:00
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 - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'purge-nodes' ) ) } ;
2015-04-08 17:12:05 +02:00
not PersistentFeature - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'persistent-items' ) ) } ;
2015-04-08 17:12:05 +02:00
not PersistentConfig - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'persistent-items' ) ) } ;
2015-04-08 17:12:05 +02:00
true - > node_call ( Host , Type , purge_node , [ Nidx , Owner ] )
end
end ,
2016-08-30 08:48:08 +02:00
Reply = undefined ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ 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
2007-12-01 06:16:30 +01:00
end .
2003-07-08 22:11:27 +02:00
2007-12-01 06:16:30 +01:00
%% @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.
2016-08-30 08:48:08 +02:00
- spec get_items ( host ( ) , binary ( ) , jid ( ) , binary ( ) ,
binary ( ) , [ binary ( ) ] , undefined | rsm_set ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub ( ) } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
get_items ( Host , Node , From , SubId , SMaxItems , ItemIds , RSM ) - >
2016-08-30 08:48:08 +02:00
MaxItems = if SMaxItems == undefined - >
case get_max_items_node ( Host ) of
undefined - > ? MAXITEMS ;
Max - > Max
end ;
true - >
SMaxItems
end ,
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 ) ,
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 ,
#pubsub { items = #ps_items { node = Node ,
items = itemsEls ( lists : sublist ( SendItems , MaxItems ) ) } ,
rsm = RsmOut } } ;
Error - >
Error
2003-07-08 22:11:27 +02:00
end .
2013-03-14 10:33:02 +01:00
2009-05-29 00:30:43 +02:00
get_items ( Host , Node ) - >
2015-04-08 17:12:05 +02:00
Action = fun ( #pubsub_node { type = Type , id = Nidx } ) - >
2016-09-13 11:30:05 +02:00
node_call ( Host , Type , get_items , [ Nidx , service_jid ( Host ) , undefined ] )
2015-04-08 17:12:05 +02:00
end ,
2009-05-29 00:30:43 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , { Items , _ } } } - > Items ;
Error - > Error
2009-05-29 00:30:43 +02:00
end .
2013-03-14 10:33:02 +01:00
2009-05-29 00:30:43 +02:00
get_item ( Host , Node , ItemId ) - >
2015-04-08 17:12:05 +02:00
Action = fun ( #pubsub_node { type = Type , id = Nidx } ) - >
node_call ( Host , Type , get_item , [ Nidx , ItemId ] )
end ,
2009-05-29 00:30:43 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Items } } - > Items ;
Error - > Error
2009-05-29 00:30:43 +02:00
end .
2013-03-14 10:33:02 +01:00
2015-04-08 17:12:05 +02:00
get_allowed_items_call ( Host , Nidx , From , Type , Options , Owners ) - >
2016-09-13 11:30:05 +02:00
case get_allowed_items_call ( Host , Nidx , From , Type , Options , Owners , undefined ) of
2015-10-07 16:14:45 +02:00
{ result , { Items , _ RSM } } - > { result , Items } ;
2015-04-08 17:12:05 +02:00
Error - > Error
end .
get_allowed_items_call ( Host , Nidx , From , Type , Options , Owners , RSM ) - >
2010-08-02 17:07:23 +02:00
AccessModel = get_option ( Options , access_model ) ,
AllowedGroups = get_option ( Options , roster_groups_allowed , [ ] ) ,
2015-04-08 17:12:05 +02:00
{ 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 ) - >
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 ) - >
2016-09-13 11:30:05 +02:00
case node_action ( Host , Type , get_items , [ Nidx , LJID , undefined ] ) of
2015-04-08 17:12:05 +02:00
{ result , { [ LastItem | _ ] , _ } } - > LastItem ;
_ - > undefined
end ;
2016-04-20 11:27:32 +02:00
get_last_item ( Host , Type , Nidx , LJID , sql ) - >
2015-04-08 17:12:05 +02:00
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 ) - >
2016-09-13 11:30:05 +02:00
case node_action ( Host , Type , get_items , [ Nidx , LJID , undefined ] ) of
2015-04-08 17:12:05 +02:00
{ result , { Items , _ } } - > lists : sublist ( Items , Number ) ;
_ - > [ ]
end ;
2016-04-20 11:27:32 +02:00
get_last_items ( Host , Type , Nidx , LJID , Number , sql ) - >
2015-04-08 17:12:05 +02:00
case node_action ( Host , Type , get_last_items , [ Nidx , LJID , Number ] ) of
{ result , Items } - > Items ;
_ - > [ ]
end ;
get_last_items ( _ Host , _ Type , _ Nidx , _ LJID , _ Number , _ ) - >
[ ] .
2008-12-20 01:00:24 +01:00
%% @doc <p>Resend the items of a node to the user.</p>
%% @todo use cache-last-item feature
2015-04-08 17:12:05 +02:00
send_items ( Host , Node , Nidx , Type , Options , LJID , last ) - >
case get_last_item ( Host , Type , Nidx , LJID ) of
undefined - >
ok ;
LastItem - >
2015-12-16 16:19:35 +01:00
Stanza = items_event_stanza ( Node , Options , [ LastItem ] ) ,
dispatch_items ( Host , LJID , Node , Stanza )
2009-05-29 00:30:43 +02:00
end ;
2015-04-08 17:12:05 +02:00
send_items ( Host , Node , Nidx , Type , Options , LJID , Number ) when Number > 0 - >
2015-12-16 16:19:35 +01:00
Stanza = items_event_stanza ( Node , Options , get_last_items ( Host , Type , Nidx , Number , LJID ) ) ,
dispatch_items ( Host , LJID , Node , Stanza ) ;
2015-04-08 17:12:05 +02:00
send_items ( Host , Node , _ Nidx , _ Type , Options , LJID , _ ) - >
2015-12-16 16:19:35 +01:00
Stanza = items_event_stanza ( Node , Options , [ ] ) ,
dispatch_items ( Host , LJID , Node , Stanza ) .
2014-11-14 00:16:13 +01:00
2015-12-16 16:19:35 +01:00
dispatch_items ( { FromU , FromS , FromR } = From , { ToU , ToS , ToR } = To ,
Node , Stanza ) - >
2014-11-14 01:33:11 +01:00
C2SPid = case ejabberd_sm : get_session_pid ( ToU , ToS , ToR ) of
2015-04-08 17:12:05 +02:00
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 ,
2014-11-14 01:33:11 +01:00
if C2SPid == undefined - > ok ;
2015-04-08 17:12:05 +02:00
true - >
ejabberd_c2s : send_filtered ( C2SPid ,
{ pep_message , < < Node / binary , " +notify " > > } ,
2015-11-24 16:44:13 +01:00
service_jid ( From ) , jid : make ( To ) ,
2015-12-16 16:19:35 +01:00
Stanza )
2014-11-14 00:16:13 +01:00
end ;
2015-12-16 16:19:35 +01:00
dispatch_items ( From , To , _ Node , Stanza ) - >
ejabberd_router : route ( service_jid ( From ) , jid : make ( To ) , Stanza ) .
2015-04-08 17:12:05 +02:00
2013-06-14 00:04:25 +02:00
%% @doc <p>Return the list of affiliations as an XMPP response.</p>
2016-08-30 08:48:08 +02:00
- spec get_affiliations ( host ( ) , binary ( ) , jid ( ) , [ binary ( ) ] ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub ( ) } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
get_affiliations ( Host , Node , JID , Plugins ) when is_list ( Plugins ) - >
2016-08-30 08:48:08 +02:00
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 - >
{ result , Affs } = node_action ( Host , Type ,
get_entity_affiliations ,
[ Host , JID ] ) ,
{ Status , [ Affs | Acc ] }
end
end ,
{ ok , [ ] } , Plugins ) ,
2012-04-11 16:50:36 +02:00
case Result of
2015-04-08 17:12:05 +02:00
{ ok , Affs } - >
2016-08-30 08:48:08 +02:00
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 } } } ;
2015-04-08 17:12:05 +02:00
{ Error , _ } - >
Error
2012-04-11 16:50:36 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_affiliations ( host ( ) , binary ( ) , jid ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub_owner ( ) } | { error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
get_affiliations ( Host , Node , JID ) - >
2016-08-30 08:48:08 +02:00
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 ( < < " Owner privileges required " > > , ? MYLANG ) } ;
true - >
node_call ( Host , Type , get_node_affiliations , [ Nidx ] )
end
end ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , [ ] } } - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_item_not_found ( ) } ;
2015-04-08 17:12:05 +02:00
{ result , { _ , Affs } } - >
2016-08-30 08:48:08 +02:00
Entities = lists : flatmap (
fun ( { _ , none } ) - >
[ ] ;
( { AJID , Aff } ) - >
[ #ps_affiliation { jid = AJID , type = Aff } ]
end , Affs ) ,
{ result , #pubsub_owner { affiliations = { Node , Entities } } } ;
2015-04-08 17:12:05 +02:00
Error - >
Error
2003-07-08 22:11:27 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec set_affiliations ( host ( ) , binary ( ) , jid ( ) , [ ps_affiliation ( ) ] ) - >
2016-09-08 16:08:48 +02:00
{ result , undefined } | { error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
set_affiliations ( Host , Node , From , Affs ) - >
2015-11-24 16:44:13 +01:00
Owner = jid : tolower ( jid : remove_resource ( From ) ) ,
2016-08-30 08:48:08 +02:00
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
2015-04-08 17:12:05 +02:00
end ,
2016-08-30 08:48:08 +02:00
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 ,
2015-04-08 17:12:05 +02:00
set_node ,
[ N #pubsub_node { owners = NewOwners } ] ) ;
2016-08-30 08:48:08 +02:00
none - >
OldOwner = jid : tolower ( jid : remove_resource ( JID ) ) ,
case lists : member ( OldOwner , Owners ) of
true - >
NewOwners = Owners -- [ OldOwner ] ,
tree_call ( Host ,
2015-04-08 17:12:05 +02:00
set_node ,
[ N #pubsub_node { owners = NewOwners } ] ) ;
2016-08-30 08:48:08 +02:00
_ - >
ok
end ;
_ - >
ok
end
end , FilteredAffs ) ,
{ result , undefined } ;
_ - >
{ error , xmpp : err_forbidden (
< < " Owner privileges required " > > , ? MYLANG ) }
end
end ,
case transaction ( Host , Node , Action , sync_dirty ) of
{ result , { _ , Result } } - > { result , Result } ;
Other - > Other
2003-07-08 22:11:27 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_options ( binary ( ) , binary ( ) , jid ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , xdata ( ) } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
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 - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'subscription-options' ) ) }
2015-04-08 17:12:05 +02:00
end
end ,
2009-05-29 04:14:07 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2016-08-30 08:48:08 +02:00
{ result , { _ Node , XForm } } - > { result , XForm } ;
2015-04-08 17:12:05 +02:00
Error - > Error
2009-05-29 04:14:07 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_options_helper ( binary ( ) , jid ( ) , binary ( ) , binary ( ) , _ , binary ( ) ,
2016-09-08 16:08:48 +02:00
binary ( ) ) - > { result , pubsub ( ) } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
get_options_helper ( Host , JID , Lang , Node , Nidx , SubId , Type ) - >
2016-08-30 08:48:08 +02:00
Subscriber = jid : tolower ( JID ) ,
2015-04-08 17:12:05 +02:00
{ result , Subs } = node_call ( Host , Type , get_subscriptions , [ Nidx , Subscriber ] ) ,
SubIds = [ Id | | { Sub , Id } < - Subs , Sub == subscribed ] ,
case { SubId , SubIds } of
{ _ , [ ] } - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) ,
err_not_subscribed ( ) ) } ;
2015-04-08 17:12:05 +02:00
{ < < > > , [ SID ] } - >
read_sub ( Host , Node , Nidx , Subscriber , SID , Lang ) ;
{ < < > > , _ } - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) ,
err_subid_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
{ _ , _ } - >
ValidSubId = lists : member ( SubId , SubIds ) ,
if ValidSubId - >
read_sub ( Host , Node , Nidx , Subscriber , SubId , Lang ) ;
true - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) ,
err_invalid_subid ( ) ) }
2015-04-08 17:12:05 +02:00
end
2009-05-29 04:14:07 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec read_sub ( binary ( ) , binary ( ) , nodeIdx ( ) , ljid ( ) , binary ( ) , binary ( ) ) - > { result , pubsub ( ) } .
2015-04-08 17:12:05 +02:00
read_sub ( Host , Node , Nidx , Subscriber , SubId , Lang ) - >
SubModule = subscription_plugin ( Host ) ,
2016-08-30 08:48:08 +02:00
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 ( ) ] } ] ) - >
2016-09-08 16:08:48 +02:00
{ result , undefined } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
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 - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'subscription-options' ) ) }
2015-04-08 17:12:05 +02:00
end
end ,
2009-05-29 04:14:07 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ Node , Result } } - > { result , Result } ;
Error - > Error
2009-05-29 04:14:07 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec set_options_helper ( binary ( ) , [ { binary ( ) , [ binary ( ) ] } ] , jid ( ) ,
nodeIdx ( ) , binary ( ) , binary ( ) ) - >
2016-09-08 16:08:48 +02:00
{ result , undefined } | { error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
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 ,
2016-08-30 08:48:08 +02:00
Subscriber = jid : tolower ( JID ) ,
2015-04-08 17:12:05 +02:00
{ result , Subs } = node_call ( Host , Type , get_subscriptions , [ Nidx , Subscriber ] ) ,
SubIds = [ Id | | { Sub , Id } < - Subs , Sub == subscribed ] ,
case { SubId , SubIds } of
{ _ , [ ] } - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) , err_not_subscribed ( ) ) } ;
2015-04-08 17:12:05 +02:00
{ < < > > , [ SID ] } - >
write_sub ( Host , Nidx , Subscriber , SID , SubOpts ) ;
{ < < > > , _ } - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_not_acceptable ( ) , err_subid_required ( ) ) } ;
2015-04-08 17:12:05 +02:00
{ _ , _ } - >
write_sub ( Host , Nidx , Subscriber , SubId , SubOpts )
2009-05-29 04:14:07 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec write_sub ( binary ( ) , nodeIdx ( ) , ljid ( ) , binary ( ) , _ ) - > { result , undefined } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2015-04-08 17:12:05 +02:00
write_sub ( _ Host , _ Nidx , _ Subscriber , _ SubId , invalid ) - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_bad_request ( ) , err_invalid_options ( ) ) } ;
2015-04-08 17:12:05 +02:00
write_sub ( _ Host , _ Nidx , _ Subscriber , _ SubId , [ ] ) - >
2016-08-30 08:48:08 +02:00
{ result , undefined } ;
2015-04-08 17:12:05 +02:00
write_sub ( Host , Nidx , Subscriber , SubId , Options ) - >
SubModule = subscription_plugin ( Host ) ,
case SubModule : set_subscription ( Subscriber , Nidx , SubId , Options ) of
2016-08-30 08:48:08 +02:00
{ result , _ } - > { result , undefined } ;
{ error , _ } - > { error , extended_error ( xmpp : err_not_acceptable ( ) ,
err_invalid_subid ( ) ) }
2009-05-29 04:14:07 +02:00
end .
2003-07-08 22:11:27 +02:00
2007-12-01 06:16:30 +01:00
%% @doc <p>Return the list of subscriptions as an XMPP response.</p>
2016-08-30 08:48:08 +02:00
- spec get_subscriptions ( host ( ) , binary ( ) , jid ( ) , [ binary ( ) ] ) - >
2016-09-08 16:08:48 +02:00
{ result , pubsub ( ) } | { error , stanza_error ( ) } .
2009-05-08 01:35:59 +02:00
get_subscriptions ( Host , Node , JID , Plugins ) when is_list ( Plugins ) - >
2015-04-08 17:12:05 +02:00
Result = lists : foldl ( fun ( Type , { Status , Acc } ) - >
Features = plugin_features ( Host , Type ) ,
RetrieveFeature = lists : member ( < < " retrieve-subscriptions " > > , Features ) ,
if not RetrieveFeature - >
2016-08-30 08:48:08 +02:00
{ { error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'retrieve-subscriptions' ) ) } ,
2015-04-08 17:12:05 +02:00
Acc } ;
true - >
2015-11-24 16:44:13 +01:00
Subscriber = jid : remove_resource ( JID ) ,
2015-04-08 17:12:05 +02:00
{ result , Subs } = node_action ( Host , Type ,
get_entity_subscriptions ,
[ Host , Subscriber ] ) ,
{ Status , [ Subs | Acc ] }
end
end ,
{ ok , [ ] } , Plugins ) ,
2007-12-01 06:16:30 +01:00
case Result of
2015-04-08 17:12:05 +02:00
{ ok , Subs } - >
Entities = lists : flatmap ( fun
( { _ , none } ) - >
[ ] ;
( { #pubsub_node { nodeid = { _ , SubsNode } } , Sub } ) - >
case Node of
< < > > - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { node = SubsNode , type = Sub } ] ;
2015-04-08 17:12:05 +02:00
SubsNode - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { type = Sub } ] ;
2015-04-08 17:12:05 +02:00
_ - >
[ ]
end ;
( { _ , none , _ } ) - >
[ ] ;
( { #pubsub_node { nodeid = { _ , SubsNode } } , Sub , SubId , SubJID } ) - >
case Node of
< < > > - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { jid = SubJID ,
subid = SubId ,
type = Sub ,
node = SubsNode } ] ;
2015-04-08 17:12:05 +02:00
SubsNode - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { jid = SubJID ,
subid = SubId ,
type = Sub } ] ;
2015-04-08 17:12:05 +02:00
_ - >
[ ]
end ;
( { #pubsub_node { nodeid = { _ , SubsNode } } , Sub , SubJID } ) - >
case Node of
< < > > - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { jid = SubJID ,
type = Sub ,
node = SubsNode } ] ;
2015-04-08 17:12:05 +02:00
SubsNode - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { jid = SubJID , type = Sub } ] ;
2015-04-08 17:12:05 +02:00
_ - >
[ ]
end
end ,
lists : usort ( lists : flatten ( Subs ) ) ) ,
2016-08-30 08:48:08 +02:00
{ result , #pubsub { subscriptions = { < < > > , Entities } } } ;
2015-04-08 17:12:05 +02:00
{ Error , _ } - >
Error
2009-05-08 01:35:59 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-09-08 14:49:27 +02:00
- spec get_subscriptions ( host ( ) , binary ( ) , jid ( ) ) - > { result , pubsub_owner ( ) } |
2016-09-08 16:08:48 +02:00
{ error , stanza_error ( ) } .
2007-12-01 06:16:30 +01:00
get_subscriptions ( Host , Node , JID ) - >
2015-04-08 17:12:05 +02:00
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 - >
2016-08-30 08:48:08 +02:00
{ error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'manage-subscriptions' ) ) } ;
2015-04-08 17:12:05 +02:00
Affiliation / = owner - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_forbidden ( < < " Owner privileges required " > > , ? MYLANG ) } ;
2015-04-08 17:12:05 +02:00
true - >
node_call ( Host , Type , get_node_subscriptions , [ Nidx ] )
end
end ,
2007-12-01 06:16:30 +01:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Subs } } - >
2016-08-30 08:48:08 +02:00
Entities =
lists : flatmap (
fun ( { _ , none } ) - >
[ ] ;
( { _ , pending , _ } ) - >
[ ] ;
( { AJID , Sub } ) - >
[ #ps_subscription { jid = AJID , type = Sub } ] ;
2015-04-08 17:12:05 +02:00
( { AJID , Sub , SubId } ) - >
2016-08-30 08:48:08 +02:00
[ #ps_subscription { jid = AJID , type = Sub , subid = SubId } ]
end , Subs ) ,
2016-09-08 14:49:27 +02:00
{ result , #pubsub_owner { subscriptions = { Node , Entities } } } ;
2015-04-08 17:12:05 +02:00
Error - >
Error
2005-07-20 05:09:34 +02:00
end .
2003-07-08 22:11:27 +02:00
2015-04-08 17:12:05 +02:00
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 ) ] ;
2016-04-20 11:27:32 +02:00
get_subscriptions_for_send_last ( Host , PType , sql , JID , LJID , BJID ) - >
2015-04-08 17:12:05 +02:00
case catch 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 ) ] ;
_ - >
[ ]
end ;
get_subscriptions_for_send_last ( _ Host , _ PType , _ , _ JID , _ LJID , _ BJID ) - >
[ ] .
2016-09-08 14:49:27 +02:00
- spec set_subscriptions ( host ( ) , binary ( ) , jid ( ) , [ ps_subscription ( ) ] ) - >
2016-09-08 16:08:48 +02:00
{ result , undefined } | { error , stanza_error ( ) } .
2016-08-30 08:48:08 +02:00
set_subscriptions ( Host , Node , From , Entities ) - >
2015-11-24 16:44:13 +01:00
Owner = jid : tolower ( jid : remove_resource ( From ) ) ,
2016-08-30 08:48:08 +02:00
Notify = fun ( #ps_subscription { jid = JID , type = Sub } ) - >
Stanza = #message {
2016-10-07 09:31:03 +02:00
sub_els = [ #ps_event {
2016-08-30 08:48:08 +02:00
subscription = #ps_subscription {
jid = JID ,
type = Sub ,
node = Node } } ] } ,
ejabberd_router : route ( service_jid ( Host ) , 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 ( _ , { 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 (
< < " Owner privileges required " > > , ? MYLANG ) }
end
end ,
case transaction ( Host , Node , Action , sync_dirty ) of
{ result , { _ , Result } } - > { result , Result } ;
Other - > Other
2003-07-08 22:11:27 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_presence_and_roster_permissions (
host ( ) , ljid ( ) , [ ljid ( ) ] , accessModel ( ) ,
[ binary ( ) ] ) - > { boolean ( ) , boolean ( ) } .
2010-08-02 17:07:23 +02:00
get_presence_and_roster_permissions ( Host , From , Owners , AccessModel , AllowedGroups ) - >
if ( AccessModel == presence ) or ( AccessModel == roster ) - >
2015-04-08 17:12:05 +02:00
case Host of
{ User , Server , _ } - >
get_roster_info ( User , Server , From , AllowedGroups ) ;
_ - >
[ { OUser , OServer , _ } | _ ] = Owners ,
get_roster_info ( OUser , OServer , From , AllowedGroups )
end ;
true - >
{ true , true }
2010-08-02 17:07:23 +02:00
end .
2015-04-08 17:12:05 +02:00
get_roster_info ( _ , _ , { < < > > , < < > > , _ } , _ ) - >
2010-08-02 17:07:23 +02:00
{ false , false } ;
2015-04-08 17:12:05 +02:00
get_roster_info ( OwnerUser , OwnerServer , { SubscriberUser , SubscriberServer , _ } , AllowedGroups ) - >
LJID = { SubscriberUser , SubscriberServer , < < > > } ,
{ Subscription , Groups } = ejabberd_hooks : run_fold ( roster_get_jid_info ,
OwnerServer , { none , [ ] } ,
[ OwnerUser , OwnerServer , LJID ] ) ,
2013-03-14 10:33:02 +01:00
PresenceSubscription = Subscription == both orelse
2015-04-08 17:12:05 +02:00
Subscription == from orelse
{ OwnerUser , OwnerServer } == { SubscriberUser , SubscriberServer } ,
2013-03-14 10:33:02 +01:00
RosterGroup = lists : any ( fun ( Group ) - >
2015-04-08 17:12:05 +02:00
lists : member ( Group , AllowedGroups )
end ,
Groups ) ,
2010-08-02 17:07:23 +02:00
{ PresenceSubscription , RosterGroup } ;
2015-04-08 17:12:05 +02:00
get_roster_info ( OwnerUser , OwnerServer , JID , AllowedGroups ) - >
2015-11-24 16:44:13 +01:00
get_roster_info ( OwnerUser , OwnerServer , jid : tolower ( JID ) , AllowedGroups ) .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec service_jid ( jid ( ) | ljid ( ) | binary ( ) ) - > jid ( ) .
2016-03-25 17:52:08 +01:00
service_jid ( #jid { } = Jid ) - > Jid ;
service_jid ( { U , S , R } ) - > jid : make ( U , S , R ) ;
2016-08-30 08:48:08 +02:00
service_jid ( Host ) - > jid : make ( Host ) .
2013-06-14 00:04:25 +02:00
2009-08-17 19:16:43 +02:00
%% @spec (LJID, NotifyType, Depth, NodeOptions, SubOptions) -> boolean()
2015-04-08 17:12:05 +02:00
%% LJID = jid()
%% NotifyType = items | nodes
%% Depth = integer()
%% NodeOptions = [{atom(), term()}]
%% SubOptions = [{atom(), term()}]
2009-05-29 04:14:07 +02:00
%% @doc <p>Check if a notification must be delivered or not based on
%% node and subscription options.</p>
2015-04-08 17:12:05 +02:00
is_to_deliver ( LJID , NotifyType , Depth , NodeOptions , SubOptions ) - >
2009-06-15 15:45:40 +02:00
sub_to_deliver ( LJID , NotifyType , Depth , SubOptions )
2015-04-08 17:12:05 +02:00
andalso node_to_deliver ( LJID , NodeOptions ) .
2009-06-15 15:45:40 +02:00
sub_to_deliver ( _ LJID , NotifyType , Depth , SubOptions ) - >
lists : all ( fun ( Option ) - >
2015-04-08 17:12:05 +02:00
sub_option_can_deliver ( NotifyType , Depth , Option )
end ,
SubOptions ) .
node_to_deliver ( LJID , NodeOptions ) - >
presence_can_deliver ( LJID , get_option ( NodeOptions , presence_based_delivery ) ) .
2009-06-15 15:45:40 +02:00
sub_option_can_deliver ( items , _ , { subscription_type , nodes } ) - > false ;
sub_option_can_deliver ( nodes , _ , { subscription_type , items } ) - > false ;
2015-04-08 17:12:05 +02:00
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 ;
2015-12-07 16:16:11 +01:00
sub_option_can_deliver ( _ , _ , { expire , When } ) - > p1_time_compat : timestamp ( ) < When ;
2015-04-08 17:12:05 +02:00
sub_option_can_deliver ( _ , _ , _ ) - > true .
2009-05-29 04:14:07 +02:00
2016-08-30 08:48:08 +02:00
- spec presence_can_deliver ( ljid ( ) , boolean ( ) ) - > boolean ( ) .
2015-04-08 17:12:05 +02:00
presence_can_deliver ( _ , false ) - >
true ;
2010-05-05 15:42:49 +02:00
presence_can_deliver ( { User , Server , Resource } , true ) - >
2016-01-19 23:48:48 +01:00
case ejabberd_sm : get_user_present_resources ( User , Server ) of
2015-04-08 17:12:05 +02:00
[ ] - >
false ;
Ss - >
lists : foldl ( fun
( _ , true ) - >
true ;
2016-01-19 23:48:48 +01:00
( { _ , R } , _ Acc ) - >
case Resource of
2015-04-08 17:12:05 +02:00
< < > > - > true ;
R - > true ;
_ - > false
end
end ,
false , Ss )
2007-12-11 17:19:17 +01:00
end .
2007-12-01 06:16:30 +01:00
2016-08-30 08:48:08 +02:00
- spec state_can_deliver ( ljid ( ) , subOptions ( ) | [ ] ) - > [ ljid ( ) ] .
2010-05-12 13:45:56 +02:00
state_can_deliver ( { U , S , R } , [ ] ) - > [ { U , S , R } ] ;
state_can_deliver ( { U , S , R } , SubOptions ) - >
2015-04-08 17:12:05 +02:00
case lists : keysearch ( show_values , 1 , SubOptions ) of
%% If not in suboptions, item can be delivered, case doesn't apply
false - > [ { U , S , R } ] ;
%% If in a suboptions ...
{ _ , { _ , ShowValues } } - >
Resources = case R of
%% If the subscriber JID is a bare one, get all its resources
< < > > - > user_resources ( U , S ) ;
%% If the subscriber JID is a full one, use its resource
R - > [ R ]
end ,
lists : foldl ( fun ( Resource , Acc ) - >
get_resource_state ( { U , S , Resource } , ShowValues , Acc )
end ,
[ ] , Resources )
2010-05-12 13:45:56 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec get_resource_state ( ljid ( ) , [ binary ( ) ] , [ ljid ( ) ] ) - > [ ljid ( ) ] .
2010-05-12 13:45:56 +02:00
get_resource_state ( { U , S , R } , ShowValues , JIDs ) - >
case ejabberd_sm : get_session_pid ( U , S , R ) of
2015-04-08 17:12:05 +02:00
none - >
%% If no PID, item can be delivered
lists : append ( [ { U , S , R } ] , JIDs ) ;
Pid - >
Show = case ejabberd_c2s : get_presence ( Pid ) of
{ _ , _ , < < " available " > > , _ } - > < < " online " > > ;
{ _ , _ , State , _ } - > State
end ,
case lists : member ( Show , ShowValues ) of
%% If yes, item can be delivered
true - > lists : append ( [ { U , S , R } ] , JIDs ) ;
%% If no, item can't be delivered
false - > JIDs
end
2010-05-12 13:45:56 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec payload_xmlelements ( [ xmlel ( ) ] ) - > non_neg_integer ( ) .
2015-04-08 17:12:05 +02:00
payload_xmlelements ( Payload ) - >
payload_xmlelements ( Payload , 0 ) .
2008-12-09 23:02:47 +01:00
payload_xmlelements ( [ ] , Count ) - > Count ;
2013-03-14 10:33:02 +01:00
payload_xmlelements ( [ #xmlel { } | Tail ] , Count ) - >
payload_xmlelements ( Tail , Count + 1 ) ;
payload_xmlelements ( [ _ | Tail ] , Count ) - >
payload_xmlelements ( Tail , Count ) .
2008-12-08 20:50:50 +01:00
2015-12-16 16:19:35 +01:00
items_event_stanza ( Node , Options , Items ) - >
2015-04-08 17:12:05 +02:00
MoreEls = case Items of
[ LastItem ] - >
{ ModifNow , ModifUSR } = LastItem #pubsub_item.modification ,
2016-08-30 08:48:08 +02:00
[ #delay { stamp = ModifNow , from = jid : make ( ModifUSR ) } ] ;
2015-04-08 17:12:05 +02:00
_ - >
[ ]
end ,
2016-08-30 08:48:08 +02:00
BaseStanza = #message {
sub_els = [ #ps_event { items = #ps_items {
node = Node ,
items = itemsEls ( Items ) } }
| MoreEls ] } ,
2015-12-16 17:04:45 +01:00
NotificationType = get_option ( Options , notification_type , headline ) ,
add_message_type ( BaseStanza , NotificationType ) .
2015-04-08 17:12:05 +02:00
2007-12-01 06:16:30 +01:00
%%%%%% broadcast functions
2015-04-08 17:12:05 +02:00
broadcast_publish_item ( Host , Node , Nidx , Type , NodeOptions , ItemId , From , Payload , Removed ) - >
2009-06-15 15:45:40 +02:00
case get_collection_subscriptions ( Host , Node ) of
SubsByDepth when is_list ( SubsByDepth ) - >
2016-08-30 08:48:08 +02:00
EventItem0 = case get_option ( NodeOptions , deliver_payloads ) of
true - > #ps_item { xml_els = Payload , id = ItemId } ;
false - > #ps_item { id = ItemId }
end ,
EventItem = case get_option ( NodeOptions , itemreply , none ) of
owner - > %% owner not supported
EventItem0 ;
publisher - >
EventItem0 #ps_item {
publisher = jid : to_string ( From ) } ;
none - >
EventItem0
end ,
Stanza = #message {
sub_els =
[ #ps_event { items =
#ps_items { node = Node ,
items = [ EventItem ] } } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , From , Node , Nidx , Type ,
2016-08-30 08:48:08 +02:00
NodeOptions , SubsByDepth , items , Stanza , true ) ,
2009-04-30 07:18:06 +02:00
case Removed of
[ ] - >
ok ;
_ - >
2009-05-29 04:14:07 +02:00
case get_option ( NodeOptions , notify_retract ) of
2009-04-30 07:18:06 +02:00
true - >
2016-08-30 08:48:08 +02:00
RetractStanza = #message {
sub_els =
[ #ps_event {
items = #ps_items {
node = Node ,
retract = Removed } } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type ,
NodeOptions , SubsByDepth ,
items , RetractStanza , true ) ;
2009-04-30 07:18:06 +02:00
_ - >
ok
end
end ,
{ result , true } ;
_ - >
{ result , false }
end .
2015-04-08 17:12:05 +02:00
broadcast_retract_items ( Host , Node , Nidx , Type , NodeOptions , ItemIds ) - >
broadcast_retract_items ( Host , Node , Nidx , Type , NodeOptions , ItemIds , false ) .
broadcast_retract_items ( _ Host , _ Node , _ Nidx , _ Type , _ NodeOptions , [ ] , _ ForceNotify ) - >
2009-04-30 07:18:06 +02:00
{ result , false } ;
2015-04-08 17:12:05 +02:00
broadcast_retract_items ( Host , Node , Nidx , Type , NodeOptions , ItemIds , ForceNotify ) - >
2009-05-29 04:14:07 +02:00
case ( get_option ( NodeOptions , notify_retract ) or ForceNotify ) of
2009-04-30 07:18:06 +02:00
true - >
2009-06-15 15:45:40 +02:00
case get_collection_subscriptions ( Host , Node ) of
SubsByDepth when is_list ( SubsByDepth ) - >
2016-08-30 08:48:08 +02:00
Stanza = #message {
sub_els =
[ #ps_event {
items = #ps_items {
node = Node ,
retract = ItemIds } } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type ,
NodeOptions , SubsByDepth , items , Stanza , true ) ,
2009-01-03 01:25:40 +01:00
{ result , true } ;
_ - >
{ result , false }
2009-04-30 07:18:06 +02:00
end ;
_ - >
{ result , false }
end .
2007-12-01 06:16:30 +01:00
2015-04-08 17:12:05 +02:00
broadcast_purge_node ( Host , Node , Nidx , Type , NodeOptions ) - >
2009-05-29 04:14:07 +02:00
case get_option ( NodeOptions , notify_retract ) of
2009-04-30 07:18:06 +02:00
true - >
2009-06-15 15:45:40 +02:00
case get_collection_subscriptions ( Host , Node ) of
SubsByDepth when is_list ( SubsByDepth ) - >
2016-08-30 08:48:08 +02:00
Stanza = #message { sub_els = [ #ps_event { purge = Node } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type ,
NodeOptions , SubsByDepth , nodes , Stanza , false ) ,
2009-04-30 07:18:06 +02:00
{ result , true } ;
2014-12-18 11:51:28 +01:00
_ - >
2009-01-03 01:25:40 +01:00
{ result , false }
2009-04-30 07:18:06 +02:00
end ;
_ - >
{ result , false }
end .
2003-07-08 22:11:27 +02:00
2015-04-08 17:12:05 +02:00
broadcast_removed_node ( Host , Node , Nidx , Type , NodeOptions , SubsByDepth ) - >
2009-05-29 04:14:07 +02:00
case get_option ( NodeOptions , notify_delete ) of
2009-04-30 07:18:06 +02:00
true - >
2009-06-15 15:45:40 +02:00
case SubsByDepth of
2010-05-28 13:34:05 +02:00
[ ] - >
2009-04-30 07:18:06 +02:00
{ result , false } ;
2009-01-03 01:25:40 +01:00
_ - >
2016-08-30 08:48:08 +02:00
Stanza = #message { sub_els = [ #ps_event { delete = { Node , < < > > } } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type ,
NodeOptions , SubsByDepth , nodes , Stanza , false ) ,
2009-04-30 07:18:06 +02:00
{ result , true }
end ;
_ - >
{ result , false }
end .
2003-07-08 22:11:27 +02:00
2010-05-28 13:34:05 +02:00
broadcast_created_node ( _ , _ , _ , _ , _ , [ ] ) - >
{ result , false } ;
2015-04-08 17:12:05 +02:00
broadcast_created_node ( Host , Node , Nidx , Type , NodeOptions , SubsByDepth ) - >
2016-08-30 08:48:08 +02:00
Stanza = #message { sub_els = [ #ps_event { create = Node } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type , NodeOptions , SubsByDepth , nodes , Stanza , true ) ,
2010-05-28 13:34:05 +02:00
{ result , true } .
2015-04-08 17:12:05 +02:00
broadcast_config_notification ( Host , Node , Nidx , Type , NodeOptions , Lang ) - >
2009-05-29 04:14:07 +02:00
case get_option ( NodeOptions , notify_config ) of
2009-04-30 07:18:06 +02:00
true - >
2009-06-15 15:45:40 +02:00
case get_collection_subscriptions ( Host , Node ) of
SubsByDepth when is_list ( SubsByDepth ) - >
2009-05-29 04:14:07 +02:00
Content = case get_option ( NodeOptions , deliver_payloads ) of
2009-04-30 07:18:06 +02:00
true - >
2016-08-30 08:48:08 +02:00
#xdata { type = result ,
fields = get_configure_xfields (
Type , NodeOptions , Lang , [ ] ) } ;
2009-04-30 07:18:06 +02:00
false - >
2016-08-30 08:48:08 +02:00
undefined
2009-04-30 07:18:06 +02:00
end ,
2016-08-30 08:48:08 +02:00
Stanza = #message {
sub_els = [ #ps_event {
configuration = { Node , Content } } ] } ,
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , Node , Nidx , Type ,
NodeOptions , SubsByDepth , nodes , Stanza , false ) ,
2009-04-30 07:18:06 +02:00
{ result , true } ;
2010-05-28 13:34:05 +02:00
_ - >
2009-01-03 01:25:40 +01:00
{ result , false }
2009-04-30 07:18:06 +02:00
end ;
_ - >
{ result , false }
end .
2007-12-01 06:16:30 +01:00
2009-06-15 15:45:40 +02:00
get_collection_subscriptions ( Host , Node ) - >
2009-08-27 23:38:23 +02:00
Action = fun ( ) - >
2015-04-08 17:12:05 +02:00
{ result , get_node_subs_by_depth ( Host , Node , service_jid ( Host ) ) }
end ,
case transaction ( Host , Action , sync_dirty ) of
2009-08-27 23:38:23 +02:00
{ result , CollSubs } - > CollSubs ;
_ - > [ ]
end .
2009-06-15 15:45:40 +02:00
2015-04-08 17:12:05 +02:00
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 ] .
get_node_subs ( Host , #pubsub_node { type = Type , id = Nidx } ) - >
2015-12-16 12:56:06 +01:00
WithOptions = lists : member ( < < " subscription-options " > > , plugin_features ( Host , Type ) ) ,
2015-04-08 17:12:05 +02:00
case node_call ( Host , Type , get_node_subscriptions , [ Nidx ] ) of
2015-12-16 12:56:06 +01:00
{ result , Subs } - > get_options_for_subs ( Host , Nidx , Subs , WithOptions ) ;
2009-08-27 23:38:23 +02:00
Other - > Other
2009-06-15 15:45:40 +02:00
end .
2015-12-16 12:56:06 +01:00
get_options_for_subs ( _ Host , _ Nidx , Subs , false ) - >
lists : foldl ( fun ( { JID , subscribed , SubID } , Acc ) - >
[ { JID , SubID , [ ] } | Acc ] ;
( _ , Acc ) - >
Acc
end , [ ] , Subs ) ;
get_options_for_subs ( Host , Nidx , Subs , true ) - >
2015-04-08 17:12:05 +02:00
SubModule = subscription_plugin ( Host ) ,
2009-07-31 16:55:05 +02:00
lists : foldl ( fun ( { JID , subscribed , SubID } , Acc ) - >
2015-04-08 17:12:05 +02:00
case SubModule : get_subscription ( JID , Nidx , SubID ) of
#pubsub_subscription { options = Options } - > [ { JID , SubID , Options } | Acc ] ;
{ error , notfound } - > [ { JID , SubID , [ ] } | Acc ]
end ;
( _ , Acc ) - >
Acc
end , [ ] , Subs ) .
2009-07-31 16:55:05 +02:00
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , _ Node , _ Nidx , _ Type , NodeOptions , SubsByDepth , NotifyType , BaseStanza , SHIM ) - >
2009-10-12 11:05:26 +02:00
NotificationType = get_option ( NodeOptions , notification_type , headline ) ,
2009-05-29 04:14:07 +02:00
BroadcastAll = get_option ( NodeOptions , broadcast_all_resources ) , %% XXX this is not standard, but usefull
2009-01-03 01:25:40 +01:00
From = service_jid ( Host ) ,
2014-12-21 18:44:53 +01:00
Stanza = add_message_type ( BaseStanza , NotificationType ) ,
2009-04-08 17:53:46 +02:00
%% Handles explicit subscriptions
2010-01-12 11:33:39 +01:00
SubIDsByJID = subscribed_nodes_by_jid ( NotifyType , SubsByDepth ) ,
2015-04-21 16:13:23 +02:00
lists : foreach ( fun ( { LJID , _ NodeName , SubIDs } ) - >
2015-04-08 17:12:05 +02:00
LJIDs = case BroadcastAll of
true - >
{ U , S , _ } = LJID ,
[ { U , S , R } | | R < - user_resources ( U , S ) ] ;
false - >
[ LJID ]
end ,
%% Determine if the stanza should have SHIM ('SubID' and 'name') headers
StanzaToSend = case { SHIM , SubIDs } of
{ false , _ } - >
Stanza ;
%% If there's only one SubID, don't add it
{ true , [ _ ] } - >
2015-04-21 16:13:23 +02:00
Stanza ;
2015-04-08 17:12:05 +02:00
{ true , SubIDs } - >
2015-04-21 16:11:00 +02:00
add_shim_headers ( Stanza , subid_shim ( SubIDs ) )
2015-04-08 17:12:05 +02:00
end ,
lists : foreach ( fun ( To ) - >
2015-11-24 16:44:13 +01:00
ejabberd_router : route ( From , jid : make ( To ) , StanzaToSend )
2015-04-08 17:12:05 +02:00
end , LJIDs )
end , SubIDsByJID ) .
broadcast_stanza ( { LUser , LServer , LResource } , Publisher , Node , Nidx , Type , NodeOptions , SubsByDepth , NotifyType , BaseStanza , SHIM ) - >
2016-03-25 17:52:08 +01:00
broadcast_stanza ( { LUser , LServer , < < > > } , Node , Nidx , Type , NodeOptions , SubsByDepth , NotifyType , BaseStanza , SHIM ) ,
2009-04-08 17:53:46 +02:00
%% Handles implicit presence subscriptions
2014-05-06 12:37:44 +02:00
SenderResource = user_resource ( LUser , LServer , LResource ) ,
2010-03-05 17:52:41 +01:00
case ejabberd_sm : get_session_pid ( LUser , LServer , SenderResource ) of
C2SPid when is_pid ( C2SPid ) - >
2014-12-21 18:44:53 +01:00
NotificationType = get_option ( NodeOptions , notification_type , headline ) ,
Stanza = add_message_type ( BaseStanza , NotificationType ) ,
2010-03-05 17:52:41 +01:00
%% 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
2010-12-02 13:56:45 +01:00
ejabberd_c2s : broadcast ( C2SPid ,
2015-04-08 17:12:05 +02:00
{ pep_message , < < ( ( Node ) ) / binary , " +notify " > > } ,
2015-11-24 16:44:13 +01:00
_ Sender = jid : make ( LUser , LServer , < < " " > > ) ,
2016-08-30 08:48:08 +02:00
_ StanzaToSend = add_extended_headers (
Stanza ,
_ ReplyTo = extended_headers ( [ Publisher ] ) ) ) ;
2008-12-08 15:10:55 +01:00
_ - >
2010-03-05 17:52:41 +01:00
? DEBUG ( " ~p @ ~p has no session; can't deliver ~p to contacts " , [ LUser , LServer , BaseStanza ] )
end ;
2015-04-08 17:12:05 +02:00
broadcast_stanza ( Host , _ Publisher , Node , Nidx , Type , NodeOptions , SubsByDepth , NotifyType , BaseStanza , SHIM ) - >
broadcast_stanza ( Host , Node , Nidx , Type , NodeOptions , SubsByDepth , NotifyType , BaseStanza , SHIM ) .
2005-07-20 05:09:34 +02:00
2009-08-27 23:38:23 +02:00
subscribed_nodes_by_jid ( NotifyType , SubsByDepth ) - >
2015-04-08 17:12:05 +02:00
NodesToDeliver = fun ( Depth , Node , Subs , Acc ) - >
2010-01-12 14:27:04 +01:00
NodeName = case Node #pubsub_node.nodeid of
{ _ , N } - > N ;
Other - > Other
end ,
2010-01-12 13:31:08 +01:00
NodeOptions = Node #pubsub_node.options ,
lists : foldl ( fun ( { LJID , SubID , SubOptions } , { JIDs , Recipients } ) - >
2015-04-08 17:12:05 +02:00
case is_to_deliver ( LJID , NotifyType , Depth , NodeOptions , SubOptions ) of
2015-10-07 00:06:58 +02:00
true - >
2015-04-08 17:12:05 +02:00
case state_can_deliver ( LJID , SubOptions ) of
2015-10-07 00:06:58 +02:00
[ ] - > { JIDs , Recipients } ;
2015-04-08 17:12:05 +02:00
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
2015-10-07 00:06:58 +02:00
{ [ JIDToDeliver | JIDsAcc ] ,
[ { JIDToDeliver , NodeName , [ SubID ] }
| RecipientsAcc ] } ;
2015-04-08 17:12:05 +02:00
true - >
%% - if the JIDs co-accumulator contains the Jid
%% get the tuple containing the Jid from the Recipient list co-accumulator
2015-10-07 00:06:58 +02:00
{ _ , { JIDToDeliver , NodeName1 , SubIDs } } =
lists : keysearch ( JIDToDeliver , 1 , RecipientsAcc ) ,
2015-04-08 17:12:05 +02:00
%% 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}
2015-10-07 00:06:58 +02:00
{ JIDsAcc ,
lists : keyreplace ( JIDToDeliver , 1 ,
RecipientsAcc ,
{ JIDToDeliver , NodeName1 ,
[ SubID | SubIDs ] } ) }
2015-04-08 17:12:05 +02:00
end
end , { JIDs , Recipients } , JIDsToDeliver )
end ;
false - >
{ JIDs , Recipients }
end
end , Acc , Subs )
end ,
2010-05-12 13:45:56 +02:00
DepthsToDeliver = fun ( { Depth , SubsByNode } , Acc1 ) - >
2010-01-12 13:31:08 +01:00
lists : foldl ( fun ( { Node , Subs } , Acc2 ) - >
2015-04-08 17:12:05 +02:00
NodesToDeliver ( Depth , Node , Subs , Acc2 )
end , Acc1 , SubsByNode )
end ,
2010-01-12 13:31:08 +01:00
{ _ , JIDSubs } = lists : foldl ( DepthsToDeliver , { [ ] , [ ] } , SubsByDepth ) ,
JIDSubs .
2009-06-15 15:45:40 +02:00
2016-08-30 08:48:08 +02:00
- spec user_resources ( binary ( ) , binary ( ) ) - > [ binary ( ) ] .
2009-03-03 23:42:37 +01:00
user_resources ( User , Server ) - >
2010-03-05 10:34:15 +01:00
ejabberd_sm : get_user_resources ( User , Server ) .
2007-12-22 01:04:05 +01:00
2016-08-30 08:48:08 +02:00
- spec user_resource ( binary ( ) , binary ( ) , binary ( ) ) - > binary ( ) .
2014-05-06 12:37:44 +02:00
user_resource ( User , Server , < < > > ) - >
case user_resources ( User , Server ) of
[ R | _ ] - > R ;
_ - > < < > >
end ;
2015-04-08 17:12:05 +02:00
user_resource ( _ , _ , Resource ) - >
Resource .
2014-05-06 12:37:44 +02:00
2007-12-01 06:16:30 +01:00
%%%%%%% Configuration handling
2016-08-30 08:48:08 +02:00
- spec get_configure ( host ( ) , binary ( ) , binary ( ) , jid ( ) ,
2016-09-08 16:08:48 +02:00
binary ( ) ) - > { error , stanza_error ( ) } | { result , pubsub_owner ( ) } .
2010-11-22 11:15:04 +01:00
get_configure ( Host , ServerHost , Node , From , Lang ) - >
2015-04-08 17:12:05 +02:00
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 ] ) ,
2016-08-30 08:48:08 +02:00
Fs = get_configure_xfields ( Type , Options , Lang , Groups ) ,
{ result , #pubsub_owner {
configure =
{ Node , #xdata { type = form , fields = Fs } } } } ;
2015-04-08 17:12:05 +02:00
_ - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_forbidden ( < < " Owner privileges required " > > , Lang ) }
2015-04-08 17:12:05 +02:00
end
end ,
2009-04-30 07:18:06 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , { _ , Result } } - > { result , Result } ;
Other - > Other
2009-04-30 07:18:06 +02:00
end .
2007-12-01 06:16:30 +01:00
2016-09-08 14:49:27 +02:00
- spec get_default ( host ( ) , binary ( ) , jid ( ) , binary ( ) ) - > { result , pubsub_owner ( ) } .
2008-09-25 19:26:06 +02:00
get_default ( Host , Node , _ From , Lang ) - >
2008-12-09 00:28:50 +01:00
Type = select_type ( Host , Host , Node ) ,
2015-04-08 17:12:05 +02:00
Options = node_options ( Host , Type ) ,
2016-08-30 08:48:08 +02:00
Fs = get_configure_xfields ( Type , Options , Lang , [ ] ) ,
{ result , #pubsub_owner { default = { < < > > , #xdata { type = form , fields = Fs } } } } .
2015-04-08 17:12:05 +02:00
2016-08-30 08:48:08 +02:00
- spec match_option ( #pubsub_node { } | [ { atom ( ) , any ( ) } ] , atom ( ) , any ( ) ) - > boolean ( ) .
2015-04-08 17:12:05 +02:00
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 ) - >
get_option ( Options , Var ) == Val ;
match_option ( _ , _ , _ ) - >
false .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec get_option ( [ { atom ( ) , any ( ) } ] , atom ( ) ) - > any ( ) .
2007-12-01 06:16:30 +01:00
get_option ( [ ] , _ ) - > false ;
2015-04-08 17:12:05 +02:00
get_option ( Options , Var ) - > get_option ( Options , Var , false ) .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec get_option ( [ { atom ( ) , any ( ) } ] , atom ( ) , any ( ) ) - > any ( ) .
2008-07-23 01:41:44 +02:00
get_option ( Options , Var , Def ) - >
2007-12-01 06:16:30 +01:00
case lists : keysearch ( Var , 1 , Options ) of
2015-04-08 17:12:05 +02:00
{ value , { _ Val , Ret } } - > Ret ;
_ - > Def
2007-12-01 06:16:30 +01:00
end .
2003-07-20 22:35:35 +02:00
2016-08-30 08:48:08 +02:00
- spec node_options ( host ( ) , binary ( ) ) - > [ { atom ( ) , any ( ) } ] .
2015-04-08 17:12:05 +02:00
node_options ( Host , Type ) - >
2016-03-13 11:16:43 +01:00
case config ( Host , default_node_config ) of
2015-11-10 16:28:57 +01:00
undefined - > node_plugin_options ( Host , Type ) ;
[ ] - > node_plugin_options ( Host , Type ) ;
Config - > Config
end .
2016-08-30 08:48:08 +02:00
- spec node_plugin_options ( host ( ) , binary ( ) ) - > [ { atom ( ) , any ( ) } ] .
2015-11-10 16:28:57 +01:00
node_plugin_options ( Host , Type ) - >
2015-04-08 17:12:05 +02:00
Module = plugin ( Host , Type ) ,
2007-12-01 06:16:30 +01:00
case catch Module : options ( ) of
2015-04-08 17:12:05 +02:00
{ 'EXIT' , { undef , _ } } - >
DefaultModule = plugin ( Host , ? STDNODE ) ,
DefaultModule : options ( ) ;
Result - >
Result
2005-07-20 05:09:34 +02:00
end .
2016-08-30 08:48:08 +02:00
- spec filter_node_options ( [ { atom ( ) , any ( ) } ] , [ { atom ( ) , any ( ) } ] ) - > [ { atom ( ) , any ( ) } ] .
2016-04-25 09:44:46 +02:00
filter_node_options ( Options , BaseOptions ) - >
2015-11-10 16:28:57 +01:00
lists : foldl ( fun ( { Key , Val } , Acc ) - >
DefaultValue = proplists : get_value ( Key , Options , Val ) ,
[ { Key , DefaultValue } | Acc ]
2016-04-25 09:44:46 +02:00
end , [ ] , BaseOptions ) .
2005-07-20 05:09:34 +02:00
2016-08-30 08:48:08 +02:00
- spec node_owners_action ( host ( ) , binary ( ) , nodeIdx ( ) , [ ljid ( ) ] ) - > [ ljid ( ) ] .
2015-04-08 17:12:05 +02:00
node_owners_action ( Host , Type , Nidx , [ ] ) - >
case gen_mod : db_type ( serverhost ( Host ) , ? MODULE ) of
2016-04-20 11:27:32 +02:00
sql - >
2015-04-08 17:12:05 +02:00
case node_action ( Host , Type , get_node_affiliations , [ Nidx ] ) of
{ result , Affs } - > [ LJID | | { LJID , Aff } < - Affs , Aff =:= owner ] ;
_ - > [ ]
end ;
_ - >
[ ]
end ;
node_owners_action ( _ Host , _ Type , _ Nidx , Owners ) - >
Owners .
2016-08-30 08:48:08 +02:00
- spec node_owners_call ( host ( ) , binary ( ) , nodeIdx ( ) , [ ljid ( ) ] ) - > [ ljid ( ) ] .
2015-04-08 17:12:05 +02:00
node_owners_call ( Host , Type , Nidx , [ ] ) - >
case gen_mod : db_type ( serverhost ( Host ) , ? MODULE ) of
2016-04-20 11:27:32 +02:00
sql - >
2015-04-08 17:12:05 +02:00
case node_call ( Host , Type , get_node_affiliations , [ Nidx ] ) of
{ result , Affs } - > [ LJID | | { LJID , Aff } < - Affs , Aff =:= owner ] ;
_ - > [ ]
end ;
_ - >
[ ]
end ;
node_owners_call ( _ Host , _ Type , _ Nidx , Owners ) - >
Owners .
2009-08-17 19:16:43 +02:00
%% @spec (Host, Options) -> MaxItems
2015-04-08 17:12:05 +02:00
%% Host = host()
%% Options = [Option]
%% Option = {Key::atom(), Value::term()}
%% MaxItems = integer() | unlimited
2007-12-01 06:16:30 +01:00
%% @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.
2016-08-30 08:48:08 +02:00
- spec max_items ( host ( ) , [ { atom ( ) , any ( ) } ] ) - > non_neg_integer ( ) .
2009-07-30 23:37:19 +02:00
max_items ( Host , Options ) - >
2007-12-01 06:16:30 +01:00
case get_option ( Options , persist_items ) of
2015-04-08 17:12:05 +02:00
true - >
case get_option ( Options , max_items ) of
2015-06-18 12:06:29 +02:00
I when is_integer ( I ) , I < 0 - > 0 ;
I when is_integer ( I ) - > I ;
_ - > ? MAXITEMS
2015-04-08 17:12:05 +02:00
end ;
false - >
case get_option ( Options , send_last_published_item ) of
never - >
0 ;
_ - >
case is_last_item_cache_enabled ( Host ) of
2015-06-18 12:54:46 +02:00
true - > 0 ;
false - > 1
2015-04-08 17:12:05 +02:00
end
end
2005-07-20 05:09:34 +02:00
end .
2016-10-07 09:31:03 +02:00
- spec get_configure_xfields ( _ , pubsub_node_config : result ( ) ,
binary ( ) , [ binary ( ) ] ) - > [ xdata_field ( ) ] .
2009-04-30 07:18:06 +02:00
get_configure_xfields ( _ Type , Options , Lang , Groups ) - >
2016-10-07 09:31:03 +02:00
pubsub_node_config : encode (
lists : map (
fun ( { roster_groups_allowed , Value } ) - >
{ roster_groups_allowed , Value , Groups } ;
( Opt ) - >
Opt
end , Options ) ,
fun ( Txt ) - > translate : translate ( Lang , Txt ) end ) .
2005-07-20 05:09:34 +02:00
2007-12-01 06:16:30 +01:00
%%<p>There are several reasons why the node configuration request might fail:</p>
%%<ul>
2008-04-01 12:11:39 +02:00
%%<li>The service does not support node configuration.</li>
%%<li>The requesting entity does not have sufficient privileges to configure the node.</li>
%%<li>The request did not specify a node.</li>
%%<li>The node has no configuration options.</li>
%%<li>The specified node does not exist.</li>
2007-12-01 06:16:30 +01:00
%%</ul>
2016-08-30 08:48:08 +02:00
- spec set_configure ( host ( ) , binary ( ) , jid ( ) , [ { binary ( ) , [ binary ( ) ] } ] ,
2016-09-08 16:08:48 +02:00
binary ( ) ) - > { result , undefined } | { error , stanza_error ( ) } .
2016-09-08 14:49:27 +02:00
set_configure ( _ Host , < < > > , _ From , _ Config , _ Lang ) - >
{ error , extended_error ( xmpp : err_bad_request ( ) , err_nodeid_required ( ) ) } ;
2016-08-30 08:48:08 +02:00
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 ,
2016-10-07 09:31:03 +02:00
NewOpts = merge_config ( Config , OldOpts ) ,
case tree_call ( Host ,
set_node ,
[ N #pubsub_node { options = NewOpts } ] ) of
{ result , Nidx } - > { result , ok } ;
ok - > { result , ok } ;
Err - > Err
2016-08-30 08:48:08 +02:00
end ;
_ - >
{ error , xmpp : err_forbidden (
< < " Owner privileges required " > > , Lang ) }
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 , undefined } ;
Other - >
Other
2005-07-20 05:09:34 +02:00
end .
2016-10-07 09:31:03 +02:00
- spec merge_config ( [ proplists : property ( ) ] , [ proplists : property ( ) ] ) - > [ proplists : property ( ) ] .
merge_config ( Config1 , Config2 ) - >
lists : foldl (
fun ( { Opt , Val } , Acc ) - >
lists : keystore ( Opt , 1 , Acc , { Opt , Val } )
end , Config2 , Config1 ) .
- 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 .
2005-07-20 05:09:34 +02:00
2016-10-07 09:31:03 +02:00
- 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 .
2005-07-20 05:09:34 +02:00
2016-10-07 09:31:03 +02:00
- spec decode_publish_options ( undefined | xdata ( ) , binary ( ) ) - >
pubsub_publish_options : result ( ) |
{ error , stanza_error ( ) } .
decode_publish_options ( undefined , _ ) - >
2016-08-30 08:48:08 +02:00
[ ] ;
2016-10-07 09:31:03 +02:00
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 ;
decode_get_pending ( undefined , Lang ) - >
{ error , xmpp : err_bad_request ( < < " No data form found " > > , Lang ) } .
- spec check_opt_range ( atom ( ) , [ proplists : property ( ) ] , non_neg_integer ( ) ) - > boolean ( ) .
check_opt_range ( Opt , Opts , Max ) - >
Val = proplists : get_value ( Opt , Opts , Max ) ,
Val =< Max .
2016-08-30 08:48:08 +02:00
- spec get_max_items_node ( host ( ) ) - > undefined | non_neg_integer ( ) .
2009-09-23 17:56:06 +02:00
get_max_items_node ( Host ) - >
2016-03-13 11:16:43 +01:00
config ( Host , max_items_node , undefined ) .
2007-12-01 06:16:30 +01:00
2016-08-30 08:48:08 +02:00
- spec get_max_subscriptions_node ( host ( ) ) - > undefined | non_neg_integer ( ) .
2015-11-10 16:28:57 +01:00
get_max_subscriptions_node ( Host ) - >
2016-03-13 11:16:43 +01:00
config ( Host , max_subscriptions_node , undefined ) .
2015-11-10 16:28:57 +01:00
2009-05-29 00:30:43 +02:00
%%%% last item cache handling
2016-08-30 08:48:08 +02:00
- spec is_last_item_cache_enabled ( host ( ) ) - > boolean ( ) .
2009-07-30 23:00:44 +02:00
is_last_item_cache_enabled ( Host ) - >
2016-03-13 11:16:43 +01:00
config ( Host , last_item_cache , false ) .
2009-07-30 23:00:44 +02:00
2016-08-30 08:48:08 +02:00
- spec set_cached_item ( host ( ) , nodeIdx ( ) , binary ( ) , binary ( ) , [ xmlel ( ) ] ) - > ok .
2015-04-08 17:12:05 +02:00
set_cached_item ( { _ , ServerHost , _ } , Nidx , ItemId , Publisher , Payload ) - >
set_cached_item ( ServerHost , Nidx , ItemId , Publisher , Payload ) ;
set_cached_item ( Host , Nidx , ItemId , Publisher , Payload ) - >
2009-07-30 23:00:44 +02:00
case is_last_item_cache_enabled ( Host ) of
2015-04-08 17:12:05 +02:00
true - > mnesia : dirty_write ( { pubsub_last_item , Nidx , ItemId ,
2015-12-07 16:16:11 +01:00
{ p1_time_compat : timestamp ( ) , jid : tolower ( jid : remove_resource ( Publisher ) ) } ,
2015-04-08 17:12:05 +02:00
Payload } ) ;
_ - > ok
2009-05-29 00:30:43 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec unset_cached_item ( host ( ) , nodeIdx ( ) ) - > ok .
2015-04-08 17:12:05 +02:00
unset_cached_item ( { _ , ServerHost , _ } , Nidx ) - >
unset_cached_item ( ServerHost , Nidx ) ;
unset_cached_item ( Host , Nidx ) - >
2009-07-30 23:00:44 +02:00
case is_last_item_cache_enabled ( Host ) of
2015-04-08 17:12:05 +02:00
true - > mnesia : dirty_delete ( { pubsub_last_item , Nidx } ) ;
_ - > ok
2009-05-29 00:30:43 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec get_cached_item ( host ( ) , nodeIdx ( ) ) - > undefined | pubsubItem ( ) .
2015-04-08 17:12:05 +02:00
get_cached_item ( { _ , ServerHost , _ } , Nidx ) - >
get_cached_item ( ServerHost , Nidx ) ;
get_cached_item ( Host , Nidx ) - >
2009-07-30 23:00:44 +02:00
case is_last_item_cache_enabled ( Host ) of
2015-04-08 17:12:05 +02:00
true - >
case mnesia : dirty_read ( { pubsub_last_item , 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 } ;
_ - >
undefined
end ;
_ - >
undefined
2009-05-29 00:30:43 +02:00
end .
2007-12-01 06:16:30 +01:00
%%%% plugin handling
2016-08-30 08:48:08 +02:00
- spec host ( binary ( ) ) - > binary ( ) .
2010-01-12 16:14:47 +01:00
host ( ServerHost ) - >
2015-04-08 17:12:05 +02:00
config ( ServerHost , host , < < " pubsub. " , ServerHost / binary > > ) .
2016-08-30 08:48:08 +02:00
- spec serverhost ( host ( ) ) - > binary ( ) .
2016-02-26 09:33:07 +01:00
serverhost ( { _ U , ServerHost , _ R } ) - >
2016-03-13 11:16:43 +01:00
serverhost ( ServerHost ) ;
2015-04-08 17:12:05 +02:00
serverhost ( Host ) - >
2016-03-13 11:16:43 +01:00
ejabberd_router : host_of_route ( Host ) .
2015-04-08 17:12:05 +02:00
2016-08-30 08:48:08 +02:00
- spec tree ( host ( ) ) - > atom ( ) .
2015-04-08 17:12:05 +02:00
tree ( Host ) - >
2016-12-25 01:24:40 +01:00
case config ( Host , nodetree ) of
2015-04-08 17:12:05 +02:00
undefined - > tree ( Host , ? STDTREE ) ;
Tree - > Tree
end .
2016-08-30 08:48:08 +02:00
- spec tree ( host ( ) , binary ( ) | atom ( ) ) - > atom ( ) .
2015-11-30 15:20:38 +01:00
tree ( _ Host , < < " virtual " > > ) - >
nodetree_virtual ; % special case, virtual does not use any backend
2015-04-08 17:12:05 +02:00
tree ( Host , Name ) - >
case gen_mod : db_type ( serverhost ( Host ) , ? MODULE ) of
mnesia - > jlib : binary_to_atom ( < < " nodetree_ " , Name / binary > > ) ;
2016-04-20 11:27:32 +02:00
sql - > jlib : binary_to_atom ( < < " nodetree_ " , Name / binary , " _sql " > > ) ;
2015-04-08 17:12:05 +02:00
_ - > Name
end .
2016-08-30 08:48:08 +02:00
- spec plugin ( host ( ) , binary ( ) | atom ( ) ) - > atom ( ) .
2015-04-08 17:12:05 +02:00
plugin ( Host , Name ) - >
case gen_mod : db_type ( serverhost ( Host ) , ? MODULE ) of
mnesia - > jlib : binary_to_atom ( < < " node_ " , Name / binary > > ) ;
2016-04-20 11:27:32 +02:00
sql - > jlib : binary_to_atom ( < < " node_ " , Name / binary , " _sql " > > ) ;
2015-04-08 17:12:05 +02:00
_ - > Name
2010-01-12 16:14:47 +01:00
end .
2016-08-30 08:48:08 +02:00
- spec plugins ( host ( ) ) - > [ binary ( ) ] .
2008-04-02 11:06:28 +02:00
plugins ( Host ) - >
2016-03-13 11:16:43 +01:00
case config ( Host , plugins ) of
2015-04-08 17:12:05 +02:00
undefined - > [ ? STDNODE ] ;
[ ] - > [ ? STDNODE ] ;
Plugins - > Plugins
end .
2016-08-30 08:48:08 +02:00
- spec subscription_plugin ( host ( ) ) - > pubsub_subscription |
pubsub_subscription_sql |
none .
2015-04-08 17:12:05 +02:00
subscription_plugin ( Host ) - >
case gen_mod : db_type ( serverhost ( Host ) , ? MODULE ) of
mnesia - > pubsub_subscription ;
2016-04-20 11:27:32 +02:00
sql - > pubsub_subscription_sql ;
2015-04-08 17:12:05 +02:00
_ - > none
end .
2016-08-30 08:48:08 +02:00
- spec config ( binary ( ) , any ( ) ) - > any ( ) .
2015-04-08 17:12:05 +02:00
config ( ServerHost , Key ) - >
config ( ServerHost , Key , undefined ) .
2016-03-13 11:16:43 +01:00
2016-08-30 08:48:08 +02:00
- spec config ( host ( ) , any ( ) , any ( ) ) - > any ( ) .
2016-03-13 11:16:43 +01:00
config ( { _ User , Host , _ Resource } , Key , Default ) - >
config ( Host , Key , Default ) ;
2015-04-08 17:12:05 +02:00
config ( ServerHost , Key , Default ) - >
case catch ets : lookup ( gen_mod : get_module_proc ( ServerHost , config ) , Key ) of
[ { Key , Value } ] - > Value ;
_ - > Default
2008-04-02 11:06:28 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec select_type ( binary ( ) , host ( ) , binary ( ) , binary ( ) ) - > binary ( ) .
2013-03-14 10:33:02 +01:00
select_type ( ServerHost , Host , Node , Type ) - >
2009-05-29 00:30:43 +02:00
SelectedType = case Host of
2015-04-08 17:12:05 +02:00
{ _ User , _ Server , _ Resource } - >
case config ( ServerHost , pep_mapping ) of
undefined - > ? PEPNODE ;
Mapping - > proplists : get_value ( Node , Mapping , ? PEPNODE )
end ;
_ - >
Type
end ,
2016-02-26 09:33:07 +01:00
ConfiguredTypes = plugins ( Host ) ,
2009-05-29 00:30:43 +02:00
case lists : member ( SelectedType , ConfiguredTypes ) of
2015-04-08 17:12:05 +02:00
true - > SelectedType ;
false - > hd ( ConfiguredTypes )
2008-09-25 19:26:06 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec select_type ( binary ( ) , host ( ) , binary ( ) ) - > binary ( ) .
2013-03-14 10:33:02 +01:00
select_type ( ServerHost , Host , Node ) - >
2016-02-26 09:33:07 +01:00
select_type ( ServerHost , Host , Node , hd ( plugins ( Host ) ) ) .
2015-04-08 17:12:05 +02:00
2016-09-08 14:49:27 +02:00
- spec feature ( binary ( ) ) - > binary ( ) .
2015-04-08 17:12:05 +02:00
feature ( < < " rsm " > > ) - > ? NS_RSM ;
feature ( Feature ) - > < < ( ? NS_PUBSUB ) / binary , " # " , Feature / binary > > .
2008-04-02 11:06:28 +02:00
2016-09-08 14:49:27 +02:00
- spec features ( ) - > [ binary ( ) ] .
2007-12-01 06:16:30 +01:00
features ( ) - >
2013-03-14 10:33:02 +01:00
[ % see plugin "access-authorize", % OPTIONAL
2016-09-08 14:49:27 +02:00
< < " 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
2015-04-08 17:12:05 +02:00
% see plugin "retrieve-items", % RECOMMENDED
% see plugin "retrieve-subscriptions", % RECOMMENDED
% see plugin "subscribe", % REQUIRED
% see plugin "subscription-options", % OPTIONAL
% see plugin "subscription-notifications" % OPTIONAL
2016-08-30 08:48:08 +02:00
- spec plugin_features ( binary ( ) , binary ( ) ) - > [ binary ( ) ] .
2015-04-08 17:12:05 +02:00
plugin_features ( Host , Type ) - >
Module = plugin ( Host , Type ) ,
case catch Module : features ( ) of
2013-03-14 10:33:02 +01:00
{ 'EXIT' , { undef , _ } } - > [ ] ;
Result - > Result
2015-04-08 17:12:05 +02:00
end .
2013-03-14 10:33:02 +01:00
2016-08-30 08:48:08 +02:00
- spec features ( binary ( ) , binary ( ) ) - > [ binary ( ) ] .
2009-11-04 20:54:15 +01:00
features ( Host , < < > > ) - >
2013-03-14 10:33:02 +01:00
lists : usort ( lists : foldl ( fun ( Plugin , Acc ) - >
2015-04-08 17:12:05 +02:00
Acc ++ plugin_features ( Host , Plugin )
end ,
features ( ) , plugins ( Host ) ) ) ;
features ( Host , Node ) when is_binary ( Node ) - >
2013-03-14 10:33:02 +01:00
Action = fun ( #pubsub_node { type = Type } ) - >
2015-04-08 17:12:05 +02:00
{ result , plugin_features ( Host , Type ) }
end ,
2009-04-30 07:18:06 +02:00
case transaction ( Host , Node , Action , sync_dirty ) of
2015-04-08 17:12:05 +02:00
{ result , Features } - > lists : usort ( features ( ) ++ Features ) ;
_ - > features ( )
2009-04-30 07:18:06 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @doc <p>node tree plugin call.</p>
tree_call ( { _ User , Server , _ Resource } , Function , Args ) - >
tree_call ( Server , Function , Args ) ;
tree_call ( Host , Function , Args ) - >
2016-04-05 12:09:44 +02:00
Tree = tree ( Host ) ,
? DEBUG ( " tree_call apply( ~s , ~s , ~p ) @ ~s " , [ Tree , Function , Args , Host ] ) ,
catch apply ( Tree , Function , Args ) .
2013-03-14 10:33:02 +01:00
2007-12-01 06:16:30 +01:00
tree_action ( Host , Function , Args ) - >
2013-03-14 10:33:02 +01:00
? DEBUG ( " tree_action ~p ~p ~p " , [ Host , Function , Args ] ) ,
2015-04-08 17:12:05 +02:00
ServerHost = serverhost ( Host ) ,
2013-03-14 10:33:02 +01:00
Fun = fun ( ) - > tree_call ( Host , Function , Args ) end ,
2015-04-08 17:12:05 +02:00
case gen_mod : db_type ( ServerHost , ? MODULE ) of
mnesia - >
catch mnesia : sync_dirty ( Fun ) ;
2016-04-20 11:27:32 +02:00
sql - >
case catch ejabberd_sql : sql_bloc ( ServerHost , Fun ) of
2015-04-08 17:12:05 +02:00
{ atomic , Result } - >
Result ;
{ aborted , Reason } - >
? ERROR_MSG ( " transaction return internal error: ~p ~n " , [ { aborted , Reason } ] ) ,
2016-08-30 08:48:08 +02:00
ErrTxt = < < " Database failure " > > ,
{ error , xmpp : err_internal_server_error ( ErrTxt , ? MYLANG ) }
2015-04-08 17:12:05 +02:00
end ;
Other - >
? ERROR_MSG ( " unsupported backend: ~p ~n " , [ Other ] ) ,
2016-08-30 08:48:08 +02:00
ErrTxt = < < " Database failure " > > ,
{ error , xmpp : err_internal_server_error ( ErrTxt , ? MYLANG ) }
2015-04-08 17:12:05 +02:00
end .
2007-12-01 06:16:30 +01:00
%% @doc <p>node plugin call.</p>
2015-04-08 17:12:05 +02:00
node_call ( Host , Type , Function , Args ) - >
2013-03-14 10:33:02 +01:00
? DEBUG ( " node_call ~p ~p ~p " , [ Type , Function , Args ] ) ,
2015-04-08 17:12:05 +02:00
Module = plugin ( Host , Type ) ,
2010-11-08 11:16:51 +01:00
case apply ( Module , Function , Args ) of
2015-04-08 17:12:05 +02:00
{ result , Result } - >
{ result , Result } ;
{ error , Error } - >
{ error , Error } ;
{ 'EXIT' , { undef , Undefined } } - >
case Type of
? STDNODE - > { error , { undef , Undefined } } ;
_ - > node_call ( Host , ? STDNODE , Function , Args )
end ;
{ 'EXIT' , Reason } - >
{ error , Reason } ;
Result - >
{ result , Result } %% any other return value is forced as result
2005-07-20 05:09:34 +02:00
end .
2009-05-26 23:50:13 +02:00
node_action ( Host , Type , Function , Args ) - >
2015-04-08 17:12:05 +02:00
? DEBUG ( " node_action ~p ~p ~p ~p " , [ Host , Type , Function , Args ] ) ,
transaction ( Host , fun ( ) - >
node_call ( Host , Type , Function , Args )
end ,
sync_dirty ) .
2007-12-01 06:16:30 +01:00
%% @doc <p>plugin transaction handling.</p>
transaction ( Host , Node , Action , Trans ) - >
2015-04-08 17:12:05 +02:00
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 , Trans ) - >
ServerHost = serverhost ( Host ) ,
DBType = gen_mod : db_type ( ServerHost , ? MODULE ) ,
Retry = case DBType of
2016-04-20 11:27:32 +02:00
sql - > 2 ;
2015-04-08 17:12:05 +02:00
_ - > 1
end ,
transaction_retry ( Host , ServerHost , Fun , Trans , DBType , Retry ) .
transaction_retry ( _ Host , _ ServerHost , _ Fun , _ Trans , _ DBType , 0 ) - >
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_internal_server_error ( < < " Database failure " > > , ? MYLANG ) } ;
2015-04-08 17:12:05 +02:00
transaction_retry ( Host , ServerHost , Fun , Trans , DBType , Count ) - >
Res = case DBType of
mnesia - >
catch mnesia : Trans ( Fun ) ;
2016-04-20 11:27:32 +02:00
sql - >
2015-04-08 17:12:05 +02:00
SqlFun = case Trans of
transaction - > sql_transaction ;
_ - > sql_bloc
end ,
2016-04-20 11:27:32 +02:00
catch ejabberd_sql : SqlFun ( ServerHost , Fun ) ;
2015-04-08 17:12:05 +02:00
_ - >
{ 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 } ] ) ,
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_internal_server_error ( < < " Database failure " > > , ? MYLANG ) } ;
2015-04-08 17:12:05 +02:00
{ '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 } ] ) ,
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_internal_server_error ( < < " Database failure " > > , ? MYLANG ) } ;
2015-04-08 17:12:05 +02:00
Other - >
? ERROR_MSG ( " transaction return internal error: ~p ~n " , [ Other ] ) ,
2016-08-30 08:48:08 +02:00
{ error , xmpp : err_internal_server_error ( < < " Database failure " > > , ? MYLANG ) }
2005-04-17 20:08:34 +02:00
end .
2007-12-01 06:16:30 +01:00
%%%% helpers
%% Add pubsub-specific error element
2016-09-08 16:08:48 +02:00
- spec extended_error ( stanza_error ( ) , ps_error ( ) ) - > stanza_error ( ) .
2016-08-30 08:48:08 +02:00
extended_error ( StanzaErr , PubSubErr ) - >
2016-09-08 16:08:48 +02:00
StanzaErr #stanza_error { sub_els = [ PubSubErr ] } .
2016-04-05 12:09:44 +02:00
2016-08-30 08:48:08 +02:00
- spec err_closed_node ( ) - > ps_error ( ) .
err_closed_node ( ) - >
#ps_error { type = 'closed-node' } .
- 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' } .
- spec err_invalid_payload ( ) - > ps_error ( ) .
err_invalid_payload ( ) - >
#ps_error { type = 'invalid-payload' } .
- spec err_invalid_subid ( ) - > ps_error ( ) .
err_invalid_subid ( ) - >
#ps_error { type = 'invalid-subid' } .
- spec err_item_forbidden ( ) - > ps_error ( ) .
err_item_forbidden ( ) - >
#ps_error { type = 'item-forbidden' } .
- spec err_item_required ( ) - > ps_error ( ) .
err_item_required ( ) - >
#ps_error { type = 'item-required' } .
- 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_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' } .
2016-09-08 14:49:27 +02:00
- spec err_unsupported ( ps_feature ( ) ) - > ps_error ( ) .
2016-08-30 08:48:08 +02:00
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' } .
2013-03-14 10:33:02 +01:00
2016-07-01 21:18:55 +02:00
- spec uniqid ( ) - > mod_pubsub : itemId ( ) .
2007-12-01 06:16:30 +01:00
uniqid ( ) - >
2015-12-07 16:16:11 +01:00
{ T1 , T2 , T3 } = p1_time_compat : timestamp ( ) ,
2016-11-24 13:06:06 +01:00
( str : format ( " ~.16B ~.16B ~.16B " , [ T1 , T2 , T3 ] ) ) .
2008-02-06 19:04:23 +01:00
2016-08-30 08:48:08 +02:00
- spec itemsEls ( [ #pubsub_item { } ] ) - > [ ps_item ( ) ] .
2009-04-08 17:53:46 +02:00
itemsEls ( Items ) - >
2016-08-30 08:48:08 +02:00
[ #ps_item { id = ItemId , xml_els = Payload }
| | #pubsub_item { itemid = { ItemId , _ } , payload = Payload } < - Items ] .
2009-06-15 15:45:40 +02:00
2016-08-30 08:48:08 +02:00
- spec add_message_type ( message ( ) , message_type ( ) ) - > message ( ) .
add_message_type ( #message { } = Message , Type ) - >
Message #message { type = Type } .
2009-09-24 21:28:43 +02:00
2010-01-12 11:33:39 +01:00
%% Place of <headers/> changed at the bottom of the stanza
%% cf. http://xmpp.org/extensions/xep-0060.html#publisher-publish-success-subid
%%
%% "[SHIM Headers] SHOULD be included after the event notification information
%% (i.e., as the last child of the <message/> stanza)".
2016-08-30 08:48:08 +02:00
- spec add_shim_headers ( stanza ( ) , [ { binary ( ) , binary ( ) } ] ) - > stanza ( ) .
add_shim_headers ( Stanza , Headers ) - >
xmpp : set_subtag ( Stanza , #shim { headers = Headers } ) .
2010-03-05 17:52:41 +01:00
2016-08-30 08:48:08 +02:00
- spec add_extended_headers ( stanza ( ) , [ address ( ) ] ) - > stanza ( ) .
add_extended_headers ( Stanza , Addrs ) - >
xmpp : set_subtag ( Stanza , #addresses { list = Addrs } ) .
2010-01-12 11:33:39 +01:00
2016-08-30 08:48:08 +02:00
- spec subid_shim ( [ binary ( ) ] ) - > [ { binary ( ) , binary ( ) } ] .
2015-04-08 17:12:05 +02:00
subid_shim ( SubIds ) - >
2016-08-30 08:48:08 +02:00
[ { < < " SubId " > > , SubId } | | SubId < - SubIds ] .
2010-01-12 11:33:39 +01:00
2010-03-05 17:52:41 +01:00
%% The argument is a list of Jids because this function could be used
%% with the 'pubsub#replyto' (type=jid-multi) node configuration.
2016-08-30 08:48:08 +02:00
- spec extended_headers ( [ jid ( ) ] ) - > [ address ( ) ] .
2010-03-05 17:52:41 +01:00
extended_headers ( Jids ) - >
2016-08-30 08:48:08 +02:00
[ #address { type = replyto , jid = Jid } | | Jid < - Jids ] .
2010-03-05 17:52:41 +01:00
2016-08-09 09:56:32 +02:00
- spec on_user_offline ( ejabberd_sm : sid ( ) , jid ( ) , ejabberd_sm : info ( ) ) - > ok .
2010-03-05 11:11:44 +01:00
on_user_offline ( _ , JID , _ ) - >
2015-11-24 16:44:13 +01:00
{ User , Server , Resource } = jid : tolower ( JID ) ,
2015-04-08 17:12:05 +02:00
case user_resources ( User , Server ) of
2010-03-05 11:35:20 +01:00
[ ] - > purge_offline ( { User , Server , Resource } ) ;
2016-08-09 09:56:32 +02:00
_ - > ok
2010-03-05 11:35:20 +01:00
end .
2010-03-05 11:11:44 +01:00
2016-08-30 08:48:08 +02:00
- spec purge_offline ( ljid ( ) ) - > ok .
2015-04-22 18:11:39 +02:00
purge_offline ( LJID ) - >
2010-03-05 11:35:20 +01:00
Host = host ( element ( 2 , LJID ) ) ,
Plugins = plugins ( Host ) ,
2013-03-14 10:33:02 +01:00
Result = lists : foldl ( fun ( Type , { Status , Acc } ) - >
2015-04-22 18:11:39 +02:00
Features = plugin_features ( Host , Type ) ,
2015-04-08 17:12:05 +02:00
case lists : member ( < < " retrieve-affiliations " > > , plugin_features ( Host , Type ) ) of
false - >
2016-08-30 08:48:08 +02:00
{ { error , extended_error ( xmpp : err_feature_not_implemented ( ) ,
err_unsupported ( 'retrieve-affiliations' ) ) } ,
2015-04-08 17:12:05 +02:00
Acc } ;
true - >
2015-04-22 18:11:39 +02:00
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
2015-04-08 17:12:05 +02:00
end
end ,
{ ok , [ ] } , Plugins ) ,
2010-03-05 11:35:20 +01:00
case Result of
2015-04-08 17:12:05 +02:00
{ ok , Affs } - >
2015-04-22 18:11:39 +02:00
lists : foreach (
fun ( { Node , Affiliation } ) - >
Options = Node #pubsub_node.options ,
Publisher = lists : member ( Affiliation , [ owner , publisher , publish_only ] ) ,
Open = ( get_option ( Options , publish_model ) == open ) ,
Purge = ( get_option ( Options , purge_offline )
andalso get_option ( Options , persist_items ) ) ,
if ( Publisher or Open ) and Purge - >
purge_offline ( Host , LJID , Node ) ;
true - >
ok
end
end , lists : usort ( lists : flatten ( Affs ) ) ) ;
{ Error , _ } - >
? DEBUG ( " on_user_offline ~p " , [ Error ] )
end .
2016-09-08 16:08:48 +02:00
- spec purge_offline ( host ( ) , ljid ( ) , binary ( ) ) - > ok | { error , stanza_error ( ) } .
2015-04-22 18:11:39 +02:00
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
{ result , { [ ] , _ } } - >
ok ;
{ result , { Items , _ } } - >
{ User , Server , _ } = LJID ,
PublishModel = get_option ( Options , publish_model ) ,
ForceNotify = get_option ( Options , notify_retract ) ,
{ _ , NodeId } = Node #pubsub_node.nodeid ,
2015-04-08 17:12:05 +02:00
lists : foreach ( fun
2015-04-22 18:11:39 +02:00
( #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
2015-04-08 17:12:05 +02:00
end ;
2015-04-22 18:11:39 +02:00
{ result , _ } - >
ok ;
2015-04-08 17:12:05 +02:00
Error - >
Error
end ;
( _ ) - >
true
2015-04-22 18:11:39 +02:00
end , Items ) ;
Error - >
Error
2010-03-05 11:35:20 +01:00
end .
2015-06-01 14:38:27 +02:00
2016-11-15 18:35:20 +01:00
mod_opt_type ( access_createnode ) - > fun acl : access_rules_validator / 1 ;
2016-04-27 16:10:50 +02:00
mod_opt_type ( db_type ) - > fun ( T ) - > ejabberd_config : v_db ( ? MODULE , T ) end ;
2015-06-01 14:38:27 +02:00
mod_opt_type ( host ) - > fun iolist_to_binary / 1 ;
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 ;
mod_opt_type ( last_item_cache ) - >
fun ( A ) when is_boolean ( A ) - > A end ;
mod_opt_type ( max_items_node ) - >
fun ( A ) when is_integer ( A ) andalso A > = 0 - > A end ;
2015-11-10 16:28:57 +01:00
mod_opt_type ( max_subscriptions_node ) - >
fun ( A ) when is_integer ( A ) andalso A > = 0 - > A end ;
mod_opt_type ( default_node_config ) - >
fun ( A ) when is_list ( A ) - > A end ;
2015-06-01 14:38:27 +02:00
mod_opt_type ( nodetree ) - >
fun ( A ) when is_binary ( A ) - > A end ;
mod_opt_type ( pep_mapping ) - >
fun ( A ) when is_list ( A ) - > A end ;
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 ,
2015-11-10 16:28:57 +01:00
max_items_node , nodetree , pep_mapping , plugins ,
max_subscriptions_node , default_node_config ] .