Skip to content

Commit 5e08d74

Browse files
committed
Optionally return authz refusal reason to client
## What? If the new config setting `authorization_failure_disclosure` is set to `true`, (`false` by default), then some RabbitMQ authorization backends (internal, HTTP, OAuth 2.0) will return the reason why access was denied to the client. (In future, additional backends can be modified to return a reason.) ## Why? This helps debugging and troubleshooting directly in the client. Some clients might not have access to the RabbitMQ logs, for other clients it's cumbersome to correlate authz denial in the client with logs on the broker. For example, in dev environments, it may be useful for clients to learn why vhost/resource/topic access was denied for a given OAuth 2.0 token. Another example is that some customers would like to pass the reason why authorization was denied from their custom HTTP auth backend via RabbitMQ back to the client. ## How? Authz backends can now return `{false, Reason}` as an alternative to just `false` if access is denied. For security reasons, the additional denial reason by the authz backend will be returned to the client only if the operator opted in by setting `authorization_failure_disclosure` to `true`. Note that `authorization_failure_disclosure` applies only to already authenticated clients when they try to access resources (e.g. vhosts, exchanges, queues, topics). For security reasons, no detailed denial reason is returned to the client if **authentication** fails.
1 parent 94a6c3f commit 5e08d74

File tree

10 files changed

+215
-138
lines changed

10 files changed

+215
-138
lines changed

deps/rabbit/Makefile

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -115,6 +115,7 @@ define PROJECT_ENV
115115
{tracking_execution_timeout, 15000},
116116
{stream_messages_soft_limit, 256},
117117
{track_auth_attempt_source, false},
118+
{authorization_failure_disclosure, false},
118119
{credentials_obfuscation_fallback_secret, <<"nocookie">>},
119120
{dead_letter_worker_consumer_prefetch, 32},
120121
{dead_letter_worker_publisher_confirm_timeout, 180000},

deps/rabbit/src/rabbit_access_control.erl

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -361,6 +361,15 @@ check_access(Fun, Module, ErrStr, ErrArgs, ErrName) ->
361361
ok;
362362
false ->
363363
rabbit_misc:protocol_error(ErrName, ErrStr, ErrArgs);
364+
{false, Reason} ->
365+
case application:get_env(rabbit, authorization_failure_disclosure) of
366+
{ok, true} ->
367+
FullErrStr = ErrStr ++ " by backend ~ts: ~ts",
368+
FullErrArgs = ErrArgs ++ [Module, Reason],
369+
rabbit_misc:protocol_error(ErrName, FullErrStr, FullErrArgs);
370+
_ ->
371+
rabbit_misc:protocol_error(ErrName, ErrStr, ErrArgs)
372+
end;
364373
{error, E} ->
365374
FullErrStr = ErrStr ++ ", backend ~ts returned an error: ~tp",
366375
FullErrArgs = ErrArgs ++ [Module, E],

deps/rabbit/src/rabbit_auth_backend_internal.erl

Lines changed: 19 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -143,16 +143,21 @@ check_resource_access(#auth_user{username = Username},
143143
_AuthContext) ->
144144
case rabbit_db_user:get_user_permissions(Username, VHostPath) of
145145
undefined ->
146-
false;
146+
{false, rabbit_misc:format("user '~ts' has no permissions for vhost '~ts'",
147+
[Username, VHostPath])};
147148
#user_permission{permission = P} ->
148149
PermRegexp = case element(permission_index(Permission), P) of
149150
%% <<"^$">> breaks Emacs' erlang mode
150151
<<"">> -> <<$^, $$>>;
151152
RE -> RE
152153
end,
153154
case re:run(Name, PermRegexp, [{capture, none}]) of
154-
match -> true;
155-
nomatch -> false
155+
match ->
156+
true;
157+
nomatch ->
158+
{false, rabbit_misc:format(
159+
"'~ts' does not match the permission regex '~ts'",
160+
[Name, PermRegexp])}
156161
end
157162
end.
158163

