aboutsummaryrefslogtreecommitdiff
path: root/src/mod_pubsub.erl
diff options
context:
space:
mode:
Diffstat (limited to 'src/mod_pubsub.erl')
-rw-r--r--src/mod_pubsub.erl353
1 files changed, 205 insertions, 148 deletions
diff --git a/src/mod_pubsub.erl b/src/mod_pubsub.erl
index 616c64929..e42d5c058 100644
--- a/src/mod_pubsub.erl
+++ b/src/mod_pubsub.erl
@@ -63,7 +63,7 @@
%% exports for console debug manual use
-export([create_node/5, create_node/7, delete_node/3,
subscribe_node/5, unsubscribe_node/5, publish_item/6,
- delete_item/4, send_items/7, get_items/2, get_item/3,
+ delete_item/4, delete_item/5, send_items/7, get_items/2, get_item/3,
get_cached_item/2, get_configure/5, set_configure/5,
tree_action/3, node_action/4, node_call/4]).
@@ -241,6 +241,7 @@ stop(Host) ->
init([ServerHost, Opts]) ->
?DEBUG("pubsub init ~p ~p", [ServerHost, Opts]),
Host = gen_mod:get_opt_host(ServerHost, Opts, <<"pubsub.@HOST@">>),
+ ejabberd_router:register_route(Host, ServerHost),
Access = gen_mod:get_opt(access_createnode, Opts,
fun(A) when is_atom(A) -> A end, all),
PepOffline = gen_mod:get_opt(ignore_pep_from_offline, Opts,
@@ -256,22 +257,26 @@ init([ServerHost, Opts]) ->
DefaultNodeCfg = gen_mod:get_opt(default_node_config, Opts,
fun(A) when is_list(A) -> filter_node_options(A) end, []),
pubsub_index:init(Host, ServerHost, Opts),
- ets:new(gen_mod:get_module_proc(ServerHost, config), [set, named_table]),
{Plugins, NodeTree, PepMapping} = init_plugins(Host, ServerHost, Opts),
mnesia:create_table(pubsub_last_item,
[{ram_copies, [node()]},
{attributes, record_info(fields, pubsub_last_item)}]),
mod_disco:register_feature(ServerHost, ?NS_PUBSUB),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {nodetree, NodeTree}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {plugins, Plugins}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {last_item_cache, LastItemCache}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {max_items_node, MaxItemsNode}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {max_subscriptions_node, MaxSubsNode}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {default_node_config, DefaultNodeCfg}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {pep_mapping, PepMapping}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {ignore_pep_from_offline, PepOffline}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {host, Host}),
- ets:insert(gen_mod:get_module_proc(ServerHost, config), {access, Access}),
+ lists:foreach(
+ fun(H) ->
+ T = gen_mod:get_module_proc(H, config),
+ ets:new(T, [set, named_table]),
+ ets:insert(T, {nodetree, NodeTree}),
+ ets:insert(T, {plugins, Plugins}),
+ ets:insert(T, {last_item_cache, LastItemCache}),
+ ets:insert(T, {max_items_node, MaxItemsNode}),
+ ets:insert(T, {max_subscriptions_node, MaxSubsNode}),
+ ets:insert(T, {default_node_config, DefaultNodeCfg}),
+ ets:insert(T, {pep_mapping, PepMapping}),
+ ets:insert(T, {ignore_pep_from_offline, PepOffline}),
+ ets:insert(T, {host, Host}),
+ ets:insert(T, {access, Access})
+ end, [Host, ServerHost]),
ejabberd_hooks:add(sm_remove_connection_hook, ServerHost,
?MODULE, on_user_offline, 75),
ejabberd_hooks:add(disco_local_identity, ServerHost,
@@ -309,7 +314,6 @@ init([ServerHost, Opts]) ->
false ->
ok
end,
- ejabberd_router:register_route(Host),
pubsub_migrate:update_node_database(Host, ServerHost),
pubsub_migrate:update_state_database(Host, ServerHost),
pubsub_migrate:update_lastitem_database(Host, ServerHost),
@@ -482,7 +486,7 @@ send_loop(State) ->
-> [xmlel()]
).
disco_local_identity(Acc, _From, To, <<>>, _Lang) ->
- case lists:member(?PEPNODE, plugins(To#jid.lserver)) of
+ case lists:member(?PEPNODE, plugins(host(To#jid.lserver))) of
true ->
[#xmlel{name = <<"identity">>,
attrs = [{<<"category">>, <<"pubsub">>},
@@ -504,7 +508,7 @@ disco_local_identity(Acc, _From, _To, _Node, _Lang) ->
-> [binary(),...]
).
disco_local_features(Acc, _From, To, <<>>, _Lang) ->
- Host = To#jid.lserver,
+ Host = host(To#jid.lserver),
Feats = case Acc of
{result, I} -> I;
_ -> []
@@ -873,7 +877,6 @@ handle_info(_Info, State) ->
%% @private
terminate(_Reason,
#state{host = Host, server_host = ServerHost, nodetree = TreePlugin, plugins = Plugins}) ->
- ejabberd_router:unregister_route(Host),
case lists:member(?PEPNODE, Plugins) of
true ->
ejabberd_hooks:delete(caps_add, ServerHost,
@@ -918,7 +921,8 @@ terminate(_Reason,
Pid ->
Pid ! stop
end,
- terminate_plugins(Host, ServerHost, Plugins, TreePlugin).
+ terminate_plugins(Host, ServerHost, Plugins, TreePlugin),
+ ejabberd_router:unregister_route(Host).
%%--------------------------------------------------------------------
%% Func: code_change(OldVsn, State, Extra) -> {ok, NewState}
@@ -951,7 +955,7 @@ do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
case jlib:iq_query_info(Packet) of
#iq{type = get, xmlns = ?NS_DISCO_INFO, sub_el = SubEl, lang = Lang} = IQ ->
#xmlel{attrs = QAttrs} = SubEl,
- Node = xml:get_attr_s(<<"node">>, QAttrs),
+ Node = fxml:get_attr_s(<<"node">>, QAttrs),
Info = ejabberd_hooks:run_fold(disco_info, ServerHost,
[],
[ServerHost, ?MODULE, <<>>, <<>>]),
@@ -968,7 +972,7 @@ do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
ejabberd_router:route(To, From, Res);
#iq{type = get, xmlns = ?NS_DISCO_ITEMS, sub_el = SubEl} = IQ ->
#xmlel{attrs = QAttrs} = SubEl,
- Node = xml:get_attr_s(<<"node">>, QAttrs),
+ Node = fxml:get_attr_s(<<"node">>, QAttrs),
Res = case iq_disco_items(Host, Node, From, jlib:rsm_decode(IQ)) of
{result, IQRes} ->
jlib:iq_to_xml(IQ#iq{type = result,
@@ -1022,7 +1026,7 @@ do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
ok
end;
<<"message">> ->
- case xml:get_attr_s(<<"type">>, Attrs) of
+ case fxml:get_attr_s(<<"type">>, Attrs) of
<<"error">> ->
ok;
_ ->
@@ -1030,7 +1034,10 @@ do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
none ->
ok;
invalid ->
- Err = jlib:make_error_reply(Packet, ?ERR_BAD_REQUEST),
+ Lang = fxml:get_attr_s(<<"xml:lang">>, Attrs),
+ Txt = <<"Incorrect authorization response">>,
+ Err = jlib:make_error_reply(
+ Packet, ?ERRT_BAD_REQUEST(Lang, Txt)),
ejabberd_router:route(To, From, Err);
XFields ->
handle_authorization_response(Host, From, To, Packet, XFields)
@@ -1040,7 +1047,7 @@ do_route(ServerHost, Access, Plugins, Host, From, To, Packet) ->
ok
end;
_ ->
- case xml:get_attr_s(<<"type">>, Attrs) of
+ case fxml:get_attr_s(<<"type">>, Attrs) of
<<"error">> ->
ok;
<<"result">> ->
@@ -1236,7 +1243,7 @@ iq_get_vcard(Lang) ->
).
iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang) ->
- iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, all, plugins(ServerHost)).
+ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, all, plugins(Host)).
-spec(iq_pubsub/8 ::
(
@@ -1255,16 +1262,16 @@ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang) ->
iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
#xmlel{children = SubEls} = SubEl,
- case xml:remove_cdata(SubEls) of
+ case fxml:remove_cdata(SubEls) of
[#xmlel{name = Name, attrs = Attrs, children = Els} | Rest] ->
- Node = xml:get_attr_s(<<"node">>, Attrs),
+ Node = fxml:get_attr_s(<<"node">>, Attrs),
case {IQType, Name} of
{set, <<"create">>} ->
Config = case Rest of
[#xmlel{name = <<"configure">>, children = C}] -> C;
_ -> []
end,
- Type = case xml:get_attr_s(<<"type">>, Attrs) of
+ Type = case fxml:get_attr_s(<<"type">>, Attrs) of
<<>> -> hd(Plugins);
T -> T
end,
@@ -1276,10 +1283,10 @@ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
create_node(Host, ServerHost, Node, From, Type, Access, Config)
end;
{set, <<"publish">>} ->
- case xml:remove_cdata(Els) of
+ case fxml:remove_cdata(Els) of
[#xmlel{name = <<"item">>, attrs = ItemAttrs,
children = Payload}] ->
- ItemId = xml:get_attr_s(<<"id">>, ItemAttrs),
+ ItemId = fxml:get_attr_s(<<"id">>, ItemAttrs),
publish_item(Host, ServerHost, Node, From, ItemId, Payload, Access);
[] ->
{error,
@@ -1289,14 +1296,14 @@ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
extended_error(?ERR_BAD_REQUEST, <<"invalid-payload">>)}
end;
{set, <<"retract">>} ->
- ForceNotify = case xml:get_attr_s(<<"notify">>, Attrs) of
+ ForceNotify = case fxml:get_attr_s(<<"notify">>, Attrs) of
<<"1">> -> true;
<<"true">> -> true;
_ -> false
end,
- case xml:remove_cdata(Els) of
+ case fxml:remove_cdata(Els) of
[#xmlel{name = <<"item">>, attrs = ItemAttrs}] ->
- ItemId = xml:get_attr_s(<<"id">>, ItemAttrs),
+ ItemId = fxml:get_attr_s(<<"id">>, ItemAttrs),
delete_item(Host, Node, From, ItemId, ForceNotify);
_ ->
{error,
@@ -1307,37 +1314,37 @@ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
[#xmlel{name = <<"options">>, children = C}] -> C;
_ -> []
end,
- JID = xml:get_attr_s(<<"jid">>, Attrs),
+ JID = fxml:get_attr_s(<<"jid">>, Attrs),
subscribe_node(Host, Node, From, JID, Config);
{set, <<"unsubscribe">>} ->
- JID = xml:get_attr_s(<<"jid">>, Attrs),
- SubId = xml:get_attr_s(<<"subid">>, Attrs),
+ JID = fxml:get_attr_s(<<"jid">>, Attrs),
+ SubId = fxml:get_attr_s(<<"subid">>, Attrs),
unsubscribe_node(Host, Node, From, JID, SubId);
{get, <<"items">>} ->
- MaxItems = xml:get_attr_s(<<"max_items">>, Attrs),
- SubId = xml:get_attr_s(<<"subid">>, Attrs),
+ MaxItems = fxml:get_attr_s(<<"max_items">>, Attrs),
+ SubId = fxml:get_attr_s(<<"subid">>, Attrs),
ItemIds = lists:foldl(fun
(#xmlel{name = <<"item">>, attrs = ItemAttrs}, Acc) ->
- case xml:get_attr_s(<<"id">>, ItemAttrs) of
+ case fxml:get_attr_s(<<"id">>, ItemAttrs) of
<<>> -> Acc;
ItemId -> [ItemId | Acc]
end;
(_, Acc) ->
Acc
end,
- [], xml:remove_cdata(Els)),
+ [], fxml:remove_cdata(Els)),
get_items(Host, Node, From, SubId, MaxItems, ItemIds, jlib:rsm_decode(SubEl));
{get, <<"subscriptions">>} ->
get_subscriptions(Host, Node, From, Plugins);
{get, <<"affiliations">>} ->
get_affiliations(Host, Node, From, Plugins);
{get, <<"options">>} ->
- SubId = xml:get_attr_s(<<"subid">>, Attrs),
- JID = xml:get_attr_s(<<"jid">>, Attrs),
+ SubId = fxml:get_attr_s(<<"subid">>, Attrs),
+ JID = fxml:get_attr_s(<<"jid">>, Attrs),
get_options(Host, Node, JID, SubId, Lang);
{set, <<"options">>} ->
- SubId = xml:get_attr_s(<<"subid">>, Attrs),
- JID = xml:get_attr_s(<<"jid">>, Attrs),
+ SubId = fxml:get_attr_s(<<"subid">>, Attrs),
+ JID = fxml:get_attr_s(<<"jid">>, Attrs),
set_options(Host, Node, JID, SubId, Els);
_ ->
{error, ?ERR_FEATURE_NOT_IMPLEMENTED}
@@ -1362,10 +1369,10 @@ iq_pubsub(Host, ServerHost, From, IQType, SubEl, Lang, Access, Plugins) ->
).
iq_pubsub_owner(Host, ServerHost, From, IQType, SubEl, Lang) ->
#xmlel{children = SubEls} = SubEl,
- Action = xml:remove_cdata(SubEls),
+ Action = fxml:remove_cdata(SubEls),
case Action of
[#xmlel{name = Name, attrs = Attrs, children = Els}] ->
- Node = xml:get_attr_s(<<"node">>, Attrs),
+ Node = fxml:get_attr_s(<<"node">>, Attrs),
case {IQType, Name} of
{get, <<"configure">>} ->
get_configure(Host, ServerHost, Node, From, Lang);
@@ -1380,11 +1387,11 @@ iq_pubsub_owner(Host, ServerHost, From, IQType, SubEl, Lang) ->
{get, <<"subscriptions">>} ->
get_subscriptions(Host, Node, From);
{set, <<"subscriptions">>} ->
- set_subscriptions(Host, Node, From, xml:remove_cdata(Els));
+ set_subscriptions(Host, Node, From, fxml:remove_cdata(Els));
{get, <<"affiliations">>} ->
get_affiliations(Host, Node, From);
{set, <<"affiliations">>} ->
- set_affiliations(Host, Node, From, xml:remove_cdata(Els));
+ set_affiliations(Host, Node, From, fxml:remove_cdata(Els));
_ ->
{error, ?ERR_FEATURE_NOT_IMPLEMENTED}
end;
@@ -1414,13 +1421,14 @@ adhoc_request(Host, _ServerHost, Owner,
send_pending_node_form(Host, Owner, Lang, Plugins);
adhoc_request(Host, _ServerHost, Owner,
#adhoc_request{node = ?NS_PUBSUB_GET_PENDING,
- action = <<"execute">>, xdata = XData},
+ action = <<"execute">>, xdata = XData, lang = Lang},
_Access, _Plugins) ->
ParseOptions = case XData of
#xmlel{name = <<"x">>} = XEl ->
case jlib:parse_xdata_submit(XEl) of
invalid ->
- {error, ?ERR_BAD_REQUEST};
+ Txt = <<"Incorrect data form">>,
+ {error, ?ERRT_BAD_REQUEST(Lang, Txt)};
XData2 ->
case set_xoption(Host, XData2, []) of
NewOpts when is_list(NewOpts) -> {result, NewOpts};
@@ -1428,8 +1436,8 @@ adhoc_request(Host, _ServerHost, Owner,
end
end;
_ ->
- ?INFO_MSG("Bad XForm: ~p", [XData]),
- {error, ?ERR_BAD_REQUEST}
+ Txt = <<"No data form found">>,
+ {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
end,
case ParseOptions of
{result, XForm} ->
@@ -1459,7 +1467,9 @@ send_pending_node_form(Host, Owner, _Lang, Plugins) ->
end,
case lists:filter(Filter, Plugins) of
[] ->
- {error, ?ERR_FEATURE_NOT_IMPLEMENTED};
+ Err = extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
+ unsupported, <<"get-pending">>),
+ {error, Err};
Ps ->
XOpts = [#xmlel{name = <<"option">>, attrs = [],
children = [#xmlel{name = <<"value">>,
@@ -1500,10 +1510,11 @@ send_pending_auth_events(Host, Node, Owner) ->
true ->
case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
{result, owner} -> node_call(Host, Type, get_node_subscriptions, [Nidx]);
- _ -> {error, ?ERR_FORBIDDEN}
+ _ -> {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
end;
false ->
- {error, ?ERR_FEATURE_NOT_IMPLEMENTED}
+ {error, extended_error(?ERR_FEATURE_NOT_IMPLEMENTED,
+ unsupported, <<"get-pending">>)}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -1597,9 +1608,9 @@ find_authorization_response(Packet) ->
#xmlel{children = Els} = Packet,
XData1 = lists:map(fun
(#xmlel{name = <<"x">>, attrs = XAttrs} = XEl) ->
- case xml:get_attr_s(<<"xmlns">>, XAttrs) of
+ case fxml:get_attr_s(<<"xmlns">>, XAttrs) of
?NS_XDATA ->
- case xml:get_attr_s(<<"type">>, XAttrs) of
+ case fxml:get_attr_s(<<"type">>, XAttrs) of
<<"cancel">> -> none;
_ -> jlib:parse_xdata_submit(XEl)
end;
@@ -1609,7 +1620,7 @@ find_authorization_response(Packet) ->
(_) ->
none
end,
- xml:remove_cdata(Els)),
+ fxml:remove_cdata(Els)),
XData = lists:filter(fun (E) -> E /= none end, XData1),
case XData of
[invalid] ->
@@ -1640,6 +1651,7 @@ send_authorization_approval(Host, JID, SNode, Subscription) ->
ejabberd_router:route(service_jid(Host), JID, Stanza).
handle_authorization_response(Host, From, To, Packet, XFields) ->
+ Lang = fxml:get_tag_attr_s(<<"xml:lang">>, Packet),
case {lists:keysearch(<<"pubsub#node">>, 1, XFields),
lists:keysearch(<<"pubsub#subscriber_jid">>, 1, XFields),
lists:keysearch(<<"pubsub#allow">>, 1, XFields)}
@@ -1661,7 +1673,7 @@ handle_authorization_response(Host, From, To, Packet, XFields) ->
{result, Subs} = node_call(Host, Type, get_subscriptions, [Nidx, Subscriber]),
update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs);
false ->
- {error, ?ERR_FORBIDDEN}
+ {error, ?ERRT_FORBIDDEN(Lang, <<"You're not an owner">>)}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -1676,7 +1688,8 @@ handle_authorization_response(Host, From, To, Packet, XFields) ->
ejabberd_router:route(To, From, Err)
end;
_ ->
- Err = jlib:make_error_reply(Packet, ?ERR_NOT_ACCEPTABLE),
+ Txt = <<"Incorrect data form">>,
+ Err = jlib:make_error_reply(Packet, ?ERRT_NOT_ACCEPTABLE(Lang, Txt)),
ejabberd_router:route(To, From, Err)
end.
@@ -1687,7 +1700,7 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
end,
Subs),
case Sub of
- [{pending, SubId}] ->
+ [{pending, SubId}|_] ->
NewSub = case Allow of
true -> subscribed;
false -> none
@@ -1696,7 +1709,8 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
send_authorization_approval(Host, Subscriber, Node, NewSub),
{result, ok};
_ ->
- {error, ?ERR_UNEXPECTED_REQUEST}
+ Txt = <<"No pending subscriptions found">>,
+ {error, ?ERRT_UNEXPECTED_REQUEST(?MYLANG, Txt)}
end.
-define(XFIELD(Type, Label, Var, Val),
@@ -1775,6 +1789,20 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
%%<li>nodetree create_node checks if nodeid already exists</li>
%%<li>node plugin create_node just sets default affiliation/subscription</li>
%%</ul>
+-spec(create_node/5 ::
+ (
+ Host :: mod_pubsub:host(),
+ ServerHost :: binary(),
+ Node :: <<>> | mod_pubsub:nodeId(),
+ Owner :: jid(),
+ Type :: binary())
+ -> {result, [xmlel(),...]}
+ %%%
+ | {error, xmlel()}
+ ).
+create_node(Host, ServerHost, Node, Owner, Type) ->
+ create_node(Host, ServerHost, Node, Owner, Type, all, []).
+
-spec(create_node/7 ::
(
Host :: mod_pubsub:host(),
@@ -1788,8 +1816,6 @@ update_auth(Host, Node, Type, Nidx, Subscriber, Allow, Subs) ->
%%%
| {error, xmlel()}
).
-create_node(Host, ServerHost, Node, Owner, Type) ->
- create_node(Host, ServerHost, Node, Owner, Type, all, []).
create_node(Host, ServerHost, <<>>, Owner, Type, Access, Configuration) ->
case lists:member(<<"instant-nodes">>, plugin_features(Host, Type)) of
true ->
@@ -1808,13 +1834,14 @@ create_node(Host, ServerHost, <<>>, Owner, Type, Access, Configuration) ->
end;
create_node(Host, ServerHost, Node, Owner, GivenType, Access, Configuration) ->
Type = select_type(ServerHost, Host, Node, GivenType),
- ParseOptions = case xml:remove_cdata(Configuration) of
+ ParseOptions = case fxml:remove_cdata(Configuration) of
[] ->
{result, node_options(Host, Type)};
[#xmlel{name = <<"x">>} = XEl] ->
case jlib:parse_xdata_submit(XEl) of
invalid ->
- {error, ?ERR_BAD_REQUEST};
+ Txt = <<"Incorrect data form">>,
+ {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)};
XData ->
case set_xoption(Host, XData, node_options(Host, Type)) of
NewOpts when is_list(NewOpts) -> {result, NewOpts};
@@ -1823,7 +1850,8 @@ create_node(Host, ServerHost, Node, Owner, GivenType, Access, Configuration) ->
end;
_ ->
?INFO_MSG("Node ~p; bad configuration: ~p", [Node, Configuration]),
- {error, ?ERR_BAD_REQUEST}
+ Txt = <<"No data form found">>,
+ {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)}
end,
case ParseOptions of
{result, NodeOptions} ->
@@ -1860,7 +1888,8 @@ create_node(Host, ServerHost, Node, Owner, GivenType, Access, Configuration) ->
Error
end;
_ ->
- {error, ?ERR_FORBIDDEN}
+ Txt1 = <<"You're not allowed to create nodes">>,
+ {error, ?ERRT_FORBIDDEN(?MYLANG, Txt1)}
end
end,
Reply = [#xmlel{name = <<"pubsub">>,
@@ -1910,7 +1939,7 @@ create_node(Host, ServerHost, Node, Owner, GivenType, Access, Configuration) ->
| {error, xmlel()}
).
delete_node(_Host, <<>>, _Owner) ->
- {error, ?ERR_NOT_ALLOWED};
+ {error, ?ERRT_NOT_ALLOWED(?MYLANG, <<"No node specified">>)};
delete_node(Host, Node, Owner) ->
Action = fun (#pubsub_node{type = Type, id = Nidx}) ->
case node_call(Host, Type, get_affiliation, [Nidx, Owner]) of
@@ -1922,7 +1951,7 @@ delete_node(Host, Node, Owner) ->
Error -> Error
end;
_ ->
- {error, ?ERR_FORBIDDEN}
+ {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
end
end,
Reply = [],
@@ -2231,22 +2260,28 @@ publish_item(Host, ServerHost, Node, Publisher, ItemId, Payload, Access) ->
{result, Reply};
{result, {_, Result}} ->
{result, Result};
- {error, ?ERR_ITEM_NOT_FOUND} ->
- Type = select_type(ServerHost, Host, Node),
- case lists:member(<<"auto-create">>, plugin_features(Host, Type)) of
+ {error, _} = Error ->
+ case is_item_not_found(Error) of
true ->
- case create_node(Host, ServerHost, Node, Publisher, Type, Access, []) of
- {result,
- [#xmlel{name = <<"pubsub">>,
- attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
- children = [#xmlel{name = <<"create">>,
- attrs = [{<<"node">>, NewNode}]}]}]} ->
- publish_item(Host, ServerHost, NewNode, Publisher, ItemId, Payload);
- _ ->
- {error, ?ERR_ITEM_NOT_FOUND}
+ Type = select_type(ServerHost, Host, Node),
+ case lists:member(<<"auto-create">>, plugin_features(Host, Type)) of
+ true ->
+ case create_node(Host, ServerHost, Node, Publisher, Type, Access, []) of
+ {result,
+ [#xmlel{name = <<"pubsub">>,
+ attrs = [{<<"xmlns">>, ?NS_PUBSUB}],
+ children = [#xmlel{name = <<"create">>,
+ attrs = [{<<"node">>, NewNode}]}]}]} ->
+ publish_item(Host, ServerHost, NewNode, Publisher, ItemId, Payload);
+ _ ->
+ {error, ?ERR_ITEM_NOT_FOUND}
+ end;
+ false ->
+ Txt = <<"Automatic node creation is not enabled">>,
+ {error, ?ERRT_ITEM_NOT_FOUND(?MYLANG, Txt)}
end;
false ->
- {error, ?ERR_ITEM_NOT_FOUND}
+ Error
end;
Error ->
Error
@@ -2400,7 +2435,9 @@ get_items(Host, Node, From, SubId, SMaxItems, ItemIds, RSM) ->
end;
true ->
case catch jlib:binary_to_integer(SMaxItems) of
- {'EXIT', _} -> {error, ?ERR_BAD_REQUEST};
+ {'EXIT', _} ->
+ Txt = <<"Value of 'max_items' should be integer">>,
+ {error, ?ERRT_BAD_REQUEST(?MYLANG, Txt)};
Val -> Val
end
end,
@@ -2490,7 +2527,7 @@ get_last_item(Host, Type, Nidx, LJID, mnesia) ->
{result, {[LastItem|_], _}} -> LastItem;
_ -> undefined
end;
-get_last_item(Host, Type, Nidx, LJID, odbc) ->
+get_last_item(Host, Type, Nidx, LJID, sql) ->
case node_action(Host, Type, get_last_items, [Nidx, LJID, 1]) of
{result, [LastItem]} -> LastItem;
_ -> undefined
@@ -2505,7 +2542,7 @@ get_last_items(Host, Type, Nidx, LJID, Number, mnesia) ->
{result, {Items, _}} -> lists:sublist(Items, Number);
_ -> []
end;
-get_last_items(Host, Type, Nidx, LJID, Number, odbc) ->
+get_last_items(Host, Type, Nidx, LJID, Number, sql) ->
case node_action(Host, Type, get_last_items, [Nidx, LJID, Number]) of
{result, Items} -> Items;
_ -> []
@@ -2623,7 +2660,7 @@ get_affiliations(Host, Node, JID) ->
{error,
extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"modify-affiliations">>)};
Affiliation /= owner ->
- {error, ?ERR_FORBIDDEN};
+ {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)};
true ->
node_call(Host, Type, get_node_affiliations, [Nidx])
end
@@ -2668,8 +2705,8 @@ set_affiliations(Host, Node, From, EntitiesEls) ->
(El, Acc) ->
case El of
#xmlel{name = <<"affiliation">>, attrs = Attrs} ->
- JID = jid:from_string(xml:get_attr_s(<<"jid">>, Attrs)),
- Affiliation = string_to_affiliation(xml:get_attr_s(<<"affiliation">>, Attrs)),
+ JID = jid:from_string(fxml:get_attr_s(<<"jid">>, Attrs)),
+ Affiliation = string_to_affiliation(fxml:get_attr_s(<<"affiliation">>, Attrs)),
if (JID == error) or (Affiliation == false) -> error;
true -> [{jid:tolower(JID), Affiliation} | Acc]
end
@@ -2716,7 +2753,7 @@ set_affiliations(Host, Node, From, EntitiesEls) ->
FilteredEntities),
{result, []};
_ ->
- {error, ?ERR_FORBIDDEN}
+ {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -2932,7 +2969,7 @@ get_subscriptions(Host, Node, JID) ->
{error,
extended_error(?ERR_FEATURE_NOT_IMPLEMENTED, unsupported, <<"manage-subscriptions">>)};
Affiliation /= owner ->
- {error, ?ERR_FORBIDDEN};
+ {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)};
true ->
node_call(Host, Type, get_node_subscriptions, [Nidx])
end
@@ -2975,7 +3012,7 @@ get_subscriptions_for_send_last(Host, PType, mnesia, JID, LJID, BJID) ->
|| {Node, Sub, SubId, SubJID} <- Subs,
Sub =:= subscribed, (SubJID == LJID) or (SubJID == BJID),
match_option(Node, send_last_published_item, on_sub_and_presence)];
-get_subscriptions_for_send_last(Host, PType, odbc, JID, LJID, BJID) ->
+get_subscriptions_for_send_last(Host, PType, sql, JID, LJID, BJID) ->
case catch node_action(Host, PType,
get_entity_subscriptions_for_send_last,
[Host, JID])
@@ -2998,9 +3035,9 @@ set_subscriptions(Host, Node, From, EntitiesEls) ->
(El, Acc) ->
case El of
#xmlel{name = <<"subscription">>, attrs = Attrs} ->
- JID = jid:from_string(xml:get_attr_s(<<"jid">>, Attrs)),
- Sub = string_to_subscription(xml:get_attr_s(<<"subscription">>, Attrs)),
- SubId = xml:get_attr_s(<<"subid">>, Attrs),
+ JID = jid:from_string(fxml:get_attr_s(<<"jid">>, Attrs)),
+ Sub = string_to_subscription(fxml:get_attr_s(<<"subscription">>, Attrs)),
+ SubId = fxml:get_attr_s(<<"subid">>, Attrs),
if (JID == error) or (Sub == false) -> error;
true -> [{jid:tolower(JID), Sub, SubId} | Acc]
end
@@ -3043,10 +3080,10 @@ set_subscriptions(Host, Node, From, EntitiesEls) ->
[], Entities),
case Result of
[] -> {result, []};
- _ -> {error, ?ERR_NOT_ACCEPTABLE}
+ [{error, E}|_] -> {error, E}
end;
_ ->
- {error, ?ERR_FORBIDDEN}
+ {error, ?ERRT_FORBIDDEN(?MYLANG, <<"You're not an owner">>)}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -3170,17 +3207,15 @@ sub_option_can_deliver(_, _, _) -> true.
presence_can_deliver(_, false) ->
true;
presence_can_deliver({User, Server, Resource}, true) ->
- case mnesia:dirty_match_object({session, '_', '_', {User, Server}, '_', '_'}) of
+ case ejabberd_sm:get_user_present_resources(User, Server) of
[] ->
false;
Ss ->
lists:foldl(fun
(_, true) ->
true;
- ({session, _, _, _, undefined, _}, _Acc) ->
- false;
- ({session, {_, _, R}, _, _, _Priority, _}, _Acc) ->
- case Resource of
+ ({_, R}, _Acc) ->
+ case Resource of
<<>> -> true;
R -> true;
_ -> false
@@ -3294,9 +3329,14 @@ broadcast_publish_item(Host, Node, Nidx, Type, NodeOptions, ItemId, From, Payloa
true -> Payload;
false -> []
end,
+ Attrs = case get_option(NodeOptions, itemreply, none) of
+ owner -> itemAttr(ItemId); %% owner not supported
+ publisher -> itemAttr(ItemId, {<<"publisher">>, jid:to_string(From)});
+ none -> itemAttr(ItemId)
+ end,
Stanza = event_stanza(
[#xmlel{name = <<"items">>, attrs = nodeAttr(Node),
- children = [#xmlel{name = <<"item">>, attrs = itemAttr(ItemId),
+ children = [#xmlel{name = <<"item">>, attrs = Attrs,
children = Content}]}]),
broadcast_stanza(Host, From, Node, Nidx, Type,
NodeOptions, SubsByDepth, items, Stanza, true),
@@ -3587,7 +3627,7 @@ get_configure(Host, ServerHost, Node, From, Lang) ->
children =
get_configure_xfields(Type, Options, Lang, Groups)}]}]}]};
_ ->
- {error, ?ERR_FORBIDDEN}
+ {error, ?ERRT_FORBIDDEN(Lang, <<"You're not an owner">>)}
end
end,
case transaction(Host, Node, Action, sync_dirty) of
@@ -3626,7 +3666,7 @@ get_option(Options, Var, Def) ->
end.
node_options(Host, Type) ->
- case config(serverhost(Host), default_node_config) of
+ case config(Host, default_node_config) of
undefined -> node_plugin_options(Host, Type);
[] -> node_plugin_options(Host, Type);
Config -> Config
@@ -3648,7 +3688,7 @@ filter_node_options(Options) ->
node_owners_action(Host, Type, Nidx, []) ->
case gen_mod:db_type(serverhost(Host), ?MODULE) of
- odbc ->
+ sql ->
case node_action(Host, Type, get_node_affiliations, [Nidx]) of
{result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
_ -> []
@@ -3661,7 +3701,7 @@ node_owners_action(_Host, _Type, _Nidx, Owners) ->
node_owners_call(Host, Type, Nidx, []) ->
case gen_mod:db_type(serverhost(Host), ?MODULE) of
- odbc ->
+ sql ->
case node_call(Host, Type, get_node_affiliations, [Nidx]) of
{result, Affs} -> [LJID || {LJID, Aff} <- Affs, Aff =:= owner];
_ -> []
@@ -3777,7 +3817,9 @@ get_configure_xfields(_Type, Options, Lang, Groups) ->
?BOOL_CONFIG_FIELD(<<"Only deliver notifications to available users">>,
presence_based_delivery),
?NLIST_CONFIG_FIELD(<<"The collections with which a node is affiliated">>,
- collection)].
+ collection),
+ ?ALIST_CONFIG_FIELD(<<"Whether owners or publisher should receive replies to items">>,
+ itemreply, [none, owner, publisher])].
%%<p>There are several reasons why the node configuration request might fail:</p>
%%<ul>
@@ -3788,9 +3830,9 @@ get_configure_xfields(_Type, Options, Lang, Groups) ->
%%<li>The specified node does not exist.</li>
%%</ul>
set_configure(Host, Node, From, Els, Lang) ->
- case xml:remove_cdata(Els) of
+ case fxml:remove_cdata(Els) of
[#xmlel{name = <<"x">>} = XEl] ->
- case {xml:get_tag_attr_s(<<"xmlns">>, XEl), xml:get_tag_attr_s(<<"type">>, XEl)} of
+ case {fxml:get_tag_attr_s(<<"xmlns">>, XEl), fxml:get_tag_attr_s(<<"type">>, XEl)} of
{?NS_XDATA, <<"cancel">>} ->
{result, []};
{?NS_XDATA, <<"submit">>} ->
@@ -3799,7 +3841,8 @@ set_configure(Host, Node, From, Els, Lang) ->
{result, owner} ->
case jlib:parse_xdata_submit(XEl) of
invalid ->
- {error, ?ERR_BAD_REQUEST};
+ Txt = <<"Incorrect data form">>,
+ {error, ?ERRT_BAD_REQUEST(Lang, Txt)};
XData ->
OldOpts = case Options of
[] -> node_options(Host, Type);
@@ -3819,7 +3862,8 @@ set_configure(Host, Node, From, Els, Lang) ->
end
end;
_ ->
- {error, ?ERR_FORBIDDEN}
+ Txt = <<"You're not an owner">>,
+ {error, ?ERRT_FORBIDDEN(Lang, Txt)}
end
end,
case transaction(Host, Node, Action, transaction) of
@@ -3833,10 +3877,12 @@ set_configure(Host, Node, From, Els, Lang) ->
Other
end;
_ ->
- {error, ?ERR_BAD_REQUEST}
+ Txt = <<"Incorrect data form">>,
+ {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
end;
_ ->
- {error, ?ERR_BAD_REQUEST}
+ Txt = <<"No data form found">>,
+ {error, ?ERRT_BAD_REQUEST(Lang, Txt)}
end.
add_opt(Key, Value, Opts) ->
@@ -3851,7 +3897,10 @@ add_opt(Key, Value, Opts) ->
_ -> error
end,
case BoolVal of
- error -> {error, ?ERR_NOT_ACCEPTABLE};
+ error ->
+ Txt = <<"Value of '~s' should be boolean">>,
+ ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
+ {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)};
_ -> set_xoption(Host, Opts, add_opt(Opt, BoolVal, NewOpts))
end).
@@ -3864,10 +3913,14 @@ add_opt(Key, Value, Opts) ->
if (Max =:= undefined) orelse (IVal =< Max) ->
set_xoption(Host, Opts, add_opt(Opt, IVal, NewOpts));
true ->
- {error, ?ERR_NOT_ACCEPTABLE}
+ Txt = <<"Incorrect value of '~s'">>,
+ ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
+ {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
end;
_ ->
- {error, ?ERR_NOT_ACCEPTABLE}
+ Txt = <<"Value of '~s' should be integer">>,
+ ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
+ {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
end).
-define(SET_ALIST_XOPT(Opt, Val, Vals),
@@ -3875,7 +3928,9 @@ add_opt(Key, Value, Opts) ->
true ->
set_xoption(Host, Opts, add_opt(Opt, jlib:binary_to_atom(Val), NewOpts));
false ->
- {error, ?ERR_NOT_ACCEPTABLE}
+ Txt = <<"Incorrect value of '~s'">>,
+ ErrTxt = iolist_to_binary(io_lib:format(Txt, [Opt])),
+ {error, ?ERRT_NOT_ACCEPTABLE(?MYLANG, ErrTxt)}
end).
-define(SET_LIST_XOPT(Opt, Val),
@@ -3931,25 +3986,21 @@ set_xoption(Host, [{<<"pubsub#collection">>, Value} | Opts], NewOpts) ->
set_xoption(Host, [{<<"pubsub#node">>, [Value]} | Opts], NewOpts) ->
% NewValue = string_to_node(Value),
?SET_LIST_XOPT(node, Value);
+set_xoption(Host, [{<<"pubsub#itemreply">>, [Val]} | Opts], NewOpts) ->
+ ?SET_ALIST_XOPT(itemreply, Val, [none, owner, publisher]);
set_xoption(Host, [_ | Opts], NewOpts) ->
set_xoption(Host, Opts, NewOpts).
-get_max_items_node({_, ServerHost, _}) ->
- get_max_items_node(ServerHost);
get_max_items_node(Host) ->
- config(serverhost(Host), max_items_node, undefined).
+ config(Host, max_items_node, undefined).
-get_max_subscriptions_node({_, ServerHost, _}) ->
- get_max_subscriptions_node(ServerHost);
get_max_subscriptions_node(Host) ->
- config(serverhost(Host), max_subscriptions_node, undefined).
+ config(Host, max_subscriptions_node, undefined).
%%%% last item cache handling
-is_last_item_cache_enabled({_, ServerHost, _}) ->
- is_last_item_cache_enabled(ServerHost);
is_last_item_cache_enabled(Host) ->
- config(serverhost(Host), last_item_cache, false).
+ config(Host, last_item_cache, false).
set_cached_item({_, ServerHost, _}, Nidx, ItemId, Publisher, Payload) ->
set_cached_item(ServerHost, Nidx, ItemId, Publisher, Payload);
@@ -3999,19 +4050,13 @@ get_cached_item(Host, Nidx) ->
host(ServerHost) ->
config(ServerHost, host, <<"pubsub.", ServerHost/binary>>).
-serverhost({_U, Server, _R})->
- Server;
+serverhost({_U, ServerHost, _R})->
+ serverhost(ServerHost);
serverhost(Host) ->
- case binary:match(Host, <<"pubsub.">>) of
- {0,7} ->
- [_,ServerHost] = binary:split(Host, <<".">>),
- ServerHost;
- _ ->
- Host
- end.
+ ejabberd_router:host_of_route(Host).
tree(Host) ->
- case config(serverhost(Host), nodetree) of
+ case config(Host, nodetree) of
undefined -> tree(Host, ?STDTREE);
Tree -> Tree
end.
@@ -4021,19 +4066,19 @@ tree(_Host, <<"virtual">>) ->
tree(Host, Name) ->
case gen_mod:db_type(serverhost(Host), ?MODULE) of
mnesia -> jlib:binary_to_atom(<<"nodetree_", Name/binary>>);
- odbc -> jlib:binary_to_atom(<<"nodetree_", Name/binary, "_odbc">>);
+ sql -> jlib:binary_to_atom(<<"nodetree_", Name/binary, "_sql">>);
_ -> Name
end.
plugin(Host, Name) ->
case gen_mod:db_type(serverhost(Host), ?MODULE) of
mnesia -> jlib:binary_to_atom(<<"node_", Name/binary>>);
- odbc -> jlib:binary_to_atom(<<"node_", Name/binary, "_odbc">>);
+ sql -> jlib:binary_to_atom(<<"node_", Name/binary, "_sql">>);
_ -> Name
end.
plugins(Host) ->
- case config(serverhost(Host), plugins) of
+ case config(Host, plugins) of
undefined -> [?STDNODE];
[] -> [?STDNODE];
Plugins -> Plugins
@@ -4042,12 +4087,15 @@ plugins(Host) ->
subscription_plugin(Host) ->
case gen_mod:db_type(serverhost(Host), ?MODULE) of
mnesia -> pubsub_subscription;
- odbc -> pubsub_subscription_odbc;
+ sql -> pubsub_subscription_sql;
_ -> none
end.
config(ServerHost, Key) ->
config(ServerHost, Key, undefined).
+
+config({_User, Host, _Resource}, Key, Default) ->
+ config(Host, Key, Default);
config(ServerHost, Key, Default) ->
case catch ets:lookup(gen_mod:get_module_proc(ServerHost, config), Key) of
[{Key, Value}] -> Value;
@@ -4064,14 +4112,14 @@ select_type(ServerHost, Host, Node, Type) ->
_ ->
Type
end,
- ConfiguredTypes = plugins(ServerHost),
+ ConfiguredTypes = plugins(Host),
case lists:member(SelectedType, ConfiguredTypes) of
true -> SelectedType;
false -> hd(ConfiguredTypes)
end.
select_type(ServerHost, Host, Node) ->
- select_type(ServerHost, Host, Node, hd(plugins(ServerHost))).
+ select_type(ServerHost, Host, Node, hd(plugins(Host))).
feature(<<"rsm">>) -> ?NS_RSM;
feature(Feature) -> <<(?NS_PUBSUB)/binary, "#", Feature/binary>>.
@@ -4125,8 +4173,9 @@ features(Host, Node) when is_binary(Node) ->
tree_call({_User, Server, _Resource}, Function, Args) ->
tree_call(Server, Function, Args);
tree_call(Host, Function, Args) ->
- ?DEBUG("tree_call ~p ~p ~p", [Host, Function, Args]),
- catch apply(tree(Host), Function, Args).
+ Tree = tree(Host),
+ ?DEBUG("tree_call apply(~s, ~s, ~p) @ ~s", [Tree, Function, Args, Host]),
+ catch apply(Tree, Function, Args).
tree_action(Host, Function, Args) ->
?DEBUG("tree_action ~p ~p ~p", [Host, Function, Args]),
@@ -4135,8 +4184,8 @@ tree_action(Host, Function, Args) ->
case gen_mod:db_type(ServerHost, ?MODULE) of
mnesia ->
catch mnesia:sync_dirty(Fun);
- odbc ->
- case catch ejabberd_odbc:sql_bloc(ServerHost, Fun) of
+ sql ->
+ case catch ejabberd_sql:sql_bloc(ServerHost, Fun) of
{atomic, Result} ->
Result;
{aborted, Reason} ->
@@ -4195,7 +4244,7 @@ transaction(Host, Fun, Trans) ->
ServerHost = serverhost(Host),
DBType = gen_mod:db_type(ServerHost, ?MODULE),
Retry = case DBType of
- odbc -> 2;
+ sql -> 2;
_ -> 1
end,
transaction_retry(Host, ServerHost, Fun, Trans, DBType, Retry).
@@ -4206,12 +4255,12 @@ transaction_retry(Host, ServerHost, Fun, Trans, DBType, Count) ->
Res = case DBType of
mnesia ->
catch mnesia:Trans(Fun);
- odbc ->
+ sql ->
SqlFun = case Trans of
transaction -> sql_transaction;
_ -> sql_bloc
end,
- catch ejabberd_odbc:SqlFun(ServerHost, Fun);
+ catch ejabberd_sql:SqlFun(ServerHost, Fun);
_ ->
{unsupported, DBType}
end,
@@ -4253,6 +4302,13 @@ extended_error(#xmlel{name = Error, attrs = Attrs, children = SubEls}, Ext, ExtA
#xmlel{name = Error, attrs = Attrs,
children = lists:reverse([#xmlel{name = Ext, attrs = ExtAttrs} | SubEls])}.
+is_item_not_found({error, ErrEl}) ->
+ case fxml:get_subtag_with_xmlns(
+ ErrEl, <<"item-not-found">>, ?NS_STANZAS) of
+ #xmlel{} -> true;
+ _ -> false
+ end.
+
string_to_ljid(JID) ->
case jid:from_string(JID) of
error ->
@@ -4273,6 +4329,7 @@ nodeAttr(Node) -> [{<<"node">>, Node}].
itemAttr([]) -> [];
itemAttr(ItemId) -> [{<<"id">>, ItemId}].
+itemAttr(ItemId, From) -> [{<<"id">>, ItemId}, From].
itemsEls(Items) ->
[#xmlel{name = <<"item">>, attrs = itemAttr(ItemId), children = Payload}