@@ -170,12 +175,17 @@ check_topic_access(#auth_user{username = Username},
170175
RE -> RE
171176
end,
172177
PermRegexpExpanded = expand_topic_permission(
173-
PermRegexp,
174-
maps:get(variable_map, Context, undefined)
175-
),
176-
case re:run(maps:get(routing_key, Context), PermRegexpExpanded, [{capture, none}]) of
177-
match -> true;
178-
nomatch -> false
178+
PermRegexp,
179+
maps:get(variable_map, Context, undefined)
180+
),
181+
Topic = maps:get(routing_key, Context),
182+
case re:run(Topic, PermRegexpExpanded, [{capture, none}]) of
183+
match ->
184+
true;
185+
nomatch ->
186+
{false, rabbit_misc:format(
187+
"topic '~ts' does not match the regex '~ts'",
188+
[Topic, PermRegexpExpanded])}
179189
end
180190
end.
181191

deps/rabbit/src/rabbit_authz_backend.erl

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,25 +32,27 @@
3232
%% Possible responses:
3333
%% true
3434
%% false
35+
%% {false, Reason}
3536
%% {error, Error}
3637
%% Something went wrong. Log and die.
3738
-callback check_vhost_access(AuthUser :: rabbit_types:auth_user(),
3839
VHost :: rabbit_types:vhost(),
3940
AuthzData :: rabbit_types:authz_data()) ->
40-
boolean() | {'error', any()}.
41+
boolean() | {false, Reason :: string()} | {'error', any()}.
4142

4243
%% Given #auth_user, resource and permission, can a user access a resource?
4344
%%
4445
%% Possible responses:
4546
%% true
4647
%% false
48+
%% {false, Reason}
4749
%% {error, Error}
4850
%% Something went wrong. Log and die.
4951
-callback check_resource_access(rabbit_types:auth_user(),
5052
rabbit_types:r(atom()),
5153
rabbit_types:permission_atom(),
5254
rabbit_types:authz_context()) ->
53-
boolean() | {'error', any()}.
55+
boolean() | {false, Reason :: string()} | {'error', any()}.
5456

5557
%% Given #auth_user, topic as resource, permission, and context, can a user access the topic?
5658
%%
@@ -63,7 +65,7 @@
6365
rabbit_types:r(atom()),
6466
rabbit_types:permission_atom(),
6567
rabbit_types:topic_access_context()) ->
66-
boolean() | {'error', any()}.
68+
boolean() | {false, Reason :: string()} | {'error', any()}.
6769

6870
%% Updates backend state that has expired.
6971
%%

deps/rabbit/test/topic_permission_SUITE.erl

Lines changed: 43 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -36,8 +36,10 @@ init_per_suite(Config) ->
3636
Config1 = rabbit_ct_helpers:set_config(
3737
Config,
3838
[{rmq_nodename_suffix, ?MODULE}]),
39+
Config2 = rabbit_ct_helpers:merge_app_env(
40+
Config1, {rabbit, [{authorization_failure_disclosure, true}]}),
3941
rabbit_ct_helpers:run_setup_steps(
40-
Config1,
42+
Config2,
4143
rabbit_ct_broker_helpers:setup_steps() ++
4244
rabbit_ct_client_helpers:setup_steps()).
4345

@@ -109,10 +111,12 @@ amqp_x_cc_annotation(Config) ->
109111
condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
110112
description = {utf8, Description1}}}}} ->
111113
?assertEqual(
112-
<<"write access to topic 'x.1' in exchange 'amq.topic' in vhost '/' refused for user 'guest'">>,
114+
<<"write access to topic 'x.1' in exchange 'amq.topic' in vhost '/' "
115+
"refused for user 'guest' by backend rabbit_auth_backend_internal: "
116+
"topic 'x.1' does not match the regex '^a'">>,
113117
Description1)
114118
after 30_000 -> amqp_utils:flush(missing_ended),
115-
ct:fail({missing_event, ?LINE})
119+
ct:fail({missing_event, ?LINE})
116120
end,
117121

118122
{ok, Session3} = amqp10_client:begin_session_sync(Connection),
@@ -132,10 +136,12 @@ amqp_x_cc_annotation(Config) ->
132136
condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
133137
description = {utf8, Description2}}}}} ->
134138
?assertEqual(
135-
<<"write access to topic 'x.2' in exchange 'amq.topic' in vhost '/' refused for user 'guest'">>,
139+
<<"write access to topic 'x.2' in exchange 'amq.topic' in vhost '/' "
140+
"refused for user 'guest' by backend rabbit_auth_backend_internal: "
141+
"topic 'x.2' does not match the regex '^a'">>,
136142
Description2)
137143
after 30_000 -> amqp_utils:flush(missing_ended),
138-
ct:fail({missing_event, ?LINE})
144+
ct:fail({missing_event, ?LINE})
139145
end,
140146

141147
{ok, #{message_count := 0}} = rabbitmq_amqp_client:delete_queue(LinkPair, QName1),
@@ -190,8 +196,9 @@ amqpl_headers(Header, Config) ->
190196
props = #'P_basic'{headers = [{Header, array, [{longstr, <<"a.2">>}]}]}}),
191197
ok = assert_channel_down(
192198
Ch1,
193-
<<"ACCESS_REFUSED - write access to topic 'x.1' in exchange "
194-
"'amq.topic' in vhost '/' refused for user 'guest'">>),
199+
<<"ACCESS_REFUSED - write access to topic 'x.1' in exchange 'amq.topic' "
200+
"in vhost '/' refused for user 'guest' by backend rabbit_auth_backend_internal: "
201+
"topic 'x.1' does not match the regex '^a'">>),
195202

196203
Ch2 = rabbit_ct_client_helpers:open_channel(Config),
197204
monitor(process, Ch2),
@@ -203,8 +210,9 @@ amqpl_headers(Header, Config) ->
203210
props = #'P_basic'{headers = [{Header, array, [{longstr, <<"x.2">>}]}]}}),
204211
ok = assert_channel_down(
205212
Ch2,
206-
<<"ACCESS_REFUSED - write access to topic 'x.2' in exchange "
207-
"'amq.topic' in vhost '/' refused for user 'guest'">>),
213+
<<"ACCESS_REFUSED - write access to topic 'x.2' in exchange 'amq.topic' "
214+
"in vhost '/' refused for user 'guest' by backend rabbit_auth_backend_internal: "
215+
"topic 'x.2' does not match the regex '^a'">>),
208216

209217
Ch3 = rabbit_ct_client_helpers:open_channel(Config),
210218
?assertEqual(#'queue.delete_ok'{message_count = 1},
@@ -337,12 +345,14 @@ topic_permission_checks1(_Config) ->
337345
Context
338346
) || Perm <- Permissions],
339347
%% user has access to exchange, routing key does not match
340-
[false = rabbit_auth_backend_internal:check_topic_access(
341-
User,
342-
Topic,
343-
Perm,
344-
#{routing_key => <<"x.y.z">>}
345-
) || Perm <- Permissions],
348+
[?assertEqual(
349+
{false, "topic 'x.y.z' does not match the regex '^a'"},
350+
rabbit_auth_backend_internal:check_topic_access(
351+
User,
352+
Topic,
353+
Perm,
354+
#{routing_key => <<"x.y.z">>}
355+
)) || Perm <- Permissions],
346356
%% user has access to exchange but not on this vhost
347357
%% let pass when there's no match
348358
[true = rabbit_auth_backend_internal:check_topic_access(
@@ -379,17 +389,20 @@ topic_permission_checks1(_Config) ->
379389
}
380390
) || Perm <- Permissions],
381391
%% routing key KO
382-
[false = rabbit_auth_backend_internal:check_topic_access(
383-
User,
384-
Topic#resource{virtual_host = <<"other-vhost">>},
385-
Perm,
386-
#{routing_key => <<"services.default.accounts.dummy.notifications">>,
387-
variable_map => #{
388-
<<"username">> => <<"guest">>,
389-
<<"vhost">> => <<"other-vhost">>
390-
}
391-
}
392-
) || Perm <- Permissions],
392+
[?assertEqual(
393+
{false, "topic 'services.default.accounts.dummy.notifications' does not "
394+
"match the regex 'services.other-vhost.accounts.guest.notifications'"},
395+
rabbit_auth_backend_internal:check_topic_access(
396+
User,
397+
Topic#resource{virtual_host = <<"other-vhost">>},
398+
Perm,
399+
#{routing_key => <<"services.default.accounts.dummy.notifications">>,
400+
variable_map => #{
401+
<<"username">> => <<"guest">>,
402+
<<"vhost">> => <<"other-vhost">>
403+
}
404+
}
405+
)) || Perm <- Permissions],
393406

394407
ok.
395408

@@ -407,11 +420,10 @@ clear_topic_permissions(Config) ->
407420
Config, 0, rabbit_auth_backend_internal, clear_topic_permissions,
408421
[<<"guest">>, <<"/">>, <<"acting-user">>]).
409422

410-
assert_channel_down(Ch, Reason) ->
423+
assert_channel_down(Ch, ExpectedReason) ->
411424
receive {'DOWN', _MonitorRef, process, Ch,
412-
{shutdown,
413-
{server_initiated_close, 403, Reason}}} ->
414-
ok
425+
{shutdown, {server_initiated_close, 403, ActualReason}}} ->
426+
?assertEqual(ExpectedReason, ActualReason)
415427
after 30_000 ->
416-
ct:fail({did_not_receive, Reason})
428+
ct:fail({missing_down, ExpectedReason})
417429
end.

deps/rabbitmq_auth_backend_http/src/rabbit_auth_backend_http.erl

Lines changed: 18 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -120,34 +120,34 @@ check_vhost_access(#auth_user{username = Username, tags = Tags}, VHost,
120120

121121
do_check_vhost_access(Username, Tags, VHost, Ip, AuthzData) ->
122122
OptionsParameters = context_as_parameters(AuthzData),
123-
bool_req(vhost_path, [{username, Username},
124-
{vhost, VHost},
125-
{ip, Ip},
126-
{tags, join_tags(Tags)}] ++ OptionsParameters).
123+
req(vhost_path, [{username, Username},
124+
{vhost, VHost},
125+
{ip, Ip},
126+
{tags, join_tags(Tags)}] ++ OptionsParameters).
127127

128128
check_resource_access(#auth_user{username = Username, tags = Tags},
129129
#resource{virtual_host = VHost, kind = Type, name = Name},
130130
Permission,
131131
AuthzContext) ->
132132
OptionsParameters = context_as_parameters(AuthzContext),
133-
bool_req(resource_path, [{username, Username},
134-
{vhost, VHost},
135-
{resource, Type},
136-
{name, Name},
137-
{permission, Permission},
138-
{tags, join_tags(Tags)}] ++ OptionsParameters).
133+
req(resource_path, [{username, Username},
134+
{vhost, VHost},
135+
{resource, Type},
136+
{name, Name},
137+
{permission, Permission},
138+
{tags, join_tags(Tags)}] ++ OptionsParameters).
139139

140140
check_topic_access(#auth_user{username = Username, tags = Tags},
141141
#resource{virtual_host = VHost, kind = topic = Type, name = Name},
142142
Permission,
143143
Context) ->
144144
OptionsParameters = context_as_parameters(Context),
145-
bool_req(topic_path, [{username, Username},
146-
{vhost, VHost},
147-
{resource, Type},
148-
{name, Name},
149-
{permission, Permission},
150-
{tags, join_tags(Tags)}] ++ OptionsParameters).
145+
req(topic_path, [{username, Username},
146+
{vhost, VHost},
147+
{resource, Type},
148+
{name, Name},
149+
{permission, Permission},
150+
{tags, join_tags(Tags)}] ++ OptionsParameters).
151151

152152
expiry_timestamp(_) -> never.
153153

@@ -163,7 +163,7 @@ context_as_parameters(Options) when is_map(Options) ->
163163
context_as_parameters(_) ->
164164
[].
165165

166-
bool_req(PathName, Props) ->
166+
req(PathName, Props) ->
167167
Path = p(PathName),
168168
Query = q(Props),
169169
case http_req(Path, Query) of
@@ -172,7 +172,7 @@ bool_req(PathName, Props) ->
172172
"deny " ++ Reason ->
173173
?LOG_INFO("HTTP authorisation denied for path ~ts with query ~ts: ~ts",
174174
[Path, Query, Reason]),
175-
false;
175+
{false, Reason};
176176
Body ->
177177
case string:lowercase(Body) of
178178
"deny" ->

deps/rabbitmq_auth_backend_oauth2/src/rabbit_oauth2_scope.erl

Lines changed: 42 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -24,37 +24,56 @@
2424
-spec vhost_access(binary(), [binary()]) -> boolean().
2525
vhost_access(VHost, Scopes) ->
2626
PermissionScopes = get_scope_permissions(Scopes),
27-
lists:any(
28-
fun({VHostPattern, _, _, _}) ->
29-
wildcard:match(VHost, VHostPattern)
30-
end,
31-
PermissionScopes).
27+
case lists:any(
28+
fun({VHostPattern, _, _, _}) ->
29+
wildcard:match(VHost, VHostPattern)
30+
end,
31+
PermissionScopes) of
32+
true ->
33+
true;
34+
false ->
35+
{false, rabbit_misc:format("no scope in ~tp matches vhost '~ts'",
36+
[Scopes, VHost])}
37+
end.
3238

3339
-spec resource_access(rabbit_types:r(atom()), permission(), [binary()]) -> boolean().
34-
resource_access(#resource{virtual_host = VHost, name = Name},
40+
resource_access(#resource{virtual_host = VHost, name = Name} = Resource,
3541
Permission, Scopes) ->
36-
lists:any(
37-
fun({VHostPattern, NamePattern, _, ScopeGrantedPermission}) ->
38-
wildcard:match(VHost, VHostPattern) andalso
39-
wildcard:match(Name, NamePattern) andalso
40-
Permission =:= ScopeGrantedPermission
41-
end,
42-
get_scope_permissions(Scopes)).
42+
case lists:any(
43+
fun({VHostPattern, NamePattern, _, ScopeGrantedPermission}) ->
44+
wildcard:match(VHost, VHostPattern) andalso
45+
wildcard:match(Name, NamePattern) andalso
46+
Permission =:= ScopeGrantedPermission
47+
end,
48+
get_scope_permissions(Scopes)) of
49+
true ->
50+
true;
51+
false ->
52+
{false, rabbit_misc:format("no scope in ~tp has '~s' permission for ~ts",
53+
[Scopes, Permission, rabbit_misc:rs(Resource)])}
54+
end.
4355

4456
-spec topic_access(rabbit_types:r(atom()), permission(), map(), [binary()]) -> boolean().
45-
topic_access(#resource{virtual_host = VHost, name = ExchangeName},
57+
topic_access(#resource{virtual_host = VHost, name = ExchangeName} = Resource,
4658
Permission,
4759
#{routing_key := RoutingKey},
4860
Scopes) ->
49-
lists:any(
50-
fun({VHostPattern, ExchangeNamePattern, RoutingKeyPattern, ScopeGrantedPermission}) ->
51-
is_binary(RoutingKeyPattern) andalso
52-
wildcard:match(VHost, VHostPattern) andalso
53-
wildcard:match(ExchangeName, ExchangeNamePattern) andalso
54-
wildcard:match(RoutingKey, RoutingKeyPattern) andalso
55-
Permission =:= ScopeGrantedPermission
56-
end,
57-
get_scope_permissions(Scopes)).
61+
case lists:any(
62+
fun({VHostPattern, ExchangeNamePattern, RoutingKeyPattern, ScopeGrantedPermission}) ->
63+
is_binary(RoutingKeyPattern) andalso
64+
wildcard:match(VHost, VHostPattern) andalso
65+
wildcard:match(ExchangeName, ExchangeNamePattern) andalso
66+
wildcard:match(RoutingKey, RoutingKeyPattern) andalso
67+
Permission =:= ScopeGrantedPermission
68+
end,
69+
get_scope_permissions(Scopes)) of
70+
true ->
71+
true;
72+
false ->
73+
{false, rabbit_misc:format(
74+
"no scope in ~tp has '~s' permission for exchange ~ts and topic '~ts'",
75+
[Scopes, Permission, rabbit_misc:rs(Resource), RoutingKey])}
76+
end.
5877

5978
%% Internal -------------------------------------------------------------------
6079

0 commit comments

Comments
 (0)