Merge branch 'main' into issue-9437-queue-storage-version

This commit is contained in:
Michael Klishin 2023-10-10 15:03:50 -04:00 committed by GitHub
commit 8e7e8f9127
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 1378 additions and 608 deletions

View File

@ -941,6 +941,11 @@ rabbitmq_integration_suite(
size = "medium",
)
rabbitmq_suite(
name = "unit_quorum_queue_SUITE",
size = "medium",
)
rabbitmq_integration_suite(
name = "unit_app_management_SUITE",
size = "medium",

8
deps/rabbit/app.bzl vendored
View File

@ -1692,6 +1692,14 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
erlc_opts = "//:test_erlc_opts",
deps = ["//deps/amqp_client:erlang_app"],
)
erlang_bytecode(
name = "unit_quorum_queue_SUITE_beam_files",
testonly = True,
srcs = ["test/unit_quorum_queue_SUITE.erl"],
outs = ["test/unit_quorum_queue_SUITE.beam"],
app_name = "rabbit",
erlc_opts = "//:test_erlc_opts",
)
erlang_bytecode(
name = "unit_app_management_SUITE_beam_files",
testonly = True,

View File

@ -22,7 +22,9 @@
add_user_sans_validation/3, put_user/2, put_user/3,
update_user/5,
update_user_with_hash/5,
add_user_sans_validation/6]).
add_user_sans_validation/6,
add_user_with_pre_hashed_password_sans_validation/3
]).
-export([set_user_limits/3, clear_user_limits/3, is_over_connection_limit/1,
is_over_channel_limit/1, get_user_limits/0, get_user_limits/1]).
@ -222,6 +224,10 @@ add_user(Username, Password, ActingUser, Limits, Tags) ->
validate_and_alternate_credentials(Username, Password, ActingUser,
add_user_sans_validation(Limits, Tags)).
add_user_with_pre_hashed_password_sans_validation(Username, PasswordHash, ActingUser) ->
HashingAlgorithm = rabbit_password:hashing_mod(),
add_user_sans_validation(Username, PasswordHash, HashingAlgorithm, [], undefined, ActingUser).
add_user_sans_validation(Username, Password, ActingUser) ->
add_user_sans_validation(Username, Password, ActingUser, undefined, []).
@ -246,14 +252,12 @@ add_user_sans_validation(Username, Password, ActingUser, Limits, Tags) ->
end,
add_user_sans_validation_in(Username, User, ConvertedTags, Limits, ActingUser).
add_user_sans_validation(Username, PasswordHash, HashingAlgorithm, Tags, Limits, ActingUser) ->
add_user_sans_validation(Username, PasswordHash, HashingMod, Tags, Limits, ActingUser) ->
rabbit_log:debug("Asked to create a new user '~ts' with password hash", [Username]),
ConvertedTags = [rabbit_data_coercion:to_atom(I) || I <- Tags],
HashingMod = rabbit_password:hashing_mod(),
User0 = internal_user:create_user(Username, PasswordHash, HashingMod),
User1 = internal_user:set_tags(
internal_user:set_password_hash(User0,
PasswordHash, HashingAlgorithm),
internal_user:set_password_hash(User0, PasswordHash, HashingMod),
ConvertedTags),
User = case Limits of
undefined -> User1;

View File

@ -227,7 +227,7 @@
%% It is called when a feature flag is being enabled. The function is
%% responsible for this feature-flag-specific verification and data
%% conversion. It returns `ok' if RabbitMQ can mark the feature flag as
%% enabled an continue with the next one, if any. `{error, Reason}' and
%% enabled and continue with the next one, if any. `{error, Reason}' and
%% exceptions are an error and the feature flag will remain disabled.
%%
%% The migration function is called on all nodes which fulfill the following

View File

@ -34,6 +34,7 @@
-define(NODE_REPLY_TIMEOUT, 5000).
-define(RABBIT_UP_RPC_TIMEOUT, 2000).
-define(RABBIT_DOWN_PING_INTERVAL, 1000).
-define(NODE_DISCONNECTION_TIMEOUT, 1000).
-record(state, {monitors, partitions, subscribers, down_ping_timer,
keepalive_timer, autoheal, guid, node_guids}).
@ -921,13 +922,23 @@ upgrade_to_full_partition(Proxy) ->
%% detect a very short partition. So we want to force a slightly
%% longer disconnect. Unfortunately we don't have a way to blacklist
%% individual nodes; the best we can do is turn off auto-connect
%% altogether.
%% altogether. If Node is not already part of the connected nodes, then
%% there's no need to repeat disabling dist_auto_connect and executing
%% disconnect_node/1, which can result in application_controller
%% timeouts and crash node monitor process. This also implies that
%% the already disconnected node was already processed. In an
%% unstable network, if we get consecutive 'up' and 'down' messages,
%% then we expect disconnect_node/1 to be executed.
disconnect(Node) ->
application:set_env(kernel, dist_auto_connect, never),
erlang:disconnect_node(Node),
timer:sleep(1000),
application:unset_env(kernel, dist_auto_connect),
ok.
case lists:member(Node, nodes()) of
true ->
application:set_env(kernel, dist_auto_connect, never),
erlang:disconnect_node(Node),
timer:sleep(?NODE_DISCONNECTION_TIMEOUT),
application:unset_env(kernel, dist_auto_connect);
false ->
ok
end.
%%--------------------------------------------------------------------
@ -998,8 +1009,12 @@ ping_all() ->
possibly_partitioned_nodes() ->
alive_rabbit_nodes() -- rabbit_mnesia:cluster_nodes(running).
startup_log([]) ->
rabbit_log:info("Starting rabbit_node_monitor", []);
startup_log(Nodes) ->
rabbit_log:info("Starting rabbit_node_monitor, might be partitioned from ~tp",
[Nodes]).
{ok, M} = application:get_env(rabbit, cluster_partition_handling),
startup_log(Nodes, M).
startup_log([], PartitionHandling) ->
rabbit_log:info("Starting rabbit_node_monitor (in ~tp mode)", [PartitionHandling]);
startup_log(Nodes, PartitionHandling) ->
rabbit_log:info("Starting rabbit_node_monitor (in ~tp mode), might be partitioned from ~tp",
[PartitionHandling, Nodes]).

View File

@ -137,6 +137,8 @@ sheet_body(PrevState) ->
case maps:get(InternalName, RaStates, undefined) of
leader -> "L";
follower -> "F";
promotable -> "f"; %% temporary non-voter
non_voter -> "-"; %% permanent non-voter
_ -> "?"
end,
format_int(proplists:get_value(memory, ProcInfo)),

View File

@ -55,6 +55,7 @@ register() ->
{operator_policy_validator, <<"max-in-memory-bytes">>},
{operator_policy_validator, <<"delivery-limit">>},
{operator_policy_validator, <<"queue-version">>},
{operator_policy_validator, <<"overflow">>},
{policy_merge_strategy, <<"expires">>},
{policy_merge_strategy, <<"message-ttl">>},
{policy_merge_strategy, <<"max-length">>},
@ -62,7 +63,8 @@ register() ->
{policy_merge_strategy, <<"max-in-memory-length">>},
{policy_merge_strategy, <<"max-in-memory-bytes">>},
{policy_merge_strategy, <<"delivery-limit">>},
{policy_merge_strategy, <<"queue-version">>}]],
{policy_merge_strategy, <<"queue-version">>},
{policy_merge_strategy, <<"overflow">>}]],
ok.
-spec validate_policy([{binary(), term()}]) -> rabbit_policy_validator:validate_results().
@ -214,5 +216,6 @@ merge_policy_value(<<"max-in-memory-bytes">>, Val, OpVal) -> min(Val, OpVal);
merge_policy_value(<<"expires">>, Val, OpVal) -> min(Val, OpVal);
merge_policy_value(<<"delivery-limit">>, Val, OpVal) -> min(Val, OpVal);
merge_policy_value(<<"queue-version">>, _Val, OpVal) -> OpVal;
merge_policy_value(<<"overflow">>, _Val, OpVal) -> OpVal;
%% use operator policy value for booleans
merge_policy_value(_Key, Val, OpVal) when is_boolean(Val) andalso is_boolean(OpVal) -> OpVal.

View File

@ -39,21 +39,24 @@
-export([open_files/1]).
-export([peek/2, peek/3]).
-export([add_member/2,
add_member/4]).
add_member/3,
add_member/4,
add_member/5]).
-export([delete_member/3, delete_member/2]).
-export([requeue/3]).
-export([policy_changed/1]).
-export([format_ra_event/3]).
-export([cleanup_data_dir/0]).
-export([shrink_all/1,
grow/4]).
grow/4,
grow/5]).
-export([transfer_leadership/2, get_replicas/1, queue_length/1]).
-export([file_handle_leader_reservation/1,
file_handle_other_reservation/0]).
-export([file_handle_release_reservation/0]).
-export([list_with_minimum_quorum/0,
filter_quorum_critical/1,
filter_quorum_critical/2,
filter_quorum_critical/3,
all_replica_states/0]).
-export([capabilities/0]).
-export([repair_amqqueue_nodes/1,
@ -84,6 +87,7 @@
-type msg_id() :: non_neg_integer().
-type qmsg() :: {rabbit_types:r('queue'), pid(), msg_id(), boolean(),
mc:state()}.
-type membership() :: voter | non_voter | promotable. %% see ra_membership() in Ra.
-define(RA_SYSTEM, quorum_queues).
-define(RA_WAL_NAME, ra_log_wal).
@ -384,13 +388,15 @@ become_leader(QName, Name) ->
all_replica_states() ->
Rows0 = ets:tab2list(ra_state),
Rows = lists:map(fun
%% TODO: support other membership types
({K, follower, promotable}) ->
{K, promotable};
({K, follower, non_voter}) ->
{K, non_voter};
({K, S, voter}) ->
{K, S};
(T) ->
T
end, Rows0),
{node(), maps:from_list(Rows)}.
-spec list_with_minimum_quorum() -> [amqqueue:amqqueue()].
@ -419,20 +425,22 @@ filter_quorum_critical(Queues) ->
ReplicaStates = maps:from_list(
rabbit_misc:append_rpc_all_nodes(rabbit_nodes:list_running(),
?MODULE, all_replica_states, [])),
filter_quorum_critical(Queues, ReplicaStates).
filter_quorum_critical(Queues, ReplicaStates, node()).
-spec filter_quorum_critical([amqqueue:amqqueue()], #{node() => #{atom() => atom()}}) -> [amqqueue:amqqueue()].
-spec filter_quorum_critical([amqqueue:amqqueue()], #{node() => #{atom() => atom()}}, node()) ->
[amqqueue:amqqueue()].
filter_quorum_critical(Queues, ReplicaStates) ->
filter_quorum_critical(Queues, ReplicaStates, Self) ->
lists:filter(fun (Q) ->
MemberNodes = rabbit_amqqueue:get_quorum_nodes(Q),
{Name, _Node} = amqqueue:get_pid(Q),
AllUp = lists:filter(fun (N) ->
{Name, _} = amqqueue:get_pid(Q),
case maps:get(N, ReplicaStates, undefined) of
#{Name := State}
when State =:= follower orelse
State =:= leader ->
State =:= leader orelse
(State =:= promotable andalso N =:= Self) orelse
(State =:= non_voter andalso N =:= Self) ->
true;
_ -> false
end
@ -1143,7 +1151,7 @@ get_sys_status(Proc) ->
end.
add_member(VHost, Name, Node, Timeout) ->
add_member(VHost, Name, Node, Membership, Timeout) when is_binary(VHost) ->
QName = #resource{virtual_host = VHost, name = Name, kind = queue},
rabbit_log:debug("Asked to add a replica for queue ~ts on node ~ts", [rabbit_misc:rs(QName), Node]),
case rabbit_amqqueue:lookup(QName) of
@ -1161,7 +1169,7 @@ add_member(VHost, Name, Node, Timeout) ->
rabbit_log:debug("Quorum ~ts already has a replica on node ~ts", [rabbit_misc:rs(QName), Node]),
ok;
false ->
add_member(Q, Node, Timeout)
add_member(Q, Node, Membership, Timeout)
end
end;
{ok, _Q} ->
@ -1171,9 +1179,15 @@ add_member(VHost, Name, Node, Timeout) ->
end.
add_member(Q, Node) ->
add_member(Q, Node, ?ADD_MEMBER_TIMEOUT).
add_member(Q, Node, promotable).
add_member(Q, Node, Timeout) when ?amqqueue_is_quorum(Q) ->
add_member(Q, Node, Membership) ->
add_member(Q, Node, Membership, ?ADD_MEMBER_TIMEOUT).
add_member(VHost, Name, Node, Timeout) when is_binary(VHost) ->
%% NOTE needed to pass mixed cluster tests.
add_member(VHost, Name, Node, promotable, Timeout);
add_member(Q, Node, Membership, Timeout) when ?amqqueue_is_quorum(Q) ->
{RaName, _} = amqqueue:get_pid(Q),
QName = amqqueue:get_name(Q),
%% TODO parallel calls might crash this, or add a duplicate in quorum_nodes
@ -1183,7 +1197,7 @@ add_member(Q, Node, Timeout) when ?amqqueue_is_quorum(Q) ->
?TICK_TIMEOUT),
SnapshotInterval = application:get_env(rabbit, quorum_snapshot_interval,
?SNAPSHOT_INTERVAL),
Conf = make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, voter),
Conf = make_ra_conf(Q, ServerId, TickTimeout, SnapshotInterval, Membership),
case ra:start_server(?RA_SYSTEM, Conf) of
ok ->
ServerIdSpec = maps:with([id, uid, membership], Conf),
@ -1295,17 +1309,21 @@ shrink_all(Node) ->
amqqueue:get_type(Q) == ?MODULE,
lists:member(Node, get_nodes(Q))].
-spec grow(node(), binary(), binary(), all | even) ->
grow(Node, VhostSpec, QueueSpec, Strategy) ->
grow(Node, VhostSpec, QueueSpec, Strategy, promotable).
-spec grow(node(), binary(), binary(), all | even, membership()) ->
[{rabbit_amqqueue:name(),
{ok, pos_integer()} | {error, pos_integer(), term()}}].
grow(Node, VhostSpec, QueueSpec, Strategy) ->
grow(Node, VhostSpec, QueueSpec, Strategy, Membership) ->
Running = rabbit_nodes:list_running(),
[begin
Size = length(get_nodes(Q)),
QName = amqqueue:get_name(Q),
rabbit_log:info("~ts: adding a new member (replica) on node ~w",
[rabbit_misc:rs(QName), Node]),
case add_member(Q, Node, ?ADD_MEMBER_TIMEOUT) of
case add_member(Q, Node, Membership) of
ok ->
{QName, {ok, Size + 1}};
{error, Err} ->

View File

@ -340,12 +340,20 @@ filter_spec(Args) ->
get_local_pid(#stream_client{local_pid = Pid} = State)
when is_pid(Pid) ->
{Pid, State};
case erlang:is_process_alive(Pid) of
true ->
{Pid, State};
false ->
query_local_pid(State)
end;
get_local_pid(#stream_client{leader = Pid} = State)
when is_pid(Pid) andalso node(Pid) == node() ->
{Pid, State#stream_client{local_pid = Pid}};
get_local_pid(#stream_client{stream_id = StreamId} = State) ->
get_local_pid(State#stream_client{local_pid = Pid});
get_local_pid(#stream_client{} = State) ->
%% query local coordinator to get pid
query_local_pid(State).
query_local_pid(#stream_client{stream_id = StreamId} = State) ->
case rabbit_stream_coordinator:local_pid(StreamId) of
{ok, Pid} ->
{Pid, State#stream_client{local_pid = Pid}};
@ -555,9 +563,10 @@ recover(_VHost, Queues) ->
{[Q | R0], F0}
end, {[], []}, Queues).
settle(QName, complete, CTag, MsgIds, #stream_client{readers = Readers0,
settle(QName, _, CTag, MsgIds, #stream_client{readers = Readers0,
local_pid = LocalPid,
name = Name} = State) ->
%% all settle reasons will "give credit" to the stream queue
Credit = length(MsgIds),
{Readers, Msgs} = case Readers0 of
#{CTag := #stream{credit = Credit0} = Str0} ->
@ -567,11 +576,7 @@ settle(QName, complete, CTag, MsgIds, #stream_client{readers = Readers0,
_ ->
{Readers0, []}
end,
{State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]};
settle(_, _, _, _, #stream_client{name = Name}) ->
{protocol_error, not_implemented,
"basic.nack and basic.reject not supported by stream queues ~ts",
[rabbit_misc:rs(Name)]}.
{State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]}.
info(Q, all_keys) ->
info(Q, ?INFO_KEYS);

View File

@ -39,6 +39,7 @@ all_tests() ->
operator_retroactive_policy_publish_ttl,
queue_type_specific_policies,
classic_queue_version_policies,
overflow_policies,
is_supported_operator_policy_expires,
is_supported_operator_policy_message_ttl,
is_supported_operator_policy_max_length,
@ -47,6 +48,7 @@ all_tests() ->
is_supported_operator_policy_max_in_memory_bytes,
is_supported_operator_policy_delivery_limit,
is_supported_operator_policy_target_group_size,
is_supported_operator_policy_overflow,
is_supported_operator_policy_ha
].
@ -306,6 +308,29 @@ classic_queue_version_policies(Config) ->
rabbit_ct_client_helpers:close_connection(Conn),
passed.
overflow_policies(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
{Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
QName = <<"policy_overflow">>,
declare(Ch, QName),
DropHead = [{<<"overflow">>, <<"drop-head">>}],
RejectPub = [{<<"overflow">>, <<"reject-publish">>}],
Opts = #{config => Config,
server => Server,
qname => QName},
%% OperPolicy has precedence always
verify_policies(DropHead, RejectPub, RejectPub, Opts),
delete(Ch, QName),
rabbit_ct_broker_helpers:clear_policy(Config, 0, <<"policy">>),
rabbit_ct_broker_helpers:clear_operator_policy(Config, 0, <<"op_policy">>),
rabbit_ct_client_helpers:close_channel(Ch),
rabbit_ct_client_helpers:close_connection(Conn),
passed.
%% See supported policies in https://www.rabbitmq.com/parameters.html#operator-policies
%% This test applies all supported operator policies to all queue types,
%% and later verifies the effective policy definitions.
@ -351,6 +376,12 @@ is_supported_operator_policy_target_group_size(Config) ->
effective_operator_policy_per_queue_type(
Config, <<"target-group-size">>, Value, undefined, Value, undefined).
is_supported_operator_policy_overflow(Config) ->
Value = <<"drop-head">>,
effective_operator_policy_per_queue_type(
Config, <<"overflow">>, Value, Value, Value, undefined).
is_supported_operator_policy_ha(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
{Conn, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),

View File

@ -1792,7 +1792,7 @@ add_member_not_running(Config) ->
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
?assertEqual({error, node_not_running},
rpc:call(Server, rabbit_quorum_queue, add_member,
[<<"/">>, QQ, 'rabbit@burrow', 5000])).
[<<"/">>, QQ, 'rabbit@burrow', voter, 5000])).
add_member_classic(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -1801,7 +1801,7 @@ add_member_classic(Config) ->
?assertEqual({'queue.declare_ok', CQ, 0, 0}, declare(Ch, CQ, [])),
?assertEqual({error, classic_queue_not_supported},
rpc:call(Server, rabbit_quorum_queue, add_member,
[<<"/">>, CQ, Server, 5000])).
[<<"/">>, CQ, Server, voter, 5000])).
add_member_wrong_type(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -1811,7 +1811,7 @@ add_member_wrong_type(Config) ->
declare(Ch, SQ, [{<<"x-queue-type">>, longstr, <<"stream">>}])),
?assertEqual({error, not_quorum_queue},
rpc:call(Server, rabbit_quorum_queue, add_member,
[<<"/">>, SQ, Server, 5000])).
[<<"/">>, SQ, Server, voter, 5000])).
add_member_already_a_member(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -1822,14 +1822,14 @@ add_member_already_a_member(Config) ->
%% idempotent by design
?assertEqual(ok,
rpc:call(Server, rabbit_quorum_queue, add_member,
[<<"/">>, QQ, Server, 5000])).
[<<"/">>, QQ, Server, voter, 5000])).
add_member_not_found(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
QQ = ?config(queue_name, Config),
?assertEqual({error, not_found},
rpc:call(Server, rabbit_quorum_queue, add_member,
[<<"/">>, QQ, Server, 5000])).
[<<"/">>, QQ, Server, voter, 5000])).
add_member(Config) ->
[Server0, Server1] = Servers0 =
@ -1840,12 +1840,12 @@ add_member(Config) ->
declare(Ch, QQ, [{<<"x-queue-type">>, longstr, <<"quorum">>}])),
?assertEqual({error, node_not_running},
rpc:call(Server0, rabbit_quorum_queue, add_member,
[<<"/">>, QQ, Server1, 5000])),
[<<"/">>, QQ, Server1, voter, 5000])),
ok = rabbit_control_helper:command(stop_app, Server1),
ok = rabbit_control_helper:command(join_cluster, Server1, [atom_to_list(Server0)], []),
rabbit_control_helper:command(start_app, Server1),
?assertEqual(ok, rpc:call(Server1, rabbit_quorum_queue, add_member,
[<<"/">>, QQ, Server1, 5000])),
[<<"/">>, QQ, Server1, voter, 5000])),
Info = rpc:call(Server0, rabbit_quorum_queue, infos,
[rabbit_misc:r(<<"/">>, queue, QQ)]),
Servers = lists:sort(Servers0),

View File

@ -1135,37 +1135,6 @@ consume_with_autoack(Config) ->
subscribe(Ch1, Q, true, 0)),
rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
consume_and_nack(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
Q = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', Q, 0, 0},
declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])),
publish_confirm(Ch, Q, [<<"msg">>]),
Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server),
qos(Ch1, 10, false),
subscribe(Ch1, Q, false, 0),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag}, _} ->
ok = amqp_channel:cast(Ch1, #'basic.nack'{delivery_tag = DeliveryTag,
multiple = false,
requeue = true}),
%% Nack will throw a not implemented exception. As it is a cast operation,
%% we'll detect the conneciton/channel closure on the next call.
%% Let's try to redeclare and see what happens
?assertExit({{shutdown, {connection_closing, {server_initiated_close, 540, _}}}, _},
amqp_channel:call(Ch1, #'queue.declare'{queue = Q,
durable = true,
auto_delete = false,
arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}))
after 10000 ->
exit(timeout)
end,
rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
basic_cancel(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
@ -1370,42 +1339,13 @@ filter_consumers(Config, Server, CTag) ->
end, [], CInfo).
consume_and_reject(Config) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
Q = ?config(queue_name, Config),
?assertEqual({'queue.declare_ok', Q, 0, 0},
declare(Config, Server, Q, [{<<"x-queue-type">>, longstr, <<"stream">>}])),
publish_confirm(Ch, Q, [<<"msg">>]),
Ch1 = rabbit_ct_client_helpers:open_channel(Config, Server),
qos(Ch1, 10, false),
subscribe(Ch1, Q, false, 0),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag}, _} ->
MRef = erlang:monitor(process, Ch1),
ok = amqp_channel:cast(Ch1, #'basic.reject'{delivery_tag = DeliveryTag,
requeue = true}),
%% Reject will throw a not implemented exception. As it is a cast
%% operation, we detect the connection error from the channel
%% process exit reason.
receive
{'DOWN', MRef, _, _, Reason} ->
?assertMatch(
{shutdown,
{connection_closing,
{server_initiated_close, 540, _}}},
Reason)
after 10000 ->
exit(timeout)
end
after 10000 ->
exit(timeout)
end,
rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
consume_and_(Config, fun (DT) -> #'basic.reject'{delivery_tag = DT} end).
consume_and_nack(Config) ->
consume_and_(Config, fun (DT) -> #'basic.nack'{delivery_tag = DT} end).
consume_and_ack(Config) ->
consume_and_(Config, fun (DT) -> #'basic.ack'{delivery_tag = DT} end).
consume_and_(Config, AckFun) ->
[Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
@ -1420,8 +1360,7 @@ consume_and_ack(Config) ->
subscribe(Ch1, Q, false, 0),
receive
{#'basic.deliver'{delivery_tag = DeliveryTag}, _} ->
ok = amqp_channel:cast(Ch1, #'basic.ack'{delivery_tag = DeliveryTag,
multiple = false}),
ok = amqp_channel:cast(Ch1, AckFun(DeliveryTag)),
%% It will succeed as ack is now a credit operation. We should be
%% able to redeclare a queue (gen_server call op) as the channel
%% should still be open and declare is an idempotent operation

View File

@ -0,0 +1,49 @@
-module(unit_quorum_queue_SUITE).
-compile(export_all).
all() ->
[
all_replica_states_includes_nonvoters,
filter_quorum_critical_accounts_nonvoters
].
filter_quorum_critical_accounts_nonvoters(_Config) ->
Nodes = [test@leader, test@follower1, test@follower2],
Qs0 = [amqqueue:new(rabbit_misc:r(<<"/">>, queue, <<"q1">>),
{q1, test@leader},
false, false, none, [], undefined, #{}),
amqqueue:new(rabbit_misc:r(<<"/">>, queue, <<"q2">>),
{q2, test@leader},
false, false, none, [], undefined, #{})],
Qs = [Q1, Q2] = lists:map(fun (Q) ->
amqqueue:set_type_state(Q, #{nodes => Nodes})
end, Qs0),
Ss = #{test@leader => #{q1 => leader, q2 => leader},
test@follower1 => #{q1 => promotable, q2 => follower},
test@follower2 => #{q1 => follower, q2 => promotable}},
Qs = rabbit_quorum_queue:filter_quorum_critical(Qs, Ss, test@leader),
[Q2] = rabbit_quorum_queue:filter_quorum_critical(Qs, Ss, test@follower1),
[Q1] = rabbit_quorum_queue:filter_quorum_critical(Qs, Ss, test@follower2),
ok.
all_replica_states_includes_nonvoters(_Config) ->
ets:new(ra_state, [named_table, public, {write_concurrency, true}]),
ets:insert(ra_state, [
{q1, leader, voter},
{q2, follower, voter},
{q3, follower, promotable},
%% pre ra-2.7.0
{q4, leader},
{q5, follower}
]),
{_, #{
q1 := leader,
q2 := follower,
q3 := promotable,
q4 := leader,
q5 := follower
}} = rabbit_quorum_queue:all_replica_states(),
true = ets:delete(ra_state),
ok.

View File

@ -54,6 +54,7 @@ defmodule RabbitMQ.CLI.Core.DocGuide do
Macros.defguide("monitoring")
Macros.defguide("networking")
Macros.defguide("parameters")
Macros.defguide("passwords")
Macros.defguide("plugins")
Macros.defguide("prometheus")
Macros.defguide("publishers")

View File

@ -10,21 +10,37 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddUserCommand do
@behaviour RabbitMQ.CLI.CommandBehaviour
use RabbitMQ.CLI.Core.MergesNoDefaults
def switches(), do: [pre_hashed_password: :boolean]
def merge_defaults(args, opts) do
{args, Map.merge(%{pre_hashed_password: false}, opts)}
end
def validate(args, _) when length(args) < 1, do: {:validation_failure, :not_enough_args}
def validate(args, _) when length(args) > 2, do: {:validation_failure, :too_many_args}
def validate([_], _), do: :ok
# Password will be provided via standard input
def validate([_username], _), do: :ok
def validate(["", _], _) do
{:validation_failure, {:bad_argument, "user cannot be an empty string"}}
end
def validate([_, base64_encoded_password_hash], %{pre_hashed_password: true}) do
case Base.decode64(base64_encoded_password_hash) do
{:ok, _password_hash} ->
:ok
_ ->
{:validation_failure,
{:bad_argument, "Could not Base64 decode provided password hash value"}}
end
end
def validate([_, _], _), do: :ok
use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
def run([username], %{node: node_name} = opts) do
def run([username], %{node: node_name, pre_hashed_password: false} = opts) do
# note: blank passwords are currently allowed, they make sense
# e.g. when a user only authenticates using X.509 certificates.
# Credential validators can be used to require passwords of a certain length
@ -43,6 +59,46 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddUserCommand do
end
end
def run([username], %{node: node_name, pre_hashed_password: true} = opts) do
case Input.infer_password("Hashed and salted password: ", opts) do
:eof ->
{:error, :not_enough_args}
base64_encoded_password_hash ->
case Base.decode64(base64_encoded_password_hash) do
{:ok, password_hash} ->
:rabbit_misc.rpc_call(
node_name,
:rabbit_auth_backend_internal,
:add_user_with_pre_hashed_password_sans_validation,
[username, password_hash, Helpers.cli_acting_user()]
)
_ ->
{:error, ExitCodes.exit_dataerr(),
"Could not Base64 decode provided password hash value"}
end
end
end
def run(
[username, base64_encoded_password_hash],
%{node: node_name, pre_hashed_password: true} = opts
) do
case Base.decode64(base64_encoded_password_hash) do
{:ok, password_hash} ->
:rabbit_misc.rpc_call(
node_name,
:rabbit_auth_backend_internal,
:add_user_with_pre_hashed_password_sans_validation,
[username, password_hash, Helpers.cli_acting_user()]
)
_ ->
{:error, ExitCodes.exit_dataerr(), "Could not Base64 decode provided password hash value"}
end
end
def run([username, password], %{node: node_name}) do
:rabbit_misc.rpc_call(
node_name,
@ -89,21 +145,30 @@ defmodule RabbitMQ.CLI.Ctl.Commands.AddUserCommand do
use RabbitMQ.CLI.DefaultOutput
def usage, do: "add_user <username> <password>"
def usage, do: "add_user <username> [<password>] [<password_hash> --pre-hashed-password]"
def usage_additional() do
[
["<username>", "Self-explanatory"],
[
"<password>",
"Password this user will authenticate with. Use a blank string to disable password-based authentication."
"Password this user will authenticate with. Use a blank string to disable password-based authentication. Mutually exclusive with <password_hash>"
],
[
"<password_hash>",
"A Base64-encoded password hash produced by the 'hash_password' command or a different method as described in the Passwords guide. Must be used in combination with --pre-hashed-password. Mutually exclusive with <password>"
],
[
"--pre-hashed-password",
"Use to pass in a password hash instead of a clear text password. Disabled by default"
]
]
end
def usage_doc_guides() do
[
DocGuide.access_control()
DocGuide.access_control(),
DocGuide.passwords()
]
end

View File

@ -10,21 +10,51 @@ defmodule RabbitMQ.CLI.Queues.Commands.AddMemberCommand do
@behaviour RabbitMQ.CLI.CommandBehaviour
@default_timeout 5_000
defp default_opts, do: %{vhost: "/", membership: "promotable", timeout: 5_000}
def merge_defaults(args, opts) do
timeout =
case opts[:timeout] do
nil -> @default_timeout
:infinity -> @default_timeout
other -> other
end
default = default_opts()
{args, Map.merge(%{vhost: "/", timeout: timeout}, opts)}
opts =
Map.update(
opts,
:timeout,
:infinity,
&case &1 do
:infinity -> default.timeout
other -> other
end
)
{args, Map.merge(default, opts)}
end
use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
use RabbitMQ.CLI.Core.AcceptsTwoPositionalArguments
def switches(),
do: [
timeout: :integer,
membership: :string
]
def aliases(), do: [t: :timeout]
def validate(args, _) when length(args) < 2 do
{:validation_failure, :not_enough_args}
end
def validate(args, _) when length(args) > 2 do
{:validation_failure, :too_many_args}
end
def validate(_, %{membership: m})
when not (m == "promotable" or
m == "non_voter" or
m == "voter") do
{:validation_failure, "voter status '#{m}' is not recognised."}
end
def validate(_, _) do
:ok
end
def validate_execution_environment(args, opts) do
Validators.chain(
@ -39,13 +69,19 @@ defmodule RabbitMQ.CLI.Queues.Commands.AddMemberCommand do
)
end
def run([name, node] = _args, %{vhost: vhost, node: node_name, timeout: timeout}) do
case :rabbit_misc.rpc_call(node_name, :rabbit_quorum_queue, :add_member, [
vhost,
name,
to_atom(node),
timeout
]) do
def run(
[name, node] = _args,
%{vhost: vhost, node: node_name, timeout: timeout, membership: membership}
) do
args = [vhost, name, to_atom(node)]
args =
case to_atom(membership) do
:promotable -> args ++ [timeout]
other -> args ++ [other, timeout]
end
case :rabbit_misc.rpc_call(node_name, :rabbit_quorum_queue, :add_member, args) do
{:error, :classic_queue_not_supported} ->
{:error, "Cannot add members to a classic queue"}
@ -59,12 +95,13 @@ defmodule RabbitMQ.CLI.Queues.Commands.AddMemberCommand do
use RabbitMQ.CLI.DefaultOutput
def usage, do: "add_member [--vhost <vhost>] <queue> <node>"
def usage, do: "add_member [--vhost <vhost>] <queue> <node> [--membership <promotable|voter>]"
def usage_additional do
[
["<queue>", "quorum queue name"],
["<node>", "node to add a new replica on"]
["<node>", "node to add a new replica on"],
["--membership <promotable|voter>", "add a promotable non-voter (default) or full voter"]
]
end

View File

@ -10,12 +10,14 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommand do
@behaviour RabbitMQ.CLI.CommandBehaviour
defp default_opts, do: %{vhost_pattern: ".*", queue_pattern: ".*", errors_only: false}
defp default_opts,
do: %{vhost_pattern: ".*", queue_pattern: ".*", membership: "promotable", errors_only: false}
def switches(),
do: [
vhost_pattern: :string,
queue_pattern: :string,
membership: :string,
errors_only: :boolean
]
@ -31,17 +33,21 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommand do
{:validation_failure, :too_many_args}
end
def validate([_, s], _) do
case s do
"all" ->
:ok
def validate([_, s], _)
when not (s == "all" or
s == "even") do
{:validation_failure, "strategy '#{s}' is not recognised."}
end
"even" ->
:ok
def validate(_, %{membership: m})
when not (m == "promotable" or
m == "non_voter" or
m == "voter") do
{:validation_failure, "voter status '#{m}' is not recognised."}
end
_ ->
{:validation_failure, "strategy '#{s}' is not recognised."}
end
def validate(_, _) do
:ok
end
def validate_execution_environment(args, opts) do
@ -58,14 +64,18 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommand do
node: node_name,
vhost_pattern: vhost_pat,
queue_pattern: queue_pat,
membership: membership,
errors_only: errors_only
}) do
case :rabbit_misc.rpc_call(node_name, :rabbit_quorum_queue, :grow, [
to_atom(node),
vhost_pat,
queue_pat,
to_atom(strategy)
]) do
args = [to_atom(node), vhost_pat, queue_pat, to_atom(strategy)]
args =
case to_atom(membership) do
:promotable -> args
other -> args ++ [other]
end
case :rabbit_misc.rpc_call(node_name, :rabbit_quorum_queue, :grow, args) do
{:error, _} = error ->
error
@ -97,7 +107,8 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommand do
def formatter(), do: RabbitMQ.CLI.Formatters.Table
def usage,
do: "grow <node> <all | even> [--vhost-pattern <pattern>] [--queue-pattern <pattern>]"
do:
"grow <node> <all | even> [--vhost-pattern <pattern>] [--queue-pattern <pattern>] [--membership <promotable|voter>]"
def usage_additional do
[
@ -108,6 +119,7 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommand do
],
["--queue-pattern <pattern>", "regular expression to match queue names"],
["--vhost-pattern <pattern>", "regular expression to match virtual host names"],
["--membership <promotable|voter>", "add a promotable non-voter (default) or full voter"],
["--errors-only", "only list queues which reported an error"]
]
end

View File

@ -9,6 +9,8 @@ defmodule AddUserCommandTest do
import TestHelper
@command RabbitMQ.CLI.Ctl.Commands.AddUserCommand
@hash_password_command RabbitMQ.CLI.Ctl.Commands.HashPasswordCommand
@authenticate_user_command RabbitMQ.CLI.Ctl.Commands.AuthenticateUserCommand
setup_all do
RabbitMQ.CLI.Core.Distribution.start()
@ -18,7 +20,7 @@ defmodule AddUserCommandTest do
setup context do
on_exit(context, fn -> delete_user(context[:user]) end)
{:ok, opts: %{node: get_rabbit_hostname()}}
{:ok, opts: %{node: get_rabbit_hostname(), pre_hashed_password: false}}
end
test "validate: no positional arguments fails" do
@ -55,6 +57,17 @@ defmodule AddUserCommandTest do
assert @command.validate([context[:user], context[:password]], context[:opts]) == :ok
end
@tag user: "someone"
test "validate: pre-hashed with a non-Base64-encoded value returns an error", context do
hashed = "this is not a Base64-encoded value"
opts = Map.merge(context[:opts], %{pre_hashed_password: true})
assert match?(
{:validation_failure, {:bad_argument, _}},
@command.validate([context[:user], hashed], opts)
)
end
@tag user: "someone", password: "password"
test "run: request to a non-existent node returns a badrpc", context do
opts = %{node: :jake@thedog, timeout: 200}
@ -62,9 +75,30 @@ defmodule AddUserCommandTest do
end
@tag user: "someone", password: "password"
test "run: default case completes successfully", context do
test "run: happy path completes successfully", context do
assert @command.run([context[:user], context[:password]], context[:opts]) == :ok
assert list_users() |> Enum.count(fn record -> record[:user] == context[:user] end) == 1
assert @authenticate_user_command.run([context[:user], context[:password]], context[:opts])
end
@tag user: "someone"
test "run: a pre-hashed request to a non-existent node returns a badrpc", context do
opts = %{node: :jake@thedog, timeout: 200}
hashed = "BMT6cj/MsI+4UOBtsPPQWpQfk7ViRLj4VqpMTxu54FU3qa1G"
assert match?({:badrpc, _}, @command.run([context[:user], hashed], opts))
end
@tag user: "someone"
test "run: pre-hashed happy path completes successfully", context do
pwd = "guest10"
hashed = @hash_password_command.hash_password(pwd)
opts = Map.merge(%{pre_hashed_password: true}, context[:opts])
assert @command.run([context[:user], hashed], opts) == :ok
assert list_users() |> Enum.count(fn record -> record[:user] == context[:user] end) == 1
assert @authenticate_user_command.run([context[:user], pwd], opts)
end
@tag user: "someone", password: "password"

View File

@ -20,6 +20,8 @@ defmodule RabbitMQ.CLI.Queues.Commands.AddMemberCommandTest do
{:ok,
opts: %{
node: get_rabbit_hostname(),
membership: "voter",
vhost: "/",
timeout: context[:test_timeout] || 30000
}}
end
@ -42,17 +44,36 @@ defmodule RabbitMQ.CLI.Queues.Commands.AddMemberCommandTest do
) == {:validation_failure, :too_many_args}
end
test "validate: when membership promotable is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "rabbit@new-node"], %{membership: "promotable"}) ==
:ok
end
test "validate: when membership voter is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "rabbit@new-node"], %{membership: "voter"}) == :ok
end
test "validate: when membership non_voter is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "rabbit@new-node"], %{membership: "non_voter"}) ==
:ok
end
test "validate: when wrong membership is provided, returns failure" do
assert @command.validate(["quorum-queue-a", "rabbit@new-node"], %{membership: "banana"}) ==
{:validation_failure, "voter status 'banana' is not recognised."}
end
test "validate: treats two positional arguments and default switches as a success" do
assert @command.validate(["quorum-queue-a", "rabbit@new-node"], %{}) == :ok
end
@tag test_timeout: 3000
test "run: targeting an unreachable node throws a badrpc" do
test "run: targeting an unreachable node throws a badrpc", context do
assert match?(
{:badrpc, _},
@command.run(
["quorum-queue-a", "rabbit@new-node"],
%{node: :jake@thedog, vhost: "/", timeout: 200}
Map.merge(context[:opts], %{node: :jake@thedog})
)
)
end

View File

@ -23,13 +23,20 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommandTest do
timeout: context[:test_timeout] || 30000,
vhost_pattern: ".*",
queue_pattern: ".*",
membership: "promotable",
errors_only: false
}}
end
test "merge_defaults: defaults to reporting complete results" do
assert @command.merge_defaults([], %{}) ==
{[], %{vhost_pattern: ".*", queue_pattern: ".*", errors_only: false}}
{[],
%{
vhost_pattern: ".*",
queue_pattern: ".*",
errors_only: false,
membership: "promotable"
}}
end
test "validate: when no arguments are provided, returns a failure" do
@ -58,13 +65,30 @@ defmodule RabbitMQ.CLI.Queues.Commands.GrowCommandTest do
{:validation_failure, :too_many_args}
end
test "validate: when membership promotable is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "all"], %{membership: "promotable"}) == :ok
end
test "validate: when membership voter is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "all"], %{membership: "voter"}) == :ok
end
test "validate: when membership non_voter is provided, returns a success" do
assert @command.validate(["quorum-queue-a", "all"], %{membership: "non_voter"}) == :ok
end
test "validate: when wrong membership is provided, returns failure" do
assert @command.validate(["quorum-queue-a", "all"], %{membership: "banana"}) ==
{:validation_failure, "voter status 'banana' is not recognised."}
end
@tag test_timeout: 3000
test "run: targeting an unreachable node throws a badrpc", context do
assert match?(
{:badrpc, _},
@command.run(
["quorum-queue-a", "all"],
Map.merge(context[:opts], %{node: :jake@thedog, timeout: 200})
Map.merge(context[:opts], %{node: :jake@thedog})
)
)
end

View File

@ -289,8 +289,9 @@
<span class="argument-link" field="definitionop" key="max-length" type="number">Max length</span> |
<span class="argument-link" field="definitionop" key="max-length-bytes" type="number">Max length bytes</span> |
<span class="argument-link" field="definitionop" key="message-ttl" type="number">Message TTL</span> |
<span class="argument-link" field="definitionop" key="queue-version" type="number">Version</span> <span class="help" id="queue-version"></span> </br>
<span class="help" id="queue-message-ttl"></span>
<span class="help" id="queue-message-ttl"></span> |
<span class="argument-link" field="definitionop" key="queue-version" type="number">Version</span> <span class="help" id="queue-version"></span> </br> |
<span class="argument-link" field="definitionop" key="overflow" type="string">Length limit overflow behavior</span> <span class="help" id="overflow"></span> </br>
</td>
</tr>
<tr>
@ -305,7 +306,8 @@
<span class="argument-link" field="definitionop" key="max-length-bytes" type="number">Max length bytes</span> |
<span class="argument-link" field="definitionop" key="message-ttl" type="number">Message TTL</span>
<span class="help" id="queue-message-ttl"></span> |
<span class="argument-link" field="definitionop" key="target-group-size" type="number">Target group size</span>
<span class="argument-link" field="definitionop" key="target-group-size" type="number">Target group size</span> |
<span class="argument-link" field="definitionop" key="overflow" type="string">Length limit overflow behavior</span> <span class="help" id="overflow"></span> </br>
</td>
</tr>
<tr>

View File

@ -38,11 +38,17 @@ accept_content(ReqData, Context) ->
QName = rabbit_mgmt_util:id(queue, ReqData),
Res = rabbit_mgmt_util:with_decode(
[node], ReqData, Context,
fun([NewReplicaNode], _Body, _ReqData) ->
fun([NewReplicaNode], Body, _ReqData) ->
Membership = maps:get(<<"membership">>, Body, promotable),
rabbit_amqqueue:with(
rabbit_misc:r(VHost, queue, QName),
fun(_Q) ->
rabbit_quorum_queue:add_member(VHost, QName, rabbit_data_coercion:to_atom(NewReplicaNode), ?TIMEOUT)
rabbit_quorum_queue:add_member(
VHost,
QName,
rabbit_data_coercion:to_atom(NewReplicaNode),
rabbit_data_coercion:to_atom(Membership),
?TIMEOUT)
end)
end),
case Res of

View File

@ -39,12 +39,14 @@ accept_content(ReqData, Context) ->
NewReplicaNode = rabbit_mgmt_util:id(node, ReqData),
rabbit_mgmt_util:with_decode(
[vhost_pattern, queue_pattern, strategy], ReqData, Context,
fun([VHPattern, QPattern, Strategy], _Body, _ReqData) ->
fun([VHPattern, QPattern, Strategy], Body, _ReqData) ->
Membership = maps:get(<<"membership">>, Body, promotable),
rabbit_quorum_queue:grow(
rabbit_data_coercion:to_atom(NewReplicaNode),
VHPattern,
QPattern,
rabbit_data_coercion:to_atom(Strategy))
rabbit_data_coercion:to_atom(Strategy),
rabbit_data_coercion:to_atom(Membership))
end),
{true, ReqData, Context}.

View File

@ -44,7 +44,7 @@
-type send_fun() :: fun((iodata()) -> ok).
-type session_expiry_interval() :: non_neg_integer() | infinity.
-type subscriptions() :: #{topic_filter() => #mqtt_subscription_opts{}}.
-type topic_aliases() :: {Inbound :: #{topic() => pos_integer()},
-type topic_aliases() :: {Inbound :: #{pos_integer() => topic()},
Outbound :: #{topic() => pos_integer()}}.
-record(auth_state,

View File

@ -16,6 +16,7 @@ def all_beam_files(name = "all_beam_files"):
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamGroupConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamPublishersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand.erl",
"src/rabbit_stream.erl",
"src/rabbit_stream_connection_sup.erl",
"src/rabbit_stream_manager.erl",
@ -55,6 +56,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamGroupConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamPublishersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand.erl",
"src/rabbit_stream.erl",
"src/rabbit_stream_connection_sup.erl",
"src/rabbit_stream_manager.erl",
@ -104,6 +106,7 @@ def all_srcs(name = "all_srcs"):
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamGroupConsumersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamPublishersCommand.erl",
"src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand.erl",
"src/rabbit_stream.erl",
"src/rabbit_stream_connection_sup.erl",
"src/rabbit_stream_manager.erl",

View File

@ -54,7 +54,7 @@ help_section() ->
{plugin, stream}.
validate(Args, _) ->
ValidKeys = lists:map(fun atom_to_list/1, ?CONSUMER_INFO_ITEMS),
ValidKeys = lists:map(fun atom_to_list/1, ?CONSUMER_GROUP_INFO_ITEMS),
case 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':validate_info_keys(Args,
ValidKeys)
of

View File

@ -0,0 +1,162 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 2.0 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at https://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved.
-module('Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand').
-include_lib("rabbitmq_stream_common/include/rabbit_stream.hrl").
-behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour').
-export([formatter/0,
scopes/0,
switches/0,
aliases/0,
usage/0,
usage_additional/0,
usage_doc_guides/0,
banner/2,
validate/2,
merge_defaults/2,
run/2,
output/2,
description/0,
help_section/0,
tracking_info/3]).
formatter() ->
'Elixir.RabbitMQ.CLI.Formatters.PrettyTable'.
scopes() ->
[streams].
switches() ->
[{stream, string}, {all, boolean}, {offset, boolean}, {writer, boolean}].
aliases() ->
[].
description() ->
<<"Lists tracking information for a stream">>.
help_section() ->
{plugin, stream}.
validate([], _Opts) ->
{validation_failure, not_enough_args};
validate([_Stream], Opts) ->
case maps:with([all, writer, offset], Opts) of
M when map_size(M) > 1 ->
{validation_failure,
"Specify only one of --all, --offset, --writer."};
_ ->
ok
end;
validate(_, _Opts) ->
{validation_failure, too_many_args}.
merge_defaults(Args, Opts) ->
case maps:with([all, writer, offset], Opts) of
M when map_size(M) =:= 0 ->
{Args, maps:merge(#{all => true, vhost => <<"/">>}, Opts)};
_ ->
{Args, maps:merge(#{vhost => <<"/">>}, Opts)}
end.
usage() ->
<<"list_stream_tracking <stream> [--all | --offset | --writer] "
"[--vhost <vhost>]">>.
usage_additional() ->
[[<<"<name>">>,
<<"The name of the stream.">>],
[<<"--all">>,
<<"List offset and writer tracking information.">>],
[<<"--offset">>,
<<"List only offset tracking information.">>],
[<<"--writer">>,
<<"List only writer deduplication tracking information.">>],
[<<"--vhost <vhost>">>,
<<"The virtual host of the stream.">>]].
usage_doc_guides() ->
[?STREAM_GUIDE_URL].
run([Stream],
#{node := NodeName,
vhost := VHost,
timeout := Timeout} = Opts) ->
TrackingType = case Opts of
#{all := true} ->
all;
#{offset := true} ->
offset;
#{writer := true} ->
writer
end,
case rabbit_misc:rpc_call(NodeName,
?MODULE,
tracking_info,
[VHost, Stream, TrackingType],
Timeout) of
{error, not_found} ->
{error, "The stream does not exist."};
{error, not_available} ->
{error, "The stream is not available."};
{error, _} = E ->
E;
R ->
R
end.
banner([Stream], _) ->
<<"Listing tracking information for stream ", Stream/binary, <<" ...">>/binary>>.
output({ok, []}, _Opts) ->
ok;
output([], _Opts) ->
ok;
output(Result, _Opts) ->
'Elixir.RabbitMQ.CLI.DefaultOutput':output(Result).
tracking_info(VHost, Stream, TrackingType) ->
case rabbit_stream_manager:lookup_leader(VHost, Stream) of
{ok, Leader} ->
TrackingInfo = osiris:read_tracking(Leader),
FieldsLabels = case TrackingType of
all ->
[{offsets, offset}, {sequences, writer}];
offset ->
[{offsets, offset}];
writer ->
[{sequences, writer}]
end,
lists:foldl(fun({F, L}, Acc) ->
Tracking = maps:get(F, TrackingInfo, #{}),
maps:fold(fun(Reference, {_, Sequence}, AccType) ->
[[{type, L},
{name, Reference},
{tracking_value, Sequence}
] | AccType];
(Reference, Offset, AccType) ->
[[{type, L},
{name, Reference},
{tracking_value, Offset}
] | AccType]
end, Acc, Tracking)
end, [], FieldsLabels);
{error, _} = E ->
E
end.

View File

@ -31,6 +31,8 @@
'Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamConsumerGroupsCommand').
-define(COMMAND_LIST_GROUP_CONSUMERS,
'Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamGroupConsumersCommand').
-define(COMMAND_LIST_STREAM_TRACKING,
'Elixir.RabbitMQ.CLI.Ctl.Commands.ListStreamTrackingCommand').
all() ->
[{group, list_connections},
@ -38,6 +40,7 @@ all() ->
{group, list_publishers},
{group, list_consumer_groups},
{group, list_group_consumers},
{group, list_stream_tracking},
{group, super_streams}].
groups() ->
@ -49,10 +52,14 @@ groups() ->
{list_publishers, [],
[list_publishers_merge_defaults, list_publishers_run]},
{list_consumer_groups, [],
[list_consumer_groups_merge_defaults, list_consumer_groups_run]},
[list_consumer_groups_validate, list_consumer_groups_merge_defaults,
list_consumer_groups_run]},
{list_group_consumers, [],
[list_group_consumers_validate, list_group_consumers_merge_defaults,
list_group_consumers_run]},
{list_stream_tracking, [],
[list_stream_tracking_validate, list_stream_tracking_merge_defaults,
list_stream_tracking_run]},
{super_streams, [],
[add_super_stream_merge_defaults,
add_super_stream_validate,
@ -332,6 +339,18 @@ list_publishers_run(Config) ->
?awaitMatch(0, publisher_count(Config), ?WAIT),
ok.
list_consumer_groups_validate(_) ->
ValidOpts = #{vhost => <<"/">>},
?assertMatch({validation_failure, {bad_info_key, [foo]}},
?COMMAND_LIST_CONSUMER_GROUPS:validate([<<"foo">>],
ValidOpts)),
?assertMatch(ok,
?COMMAND_LIST_CONSUMER_GROUPS:validate([<<"reference">>],
ValidOpts)),
?assertMatch(ok,
?COMMAND_LIST_CONSUMER_GROUPS:validate([], ValidOpts)).
list_consumer_groups_merge_defaults(_Config) ->
DefaultItems =
[rabbit_data_coercion:to_binary(Item)
@ -521,6 +540,106 @@ assertConsumerGroup(S, R, PI, Cs, Record) ->
?assertEqual(Cs, proplists:get_value(consumers, Record)),
ok.
list_stream_tracking_validate(_) ->
ValidOpts = #{vhost => <<"/">>, <<"writer">> => true},
?assertMatch({validation_failure, not_enough_args},
?COMMAND_LIST_STREAM_TRACKING:validate([], #{})),
?assertMatch({validation_failure, not_enough_args},
?COMMAND_LIST_STREAM_TRACKING:validate([],
#{vhost =>
<<"test">>})),
?assertMatch({validation_failure, "Specify only one of --all, --offset, --writer."},
?COMMAND_LIST_STREAM_TRACKING:validate([<<"stream">>],
#{all => true, writer => true})),
?assertMatch({validation_failure, too_many_args},
?COMMAND_LIST_STREAM_TRACKING:validate([<<"stream">>, <<"bad">>],
ValidOpts)),
?assertMatch(ok,
?COMMAND_LIST_STREAM_TRACKING:validate([<<"stream">>],
ValidOpts)).
list_stream_tracking_merge_defaults(_Config) ->
?assertMatch({[<<"s">>], #{all := true, vhost := <<"/">>}},
?COMMAND_LIST_STREAM_TRACKING:merge_defaults([<<"s">>], #{})),
?assertMatch({[<<"s">>], #{writer := true, vhost := <<"/">>}},
?COMMAND_LIST_STREAM_TRACKING:merge_defaults([<<"s">>], #{writer => true})),
?assertMatch({[<<"s">>], #{all := true, vhost := <<"dev">>}},
?COMMAND_LIST_STREAM_TRACKING:merge_defaults([<<"s">>], #{vhost => <<"dev">>})),
?assertMatch({[<<"s">>], #{writer := true, vhost := <<"dev">>}},
?COMMAND_LIST_STREAM_TRACKING:merge_defaults([<<"s">>], #{writer => true, vhost => <<"dev">>})).
list_stream_tracking_run(Config) ->
Node = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
Stream = <<"list_stream_tracking_run">>,
ConsumerReference = <<"foo">>,
PublisherReference = <<"bar">>,
Opts =
#{node => Node,
timeout => 10000,
vhost => <<"/">>},
Args = [Stream],
%% the stream does not exist yet
?assertMatch({error, "The stream does not exist."},
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{all => true})),
StreamPort = rabbit_stream_SUITE:get_stream_port(Config),
{S, C} = start_stream_connection(StreamPort),
?awaitMatch(1, connection_count(Config), ?WAIT),
create_stream(S, Stream, C),
?assertMatch([],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{all => true})),
store_offset(S, Stream, ConsumerReference, 42, C),
?assertMatch([[{type,offset}, {name, ConsumerReference}, {tracking_value, 42}]],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{all => true})),
?assertMatch([[{type,offset}, {name, ConsumerReference}, {tracking_value, 42}]],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{offset => true})),
ok = store_offset(S, Stream, ConsumerReference, 55, C),
?assertMatch([[{type,offset}, {name, ConsumerReference}, {tracking_value, 55}]],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{offset => true})),
PublisherId = 1,
rabbit_stream_SUITE:test_declare_publisher(gen_tcp, S, PublisherId,
PublisherReference, Stream, C),
rabbit_stream_SUITE:test_publish_confirm(gen_tcp, S, PublisherId, 42, <<"">>, C),
ok = check_publisher_sequence(S, Stream, PublisherReference, 42, C),
?assertMatch([
[{type,writer},{name,<<"bar">>},{tracking_value, 42}],
[{type,offset},{name,<<"foo">>},{tracking_value, 55}]
],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{all => true})),
?assertMatch([
[{type,writer},{name,<<"bar">>},{tracking_value, 42}]
],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{writer => true})),
rabbit_stream_SUITE:test_publish_confirm(gen_tcp, S, PublisherId, 66, <<"">>, C),
ok = check_publisher_sequence(S, Stream, PublisherReference, 66, C),
?assertMatch([
[{type,writer},{name,<<"bar">>},{tracking_value, 66}]
],
?COMMAND_LIST_STREAM_TRACKING:run(Args, Opts#{writer => true})),
delete_stream(S, Stream, C),
close(S, C),
ok.
add_super_stream_merge_defaults(_Config) ->
?assertMatch({[<<"super-stream">>],
#{partitions := 3, vhost := <<"/">>}},
@ -714,6 +833,9 @@ declare_publisher(S, PubId, Stream, C) ->
delete_stream(S, Stream, C) ->
rabbit_stream_SUITE:test_delete_stream(gen_tcp, S, Stream, C).
delete_stream_no_metadata_update(S, Stream, C) ->
rabbit_stream_SUITE:test_delete_stream(gen_tcp, S, Stream, C, false).
metadata_update_stream_deleted(S, Stream, C) ->
rabbit_stream_SUITE:test_metadata_update_stream_deleted(gen_tcp,
S,
@ -798,3 +920,52 @@ queue_lookup(Config, Q) ->
rabbit_amqqueue,
lookup,
[QueueName]).
store_offset(S, Stream, Reference, Value, C) ->
StoreOffsetFrame =
rabbit_stream_core:frame({store_offset, Reference, Stream, Value}),
ok = gen_tcp:send(S, StoreOffsetFrame),
case check_stored_offset(S, Stream, Reference, Value, C, 20) of
ok ->
ok;
_ ->
{error, offset_not_stored}
end.
check_stored_offset(_, _, _, _, _, 0) ->
error;
check_stored_offset(S, Stream, Reference, Expected, C, Attempt) ->
QueryOffsetFrame =
rabbit_stream_core:frame({request, 1, {query_offset, Reference, Stream}}),
ok = gen_tcp:send(S, QueryOffsetFrame),
{Cmd, _} = rabbit_stream_SUITE:receive_commands(gen_tcp, S, C),
?assertMatch({response, 1, {query_offset, ?RESPONSE_CODE_OK, _}}, Cmd),
{response, 1, {query_offset, ?RESPONSE_CODE_OK, StoredValue}} = Cmd,
case StoredValue of
Expected ->
ok;
_ ->
timer:sleep(50),
check_stored_offset(S, Stream, Reference, Expected, C, Attempt - 1)
end.
check_publisher_sequence(S, Stream, Reference, Expected, C) ->
check_publisher_sequence(S, Stream, Reference, Expected, C, 20).
check_publisher_sequence(_, _, _, _, _, 0) ->
error;
check_publisher_sequence(S, Stream, Reference, Expected, C, Attempt) ->
QueryFrame =
rabbit_stream_core:frame({request, 1, {query_publisher_sequence, Reference, Stream}}),
ok = gen_tcp:send(S, QueryFrame),
{Cmd, _} = rabbit_stream_SUITE:receive_commands(gen_tcp, S, C),
?assertMatch({response, 1, {query_publisher_sequence, _, _}}, Cmd),
{response, 1, {query_publisher_sequence, _, StoredValue}} = Cmd,
case StoredValue of
Expected ->
ok;
_ ->
timer:sleep(50),
check_publisher_sequence(S, Stream, Reference, Expected, C, Attempt - 1)
end.

View File

@ -724,11 +724,14 @@ test_metadata_update_stream_deleted(Transport, S, Stream, C0) ->
C1.
test_declare_publisher(Transport, S, PublisherId, Stream, C0) ->
test_declare_publisher(Transport, S, PublisherId, <<>>, Stream, C0).
test_declare_publisher(Transport, S, PublisherId, Reference, Stream, C0) ->
DeclarePublisherFrame =
rabbit_stream_core:frame({request, 1,
{declare_publisher,
PublisherId,
<<>>,
Reference,
Stream}}),
ok = Transport:send(S, DeclarePublisherFrame),
{Cmd, C} = receive_commands(Transport, S, C0),
@ -737,25 +740,35 @@ test_declare_publisher(Transport, S, PublisherId, Stream, C0) ->
C.
test_publish_confirm(Transport, S, PublisherId, Body, C0) ->
test_publish_confirm(Transport, S, publish, PublisherId, Body,
test_publish_confirm(Transport, S, PublisherId, 1, Body, C0).
test_publish_confirm(Transport, S, PublisherId, Sequence, Body, C0) ->
test_publish_confirm(Transport, S, publish, PublisherId, Sequence, Body,
publish_confirm, C0).
test_publish_confirm(Transport, S, publish = PublishCmd, PublisherId, Body,
ExpectedConfirmCommand,C0) ->
test_publish_confirm(Transport, S, PublishCmd, PublisherId, Body,
ExpectedConfirmCommand, C0) ->
test_publish_confirm(Transport, S, PublishCmd, PublisherId, 1, Body,
ExpectedConfirmCommand, C0).
test_publish_confirm(Transport, S, publish = PublishCmd, PublisherId,
Sequence, Body,
ExpectedConfirmCommand, C0) ->
BodySize = byte_size(Body),
Messages = [<<1:64, 0:1, BodySize:31, Body:BodySize/binary>>],
Messages = [<<Sequence:64, 0:1, BodySize:31, Body:BodySize/binary>>],
PublishFrame =
rabbit_stream_core:frame({PublishCmd, PublisherId, 1, Messages}),
ok = Transport:send(S, PublishFrame),
{Cmd, C} = receive_commands(Transport, S, C0),
?assertMatch({ExpectedConfirmCommand, PublisherId, [1]}, Cmd),
?assertMatch({ExpectedConfirmCommand, PublisherId, [Sequence]}, Cmd),
C;
test_publish_confirm(Transport, S, publish_v2 = PublishCmd, PublisherId, Body,
test_publish_confirm(Transport, S, publish_v2 = PublishCmd, PublisherId,
Sequence, Body,
ExpectedConfirmCommand, C0) ->
BodySize = byte_size(Body),
FilterValue = <<"foo">>,
FilterValueSize = byte_size(FilterValue),
Messages = [<<1:64, FilterValueSize:16, FilterValue:FilterValueSize/binary,
Messages = [<<Sequence:64, FilterValueSize:16, FilterValue:FilterValueSize/binary,
0:1, BodySize:31, Body:BodySize/binary>>],
PublishFrame =
rabbit_stream_core:frame({PublishCmd, PublisherId, 1, Messages}),
@ -763,9 +776,9 @@ test_publish_confirm(Transport, S, publish_v2 = PublishCmd, PublisherId, Body,
{Cmd, C} = receive_commands(Transport, S, C0),
case ExpectedConfirmCommand of
publish_confirm ->
?assertMatch({ExpectedConfirmCommand, PublisherId, [1]}, Cmd);
?assertMatch({ExpectedConfirmCommand, PublisherId, [Sequence]}, Cmd);
publish_error ->
?assertMatch({ExpectedConfirmCommand, PublisherId, _, [1]}, Cmd)
?assertMatch({ExpectedConfirmCommand, PublisherId, _, [Sequence]}, Cmd)
end,
C.

View File

@ -18,6 +18,7 @@ def all_beam_files(name = "all_beam_files"):
"src/rabbit_stream_management_utils.erl",
"src/rabbit_stream_mgmt_db.erl",
"src/rabbit_stream_publishers_mgmt.erl",
"src/rabbit_stream_tracking_mgmt.erl",
],
hdrs = [":public_and_private_hdrs"],
app_name = "rabbitmq_stream_management",
@ -51,6 +52,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
"src/rabbit_stream_management_utils.erl",
"src/rabbit_stream_mgmt_db.erl",
"src/rabbit_stream_publishers_mgmt.erl",
"src/rabbit_stream_tracking_mgmt.erl",
],
hdrs = [":public_and_private_hdrs"],
app_name = "rabbitmq_stream_management",
@ -101,6 +103,7 @@ def all_srcs(name = "all_srcs"):
"src/rabbit_stream_management_utils.erl",
"src/rabbit_stream_mgmt_db.erl",
"src/rabbit_stream_publishers_mgmt.erl",
"src/rabbit_stream_tracking_mgmt.erl",
],
)
filegroup(

View File

@ -0,0 +1,102 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2023 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_stream_tracking_mgmt).
-behaviour(rabbit_mgmt_extension).
-export([dispatcher/0,
web_ui/0]).
-export([init/2,
resource_exists/2,
to_json/2,
content_types_provided/2,
is_authorized/2]).
-include_lib("rabbitmq_management_agent/include/rabbit_mgmt_records.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
dispatcher() ->
[{"/stream/:vhost/:queue/tracking", ?MODULE, []}].
web_ui() ->
[].
%%--------------------------------------------------------------------
init(Req, _Opts) ->
{cowboy_rest, rabbit_mgmt_cors:set_headers(Req, ?MODULE), #context{}}.
content_types_provided(ReqData, Context) ->
{[{<<"application/json">>, to_json}], ReqData, Context}.
resource_exists(ReqData, Context) ->
{case rabbit_mgmt_util:vhost(ReqData) of
not_found ->
false;
none ->
false; % none means `all`
_ ->
case rabbit_mgmt_util:id(queue, ReqData) of
none ->
false;
_ ->
case rabbit_mgmt_wm_queue:queue(ReqData) of
not_found ->
false;
_ ->
true
end
end
end,
ReqData, Context}.
to_json(ReqData, Context) ->
case rabbit_mgmt_util:disable_stats(ReqData) of
false ->
VHost = rabbit_mgmt_util:vhost(ReqData),
Stream = rabbit_mgmt_util:id(queue, ReqData),
case rabbit_stream_manager:lookup_leader(VHost, Stream) of
{ok, Leader} ->
Type = tracking_type(rabbit_mgmt_util:get_value_param(<<"type">>, ReqData)),
TrackingInfo = maps:remove(timestamps, osiris:read_tracking(Leader)),
rabbit_mgmt_util:reply(transform_tracking(Type, TrackingInfo),
ReqData,
Context);
{error, _} ->
rabbit_mgmt_util:service_unavailable(<<"The stream leader is not available">>,
ReqData, Context)
end;
true ->
rabbit_mgmt_util:bad_request(<<"Stats in management UI are disabled on this node">>,
ReqData, Context)
end.
tracking_type(undefined) ->
all;
tracking_type("offset") ->
offset;
tracking_type("writer") ->
writer;
tracking_type(_) ->
all.
transform_tracking(offset, Tracking) ->
maps:remove(sequences, Tracking);
transform_tracking(writer, Tracking) ->
#{writers => convert_writer_tracking(maps:get(sequences, Tracking))};
transform_tracking(all, Tracking) ->
#{offsets => maps:get(offsets, Tracking),
writers => convert_writer_tracking(maps:get(sequences, Tracking))}.
convert_writer_tracking(Writers) ->
maps:fold(fun(Ref, {_, Seq}, Acc) ->
Acc#{Ref => Seq}
end, #{}, Writers).
is_authorized(ReqData, Context) ->
rabbit_mgmt_util:is_authorized(ReqData, Context).

View File

@ -1,117 +0,0 @@
/*
* Copyright 2007-present the original author or authors.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.net.*;
import java.io.*;
import java.nio.channels.*;
import java.util.Properties;
public class MavenWrapperDownloader {
private static final String WRAPPER_VERSION = "0.5.6";
/**
* Default URL to download the maven-wrapper.jar from, if no 'downloadUrl' is provided.
*/
private static final String DEFAULT_DOWNLOAD_URL = "https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/"
+ WRAPPER_VERSION + "/maven-wrapper-" + WRAPPER_VERSION + ".jar";
/**
* Path to the maven-wrapper.properties file, which might contain a downloadUrl property to
* use instead of the default one.
*/
private static final String MAVEN_WRAPPER_PROPERTIES_PATH =
".mvn/wrapper/maven-wrapper.properties";
/**
* Path where the maven-wrapper.jar will be saved to.
*/
private static final String MAVEN_WRAPPER_JAR_PATH =
".mvn/wrapper/maven-wrapper.jar";
/**
* Name of the property which should be used to override the default download url for the wrapper.
*/
private static final String PROPERTY_NAME_WRAPPER_URL = "wrapperUrl";
public static void main(String args[]) {
System.out.println("- Downloader started");
File baseDirectory = new File(args[0]);
System.out.println("- Using base directory: " + baseDirectory.getAbsolutePath());
// If the maven-wrapper.properties exists, read it and check if it contains a custom
// wrapperUrl parameter.
File mavenWrapperPropertyFile = new File(baseDirectory, MAVEN_WRAPPER_PROPERTIES_PATH);
String url = DEFAULT_DOWNLOAD_URL;
if(mavenWrapperPropertyFile.exists()) {
FileInputStream mavenWrapperPropertyFileInputStream = null;
try {
mavenWrapperPropertyFileInputStream = new FileInputStream(mavenWrapperPropertyFile);
Properties mavenWrapperProperties = new Properties();
mavenWrapperProperties.load(mavenWrapperPropertyFileInputStream);
url = mavenWrapperProperties.getProperty(PROPERTY_NAME_WRAPPER_URL, url);
} catch (IOException e) {
System.out.println("- ERROR loading '" + MAVEN_WRAPPER_PROPERTIES_PATH + "'");
} finally {
try {
if(mavenWrapperPropertyFileInputStream != null) {
mavenWrapperPropertyFileInputStream.close();
}
} catch (IOException e) {
// Ignore ...
}
}
}
System.out.println("- Downloading from: " + url);
File outputFile = new File(baseDirectory.getAbsolutePath(), MAVEN_WRAPPER_JAR_PATH);
if(!outputFile.getParentFile().exists()) {
if(!outputFile.getParentFile().mkdirs()) {
System.out.println(
"- ERROR creating output directory '" + outputFile.getParentFile().getAbsolutePath() + "'");
}
}
System.out.println("- Downloading to: " + outputFile.getAbsolutePath());
try {
downloadFileFromURL(url, outputFile);
System.out.println("Done");
System.exit(0);
} catch (Throwable e) {
System.out.println("- Error downloading");
e.printStackTrace();
System.exit(1);
}
}
private static void downloadFileFromURL(String urlString, File destination) throws Exception {
if (System.getenv("MVNW_USERNAME") != null && System.getenv("MVNW_PASSWORD") != null) {
String username = System.getenv("MVNW_USERNAME");
char[] password = System.getenv("MVNW_PASSWORD").toCharArray();
Authenticator.setDefault(new Authenticator() {
@Override
protected PasswordAuthentication getPasswordAuthentication() {
return new PasswordAuthentication(username, password);
}
});
}
URL website = new URL(urlString);
ReadableByteChannel rbc;
rbc = Channels.newChannel(website.openStream());
FileOutputStream fos = new FileOutputStream(destination);
fos.getChannel().transferFrom(rbc, 0, Long.MAX_VALUE);
fos.close();
rbc.close();
}
}

View File

@ -1,2 +1,18 @@
distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.6.3/apache-maven-3.6.3-bin.zip
wrapperUrl=https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/0.5.6/maven-wrapper-0.5.6.jar
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.5/apache-maven-3.9.5-bin.zip
wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar

View File

@ -19,7 +19,7 @@
# ----------------------------------------------------------------------------
# ----------------------------------------------------------------------------
# Maven Start Up Batch script
# Apache Maven Wrapper startup batch script, version 3.2.0
#
# Required ENV vars:
# ------------------
@ -27,7 +27,6 @@
#
# Optional ENV vars
# -----------------
# M2_HOME - location of maven2's installed home dir
# MAVEN_OPTS - parameters passed to the Java VM when running Maven
# e.g. to debug Maven itself, use
# set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000
@ -36,6 +35,10 @@
if [ -z "$MAVEN_SKIP_RC" ] ; then
if [ -f /usr/local/etc/mavenrc ] ; then
. /usr/local/etc/mavenrc
fi
if [ -f /etc/mavenrc ] ; then
. /etc/mavenrc
fi
@ -50,7 +53,7 @@ fi
cygwin=false;
darwin=false;
mingw=false
case "`uname`" in
case "$(uname)" in
CYGWIN*) cygwin=true ;;
MINGW*) mingw=true;;
Darwin*) darwin=true
@ -58,9 +61,9 @@ case "`uname`" in
# See https://developer.apple.com/library/mac/qa/qa1170/_index.html
if [ -z "$JAVA_HOME" ]; then
if [ -x "/usr/libexec/java_home" ]; then
export JAVA_HOME="`/usr/libexec/java_home`"
JAVA_HOME="$(/usr/libexec/java_home)"; export JAVA_HOME
else
export JAVA_HOME="/Library/Java/Home"
JAVA_HOME="/Library/Java/Home"; export JAVA_HOME
fi
fi
;;
@ -68,68 +71,38 @@ esac
if [ -z "$JAVA_HOME" ] ; then
if [ -r /etc/gentoo-release ] ; then
JAVA_HOME=`java-config --jre-home`
JAVA_HOME=$(java-config --jre-home)
fi
fi
if [ -z "$M2_HOME" ] ; then
## resolve links - $0 may be a link to maven's home
PRG="$0"
# need this for relative symlinks
while [ -h "$PRG" ] ; do
ls=`ls -ld "$PRG"`
link=`expr "$ls" : '.*-> \(.*\)$'`
if expr "$link" : '/.*' > /dev/null; then
PRG="$link"
else
PRG="`dirname "$PRG"`/$link"
fi
done
saveddir=`pwd`
M2_HOME=`dirname "$PRG"`/..
# make it fully qualified
M2_HOME=`cd "$M2_HOME" && pwd`
cd "$saveddir"
# echo Using m2 at $M2_HOME
fi
# For Cygwin, ensure paths are in UNIX format before anything is touched
if $cygwin ; then
[ -n "$M2_HOME" ] &&
M2_HOME=`cygpath --unix "$M2_HOME"`
[ -n "$JAVA_HOME" ] &&
JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
JAVA_HOME=$(cygpath --unix "$JAVA_HOME")
[ -n "$CLASSPATH" ] &&
CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
CLASSPATH=$(cygpath --path --unix "$CLASSPATH")
fi
# For Mingw, ensure paths are in UNIX format before anything is touched
if $mingw ; then
[ -n "$M2_HOME" ] &&
M2_HOME="`(cd "$M2_HOME"; pwd)`"
[ -n "$JAVA_HOME" ] &&
JAVA_HOME="`(cd "$JAVA_HOME"; pwd)`"
[ -n "$JAVA_HOME" ] && [ -d "$JAVA_HOME" ] &&
JAVA_HOME="$(cd "$JAVA_HOME" || (echo "cannot cd into $JAVA_HOME."; exit 1); pwd)"
fi
if [ -z "$JAVA_HOME" ]; then
javaExecutable="`which javac`"
if [ -n "$javaExecutable" ] && ! [ "`expr \"$javaExecutable\" : '\([^ ]*\)'`" = "no" ]; then
javaExecutable="$(which javac)"
if [ -n "$javaExecutable" ] && ! [ "$(expr "\"$javaExecutable\"" : '\([^ ]*\)')" = "no" ]; then
# readlink(1) is not available as standard on Solaris 10.
readLink=`which readlink`
if [ ! `expr "$readLink" : '\([^ ]*\)'` = "no" ]; then
readLink=$(which readlink)
if [ ! "$(expr "$readLink" : '\([^ ]*\)')" = "no" ]; then
if $darwin ; then
javaHome="`dirname \"$javaExecutable\"`"
javaExecutable="`cd \"$javaHome\" && pwd -P`/javac"
javaHome="$(dirname "\"$javaExecutable\"")"
javaExecutable="$(cd "\"$javaHome\"" && pwd -P)/javac"
else
javaExecutable="`readlink -f \"$javaExecutable\"`"
javaExecutable="$(readlink -f "\"$javaExecutable\"")"
fi
javaHome="`dirname \"$javaExecutable\"`"
javaHome=`expr "$javaHome" : '\(.*\)/bin'`
javaHome="$(dirname "\"$javaExecutable\"")"
javaHome=$(expr "$javaHome" : '\(.*\)/bin')
JAVA_HOME="$javaHome"
export JAVA_HOME
fi
@ -145,7 +118,7 @@ if [ -z "$JAVACMD" ] ; then
JAVACMD="$JAVA_HOME/bin/java"
fi
else
JAVACMD="`which java`"
JAVACMD="$(\unset -f command 2>/dev/null; \command -v java)"
fi
fi
@ -159,12 +132,9 @@ if [ -z "$JAVA_HOME" ] ; then
echo "Warning: JAVA_HOME environment variable is not set."
fi
CLASSWORLDS_LAUNCHER=org.codehaus.plexus.classworlds.launcher.Launcher
# traverses directory structure from process work directory to filesystem root
# first directory with .mvn subdirectory is considered project base directory
find_maven_basedir() {
if [ -z "$1" ]
then
echo "Path not specified to find_maven_basedir"
@ -180,96 +150,99 @@ find_maven_basedir() {
fi
# workaround for JBEAP-8937 (on Solaris 10/Sparc)
if [ -d "${wdir}" ]; then
wdir=`cd "$wdir/.."; pwd`
wdir=$(cd "$wdir/.." || exit 1; pwd)
fi
# end of workaround
done
echo "${basedir}"
printf '%s' "$(cd "$basedir" || exit 1; pwd)"
}
# concatenates all lines of a file
concat_lines() {
if [ -f "$1" ]; then
echo "$(tr -s '\n' ' ' < "$1")"
# Remove \r in case we run on Windows within Git Bash
# and check out the repository with auto CRLF management
# enabled. Otherwise, we may read lines that are delimited with
# \r\n and produce $'-Xarg\r' rather than -Xarg due to word
# splitting rules.
tr -s '\r\n' ' ' < "$1"
fi
}
BASE_DIR=`find_maven_basedir "$(pwd)"`
log() {
if [ "$MVNW_VERBOSE" = true ]; then
printf '%s\n' "$1"
fi
}
BASE_DIR=$(find_maven_basedir "$(dirname "$0")")
if [ -z "$BASE_DIR" ]; then
exit 1;
fi
MAVEN_PROJECTBASEDIR=${MAVEN_BASEDIR:-"$BASE_DIR"}; export MAVEN_PROJECTBASEDIR
log "$MAVEN_PROJECTBASEDIR"
##########################################################################################
# Extension to allow automatically downloading the maven-wrapper.jar from Maven-central
# This allows using the maven wrapper in projects that prohibit checking in binary data.
##########################################################################################
if [ -r "$BASE_DIR/.mvn/wrapper/maven-wrapper.jar" ]; then
if [ "$MVNW_VERBOSE" = true ]; then
echo "Found .mvn/wrapper/maven-wrapper.jar"
fi
wrapperJarPath="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar"
if [ -r "$wrapperJarPath" ]; then
log "Found $wrapperJarPath"
else
if [ "$MVNW_VERBOSE" = true ]; then
echo "Couldn't find .mvn/wrapper/maven-wrapper.jar, downloading it ..."
fi
log "Couldn't find $wrapperJarPath, downloading it ..."
if [ -n "$MVNW_REPOURL" ]; then
jarUrl="$MVNW_REPOURL/io/takari/maven-wrapper/0.5.6/maven-wrapper-0.5.6.jar"
wrapperUrl="$MVNW_REPOURL/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar"
else
jarUrl="https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/0.5.6/maven-wrapper-0.5.6.jar"
wrapperUrl="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar"
fi
while IFS="=" read key value; do
case "$key" in (wrapperUrl) jarUrl="$value"; break ;;
while IFS="=" read -r key value; do
# Remove '\r' from value to allow usage on windows as IFS does not consider '\r' as a separator ( considers space, tab, new line ('\n'), and custom '=' )
safeValue=$(echo "$value" | tr -d '\r')
case "$key" in (wrapperUrl) wrapperUrl="$safeValue"; break ;;
esac
done < "$BASE_DIR/.mvn/wrapper/maven-wrapper.properties"
if [ "$MVNW_VERBOSE" = true ]; then
echo "Downloading from: $jarUrl"
fi
wrapperJarPath="$BASE_DIR/.mvn/wrapper/maven-wrapper.jar"
done < "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.properties"
log "Downloading from: $wrapperUrl"
if $cygwin; then
wrapperJarPath=`cygpath --path --windows "$wrapperJarPath"`
wrapperJarPath=$(cygpath --path --windows "$wrapperJarPath")
fi
if command -v wget > /dev/null; then
if [ "$MVNW_VERBOSE" = true ]; then
echo "Found wget ... using wget"
fi
log "Found wget ... using wget"
[ "$MVNW_VERBOSE" = true ] && QUIET="" || QUIET="--quiet"
if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then
wget "$jarUrl" -O "$wrapperJarPath"
wget $QUIET "$wrapperUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath"
else
wget --http-user=$MVNW_USERNAME --http-password=$MVNW_PASSWORD "$jarUrl" -O "$wrapperJarPath"
wget $QUIET --http-user="$MVNW_USERNAME" --http-password="$MVNW_PASSWORD" "$wrapperUrl" -O "$wrapperJarPath" || rm -f "$wrapperJarPath"
fi
elif command -v curl > /dev/null; then
if [ "$MVNW_VERBOSE" = true ]; then
echo "Found curl ... using curl"
fi
log "Found curl ... using curl"
[ "$MVNW_VERBOSE" = true ] && QUIET="" || QUIET="--silent"
if [ -z "$MVNW_USERNAME" ] || [ -z "$MVNW_PASSWORD" ]; then
curl -o "$wrapperJarPath" "$jarUrl" -f
curl $QUIET -o "$wrapperJarPath" "$wrapperUrl" -f -L || rm -f "$wrapperJarPath"
else
curl --user $MVNW_USERNAME:$MVNW_PASSWORD -o "$wrapperJarPath" "$jarUrl" -f
curl $QUIET --user "$MVNW_USERNAME:$MVNW_PASSWORD" -o "$wrapperJarPath" "$wrapperUrl" -f -L || rm -f "$wrapperJarPath"
fi
else
if [ "$MVNW_VERBOSE" = true ]; then
echo "Falling back to using Java to download"
fi
javaClass="$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.java"
log "Falling back to using Java to download"
javaSource="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/MavenWrapperDownloader.java"
javaClass="$MAVEN_PROJECTBASEDIR/.mvn/wrapper/MavenWrapperDownloader.class"
# For Cygwin, switch paths to Windows format before running javac
if $cygwin; then
javaClass=`cygpath --path --windows "$javaClass"`
javaSource=$(cygpath --path --windows "$javaSource")
javaClass=$(cygpath --path --windows "$javaClass")
fi
if [ -e "$javaClass" ]; then
if [ ! -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then
if [ "$MVNW_VERBOSE" = true ]; then
echo " - Compiling MavenWrapperDownloader.java ..."
fi
# Compiling the Java class
("$JAVA_HOME/bin/javac" "$javaClass")
if [ -e "$javaSource" ]; then
if [ ! -e "$javaClass" ]; then
log " - Compiling MavenWrapperDownloader.java ..."
("$JAVA_HOME/bin/javac" "$javaSource")
fi
if [ -e "$BASE_DIR/.mvn/wrapper/MavenWrapperDownloader.class" ]; then
# Running the downloader
if [ "$MVNW_VERBOSE" = true ]; then
echo " - Running MavenWrapperDownloader.java ..."
fi
("$JAVA_HOME/bin/java" -cp .mvn/wrapper MavenWrapperDownloader "$MAVEN_PROJECTBASEDIR")
if [ -e "$javaClass" ]; then
log " - Running MavenWrapperDownloader.java ..."
("$JAVA_HOME/bin/java" -cp .mvn/wrapper MavenWrapperDownloader "$wrapperUrl" "$wrapperJarPath") || rm -f "$wrapperJarPath"
fi
fi
fi
@ -278,33 +251,58 @@ fi
# End of extension
##########################################################################################
export MAVEN_PROJECTBASEDIR=${MAVEN_BASEDIR:-"$BASE_DIR"}
if [ "$MVNW_VERBOSE" = true ]; then
echo $MAVEN_PROJECTBASEDIR
# If specified, validate the SHA-256 sum of the Maven wrapper jar file
wrapperSha256Sum=""
while IFS="=" read -r key value; do
case "$key" in (wrapperSha256Sum) wrapperSha256Sum=$value; break ;;
esac
done < "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.properties"
if [ -n "$wrapperSha256Sum" ]; then
wrapperSha256Result=false
if command -v sha256sum > /dev/null; then
if echo "$wrapperSha256Sum $wrapperJarPath" | sha256sum -c > /dev/null 2>&1; then
wrapperSha256Result=true
fi
elif command -v shasum > /dev/null; then
if echo "$wrapperSha256Sum $wrapperJarPath" | shasum -a 256 -c > /dev/null 2>&1; then
wrapperSha256Result=true
fi
else
echo "Checksum validation was requested but neither 'sha256sum' or 'shasum' are available."
echo "Please install either command, or disable validation by removing 'wrapperSha256Sum' from your maven-wrapper.properties."
exit 1
fi
if [ $wrapperSha256Result = false ]; then
echo "Error: Failed to validate Maven wrapper SHA-256, your Maven wrapper might be compromised." >&2
echo "Investigate or delete $wrapperJarPath to attempt a clean download." >&2
echo "If you updated your Maven version, you need to update the specified wrapperSha256Sum property." >&2
exit 1
fi
fi
MAVEN_OPTS="$(concat_lines "$MAVEN_PROJECTBASEDIR/.mvn/jvm.config") $MAVEN_OPTS"
# For Cygwin, switch paths to Windows format before running java
if $cygwin; then
[ -n "$M2_HOME" ] &&
M2_HOME=`cygpath --path --windows "$M2_HOME"`
[ -n "$JAVA_HOME" ] &&
JAVA_HOME=`cygpath --path --windows "$JAVA_HOME"`
JAVA_HOME=$(cygpath --path --windows "$JAVA_HOME")
[ -n "$CLASSPATH" ] &&
CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
CLASSPATH=$(cygpath --path --windows "$CLASSPATH")
[ -n "$MAVEN_PROJECTBASEDIR" ] &&
MAVEN_PROJECTBASEDIR=`cygpath --path --windows "$MAVEN_PROJECTBASEDIR"`
MAVEN_PROJECTBASEDIR=$(cygpath --path --windows "$MAVEN_PROJECTBASEDIR")
fi
# Provide a "standardized" way to retrieve the CLI args that will
# work with both Windows and non-Windows executions.
MAVEN_CMD_LINE_ARGS="$MAVEN_CONFIG $@"
MAVEN_CMD_LINE_ARGS="$MAVEN_CONFIG $*"
export MAVEN_CMD_LINE_ARGS
WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain
# shellcheck disable=SC2086 # safe args
exec "$JAVACMD" \
$MAVEN_OPTS \
$MAVEN_DEBUG_OPTS \
-classpath "$MAVEN_PROJECTBASEDIR/.mvn/wrapper/maven-wrapper.jar" \
"-Dmaven.home=${M2_HOME}" "-Dmaven.multiModuleProjectDirectory=${MAVEN_PROJECTBASEDIR}" \
"-Dmaven.multiModuleProjectDirectory=${MAVEN_PROJECTBASEDIR}" \
${WRAPPER_LAUNCHER} $MAVEN_CONFIG "$@"

View File

@ -1,182 +1,205 @@
@REM ----------------------------------------------------------------------------
@REM Licensed to the Apache Software Foundation (ASF) under one
@REM or more contributor license agreements. See the NOTICE file
@REM distributed with this work for additional information
@REM regarding copyright ownership. The ASF licenses this file
@REM to you under the Apache License, Version 2.0 (the
@REM "License"); you may not use this file except in compliance
@REM with the License. You may obtain a copy of the License at
@REM
@REM http://www.apache.org/licenses/LICENSE-2.0
@REM
@REM Unless required by applicable law or agreed to in writing,
@REM software distributed under the License is distributed on an
@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@REM KIND, either express or implied. See the License for the
@REM specific language governing permissions and limitations
@REM under the License.
@REM ----------------------------------------------------------------------------
@REM ----------------------------------------------------------------------------
@REM Maven Start Up Batch script
@REM
@REM Required ENV vars:
@REM JAVA_HOME - location of a JDK home dir
@REM
@REM Optional ENV vars
@REM M2_HOME - location of maven2's installed home dir
@REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands
@REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending
@REM MAVEN_OPTS - parameters passed to the Java VM when running Maven
@REM e.g. to debug Maven itself, use
@REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000
@REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files
@REM ----------------------------------------------------------------------------
@REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on'
@echo off
@REM set title of command window
title %0
@REM enable echoing by setting MAVEN_BATCH_ECHO to 'on'
@if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO%
@REM set %HOME% to equivalent of $HOME
if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%")
@REM Execute a user defined script before this one
if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre
@REM check for pre script, once with legacy .bat ending and once with .cmd ending
if exist "%HOME%\mavenrc_pre.bat" call "%HOME%\mavenrc_pre.bat"
if exist "%HOME%\mavenrc_pre.cmd" call "%HOME%\mavenrc_pre.cmd"
:skipRcPre
@setlocal
set ERROR_CODE=0
@REM To isolate internal variables from possible post scripts, we use another setlocal
@setlocal
@REM ==== START VALIDATION ====
if not "%JAVA_HOME%" == "" goto OkJHome
echo.
echo Error: JAVA_HOME not found in your environment. >&2
echo Please set the JAVA_HOME variable in your environment to match the >&2
echo location of your Java installation. >&2
echo.
goto error
:OkJHome
if exist "%JAVA_HOME%\bin\java.exe" goto init
echo.
echo Error: JAVA_HOME is set to an invalid directory. >&2
echo JAVA_HOME = "%JAVA_HOME%" >&2
echo Please set the JAVA_HOME variable in your environment to match the >&2
echo location of your Java installation. >&2
echo.
goto error
@REM ==== END VALIDATION ====
:init
@REM Find the project base dir, i.e. the directory that contains the folder ".mvn".
@REM Fallback to current working directory if not found.
set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR%
IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir
set EXEC_DIR=%CD%
set WDIR=%EXEC_DIR%
:findBaseDir
IF EXIST "%WDIR%"\.mvn goto baseDirFound
cd ..
IF "%WDIR%"=="%CD%" goto baseDirNotFound
set WDIR=%CD%
goto findBaseDir
:baseDirFound
set MAVEN_PROJECTBASEDIR=%WDIR%
cd "%EXEC_DIR%"
goto endDetectBaseDir
:baseDirNotFound
set MAVEN_PROJECTBASEDIR=%EXEC_DIR%
cd "%EXEC_DIR%"
:endDetectBaseDir
IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig
@setlocal EnableExtensions EnableDelayedExpansion
for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a
@endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS%
:endReadAdditionalConfig
SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe"
set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar"
set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain
set DOWNLOAD_URL="https://repo.maven.apache.org/maven2/io/takari/maven-wrapper/0.5.6/maven-wrapper-0.5.6.jar"
FOR /F "tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO (
IF "%%A"=="wrapperUrl" SET DOWNLOAD_URL=%%B
)
@REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central
@REM This allows using the maven wrapper in projects that prohibit checking in binary data.
if exist %WRAPPER_JAR% (
if "%MVNW_VERBOSE%" == "true" (
echo Found %WRAPPER_JAR%
)
) else (
if not "%MVNW_REPOURL%" == "" (
SET DOWNLOAD_URL="%MVNW_REPOURL%/io/takari/maven-wrapper/0.5.6/maven-wrapper-0.5.6.jar"
)
if "%MVNW_VERBOSE%" == "true" (
echo Couldn't find %WRAPPER_JAR%, downloading it ...
echo Downloading from: %DOWNLOAD_URL%
)
powershell -Command "&{"^
"$webclient = new-object System.Net.WebClient;"^
"if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^
"$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^
"}"^
"[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%DOWNLOAD_URL%', '%WRAPPER_JAR%')"^
"}"
if "%MVNW_VERBOSE%" == "true" (
echo Finished downloading %WRAPPER_JAR%
)
)
@REM End of extension
@REM Provide a "standardized" way to retrieve the CLI args that will
@REM work with both Windows and non-Windows executions.
set MAVEN_CMD_LINE_ARGS=%*
%MAVEN_JAVA_EXE% %JVM_CONFIG_MAVEN_PROPS% %MAVEN_OPTS% %MAVEN_DEBUG_OPTS% -classpath %WRAPPER_JAR% "-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" %WRAPPER_LAUNCHER% %MAVEN_CONFIG% %*
if ERRORLEVEL 1 goto error
goto end
:error
set ERROR_CODE=1
:end
@endlocal & set ERROR_CODE=%ERROR_CODE%
if not "%MAVEN_SKIP_RC%" == "" goto skipRcPost
@REM check for post script, once with legacy .bat ending and once with .cmd ending
if exist "%HOME%\mavenrc_post.bat" call "%HOME%\mavenrc_post.bat"
if exist "%HOME%\mavenrc_post.cmd" call "%HOME%\mavenrc_post.cmd"
:skipRcPost
@REM pause the script if MAVEN_BATCH_PAUSE is set to 'on'
if "%MAVEN_BATCH_PAUSE%" == "on" pause
if "%MAVEN_TERMINATE_CMD%" == "on" exit %ERROR_CODE%
exit /B %ERROR_CODE%
@REM ----------------------------------------------------------------------------
@REM Licensed to the Apache Software Foundation (ASF) under one
@REM or more contributor license agreements. See the NOTICE file
@REM distributed with this work for additional information
@REM regarding copyright ownership. The ASF licenses this file
@REM to you under the Apache License, Version 2.0 (the
@REM "License"); you may not use this file except in compliance
@REM with the License. You may obtain a copy of the License at
@REM
@REM http://www.apache.org/licenses/LICENSE-2.0
@REM
@REM Unless required by applicable law or agreed to in writing,
@REM software distributed under the License is distributed on an
@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@REM KIND, either express or implied. See the License for the
@REM specific language governing permissions and limitations
@REM under the License.
@REM ----------------------------------------------------------------------------
@REM ----------------------------------------------------------------------------
@REM Apache Maven Wrapper startup batch script, version 3.2.0
@REM
@REM Required ENV vars:
@REM JAVA_HOME - location of a JDK home dir
@REM
@REM Optional ENV vars
@REM MAVEN_BATCH_ECHO - set to 'on' to enable the echoing of the batch commands
@REM MAVEN_BATCH_PAUSE - set to 'on' to wait for a keystroke before ending
@REM MAVEN_OPTS - parameters passed to the Java VM when running Maven
@REM e.g. to debug Maven itself, use
@REM set MAVEN_OPTS=-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=8000
@REM MAVEN_SKIP_RC - flag to disable loading of mavenrc files
@REM ----------------------------------------------------------------------------
@REM Begin all REM lines with '@' in case MAVEN_BATCH_ECHO is 'on'
@echo off
@REM set title of command window
title %0
@REM enable echoing by setting MAVEN_BATCH_ECHO to 'on'
@if "%MAVEN_BATCH_ECHO%" == "on" echo %MAVEN_BATCH_ECHO%
@REM set %HOME% to equivalent of $HOME
if "%HOME%" == "" (set "HOME=%HOMEDRIVE%%HOMEPATH%")
@REM Execute a user defined script before this one
if not "%MAVEN_SKIP_RC%" == "" goto skipRcPre
@REM check for pre script, once with legacy .bat ending and once with .cmd ending
if exist "%USERPROFILE%\mavenrc_pre.bat" call "%USERPROFILE%\mavenrc_pre.bat" %*
if exist "%USERPROFILE%\mavenrc_pre.cmd" call "%USERPROFILE%\mavenrc_pre.cmd" %*
:skipRcPre
@setlocal
set ERROR_CODE=0
@REM To isolate internal variables from possible post scripts, we use another setlocal
@setlocal
@REM ==== START VALIDATION ====
if not "%JAVA_HOME%" == "" goto OkJHome
echo.
echo Error: JAVA_HOME not found in your environment. >&2
echo Please set the JAVA_HOME variable in your environment to match the >&2
echo location of your Java installation. >&2
echo.
goto error
:OkJHome
if exist "%JAVA_HOME%\bin\java.exe" goto init
echo.
echo Error: JAVA_HOME is set to an invalid directory. >&2
echo JAVA_HOME = "%JAVA_HOME%" >&2
echo Please set the JAVA_HOME variable in your environment to match the >&2
echo location of your Java installation. >&2
echo.
goto error
@REM ==== END VALIDATION ====
:init
@REM Find the project base dir, i.e. the directory that contains the folder ".mvn".
@REM Fallback to current working directory if not found.
set MAVEN_PROJECTBASEDIR=%MAVEN_BASEDIR%
IF NOT "%MAVEN_PROJECTBASEDIR%"=="" goto endDetectBaseDir
set EXEC_DIR=%CD%
set WDIR=%EXEC_DIR%
:findBaseDir
IF EXIST "%WDIR%"\.mvn goto baseDirFound
cd ..
IF "%WDIR%"=="%CD%" goto baseDirNotFound
set WDIR=%CD%
goto findBaseDir
:baseDirFound
set MAVEN_PROJECTBASEDIR=%WDIR%
cd "%EXEC_DIR%"
goto endDetectBaseDir
:baseDirNotFound
set MAVEN_PROJECTBASEDIR=%EXEC_DIR%
cd "%EXEC_DIR%"
:endDetectBaseDir
IF NOT EXIST "%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config" goto endReadAdditionalConfig
@setlocal EnableExtensions EnableDelayedExpansion
for /F "usebackq delims=" %%a in ("%MAVEN_PROJECTBASEDIR%\.mvn\jvm.config") do set JVM_CONFIG_MAVEN_PROPS=!JVM_CONFIG_MAVEN_PROPS! %%a
@endlocal & set JVM_CONFIG_MAVEN_PROPS=%JVM_CONFIG_MAVEN_PROPS%
:endReadAdditionalConfig
SET MAVEN_JAVA_EXE="%JAVA_HOME%\bin\java.exe"
set WRAPPER_JAR="%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.jar"
set WRAPPER_LAUNCHER=org.apache.maven.wrapper.MavenWrapperMain
set WRAPPER_URL="https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar"
FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO (
IF "%%A"=="wrapperUrl" SET WRAPPER_URL=%%B
)
@REM Extension to allow automatically downloading the maven-wrapper.jar from Maven-central
@REM This allows using the maven wrapper in projects that prohibit checking in binary data.
if exist %WRAPPER_JAR% (
if "%MVNW_VERBOSE%" == "true" (
echo Found %WRAPPER_JAR%
)
) else (
if not "%MVNW_REPOURL%" == "" (
SET WRAPPER_URL="%MVNW_REPOURL%/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar"
)
if "%MVNW_VERBOSE%" == "true" (
echo Couldn't find %WRAPPER_JAR%, downloading it ...
echo Downloading from: %WRAPPER_URL%
)
powershell -Command "&{"^
"$webclient = new-object System.Net.WebClient;"^
"if (-not ([string]::IsNullOrEmpty('%MVNW_USERNAME%') -and [string]::IsNullOrEmpty('%MVNW_PASSWORD%'))) {"^
"$webclient.Credentials = new-object System.Net.NetworkCredential('%MVNW_USERNAME%', '%MVNW_PASSWORD%');"^
"}"^
"[Net.ServicePointManager]::SecurityProtocol = [Net.SecurityProtocolType]::Tls12; $webclient.DownloadFile('%WRAPPER_URL%', '%WRAPPER_JAR%')"^
"}"
if "%MVNW_VERBOSE%" == "true" (
echo Finished downloading %WRAPPER_JAR%
)
)
@REM End of extension
@REM If specified, validate the SHA-256 sum of the Maven wrapper jar file
SET WRAPPER_SHA_256_SUM=""
FOR /F "usebackq tokens=1,2 delims==" %%A IN ("%MAVEN_PROJECTBASEDIR%\.mvn\wrapper\maven-wrapper.properties") DO (
IF "%%A"=="wrapperSha256Sum" SET WRAPPER_SHA_256_SUM=%%B
)
IF NOT %WRAPPER_SHA_256_SUM%=="" (
powershell -Command "&{"^
"$hash = (Get-FileHash \"%WRAPPER_JAR%\" -Algorithm SHA256).Hash.ToLower();"^
"If('%WRAPPER_SHA_256_SUM%' -ne $hash){"^
" Write-Output 'Error: Failed to validate Maven wrapper SHA-256, your Maven wrapper might be compromised.';"^
" Write-Output 'Investigate or delete %WRAPPER_JAR% to attempt a clean download.';"^
" Write-Output 'If you updated your Maven version, you need to update the specified wrapperSha256Sum property.';"^
" exit 1;"^
"}"^
"}"
if ERRORLEVEL 1 goto error
)
@REM Provide a "standardized" way to retrieve the CLI args that will
@REM work with both Windows and non-Windows executions.
set MAVEN_CMD_LINE_ARGS=%*
%MAVEN_JAVA_EXE% ^
%JVM_CONFIG_MAVEN_PROPS% ^
%MAVEN_OPTS% ^
%MAVEN_DEBUG_OPTS% ^
-classpath %WRAPPER_JAR% ^
"-Dmaven.multiModuleProjectDirectory=%MAVEN_PROJECTBASEDIR%" ^
%WRAPPER_LAUNCHER% %MAVEN_CONFIG% %*
if ERRORLEVEL 1 goto error
goto end
:error
set ERROR_CODE=1
:end
@endlocal & set ERROR_CODE=%ERROR_CODE%
if not "%MAVEN_SKIP_RC%"=="" goto skipRcPost
@REM check for post script, once with legacy .bat ending and once with .cmd ending
if exist "%USERPROFILE%\mavenrc_post.bat" call "%USERPROFILE%\mavenrc_post.bat"
if exist "%USERPROFILE%\mavenrc_post.cmd" call "%USERPROFILE%\mavenrc_post.cmd"
:skipRcPost
@REM pause the script if MAVEN_BATCH_PAUSE is set to 'on'
if "%MAVEN_BATCH_PAUSE%"=="on" pause
if "%MAVEN_TERMINATE_CMD%"=="on" exit %ERROR_CODE%
cmd /C exit /B %ERROR_CODE%

View File

@ -32,8 +32,8 @@
<logback.version>1.2.12</logback.version>
<maven.compiler.plugin.version>3.11.0</maven.compiler.plugin.version>
<maven-surefire-plugin.version>3.1.2</maven-surefire-plugin.version>
<spotless.version>2.37.0</spotless.version>
<google-java-format.version>1.17.0</google-java-format.version>
<spotless.version>2.40.0</spotless.version>
<google-java-format.version>1.18.1</google-java-format.version>
<okhttp.version>4.11.0</okhttp.version>
<gson.version>2.10.1</gson.version>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

View File

@ -784,7 +784,9 @@ public class HttpTest {
"/stream/consumers/foo-virtual-host",
"/stream/publishers/foo-virtual-host",
"/stream/publishers/foo-virtual-host",
"/stream/publishers/%2F/foo-stream"
"/stream/publishers/%2F/foo-stream",
"/stream/%2F/foo-stream/tracking",
"/stream/foo-virtual-host/foo-stream/tracking",
})
void shouldReturnNotFound(String endpoint) {
assertThatThrownBy(() -> get(endpoint)).hasMessageContaining("404");
@ -867,6 +869,112 @@ public class HttpTest {
}
}
@Test
void trackingInfo() throws Exception {
String endpoint = "/stream/%2F/" + stream + "/tracking";
Callable<Map<String, Object>> getTracking = () -> toMap(get(endpoint));
Map<String, Object> tracking = getTracking.call();
assertThat(tracking).hasSize(2).containsKeys("offsets", "writers");
assertThat(trackingOffsets(tracking)).isEmpty();
assertThat(trackingWriters(tracking)).isEmpty();
String consumerReference1 = "foo";
String consumerReference2 = "bar";
Client client = cf.get();
client.storeOffset(consumerReference1, stream, 42);
waitUntil(() -> client.queryOffset(consumerReference1, stream).getOffset() == 42);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking)).hasSize(1).containsEntry(consumerReference1, d(42));
assertThat(trackingWriters(tracking)).isEmpty();
client.storeOffset(consumerReference1, stream, 55);
waitUntil(() -> client.queryOffset(consumerReference1, stream).getOffset() == 55);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking)).hasSize(1).containsEntry(consumerReference1, d(55));
assertThat(trackingWriters(tracking)).isEmpty();
client.storeOffset(consumerReference2, stream, 12);
waitUntil(() -> client.queryOffset(consumerReference2, stream).getOffset() == 12);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking))
.hasSize(2)
.containsEntry(consumerReference1, d(55))
.containsEntry(consumerReference2, d(12));
assertThat(trackingWriters(tracking)).isEmpty();
tracking = toMap(get(endpoint + "?type=offset"));
assertThat(tracking).hasSize(1).containsKey("offsets");
String publisherReference1 = "foobar1";
String publisherReference2 = "foobar2";
byte pub1 = 0;
byte pub2 = 1;
assertThat(client.declarePublisher(pub1, publisherReference1, stream).isOk()).isTrue();
assertThat(client.declarePublisher(pub2, publisherReference2, stream).isOk()).isTrue();
client.publish(pub1, message(client), o -> 25);
waitUntil(() -> client.queryPublisherSequence(publisherReference1, stream) == 25);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking))
.hasSize(2)
.containsEntry(consumerReference1, d(55))
.containsEntry(consumerReference2, d(12));
assertThat(trackingWriters(tracking)).hasSize(1).containsEntry(publisherReference1, d(25));
client.publish(pub1, message(client), o -> 36);
waitUntil(() -> client.queryPublisherSequence(publisherReference1, stream) == 36);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking))
.hasSize(2)
.containsEntry(consumerReference1, d(55))
.containsEntry(consumerReference2, d(12));
assertThat(trackingWriters(tracking)).hasSize(1).containsEntry(publisherReference1, d(36));
client.publish(pub2, message(client), o -> 45);
waitUntil(() -> client.queryPublisherSequence(publisherReference2, stream) == 45);
tracking = getTracking.call();
assertThat(trackingOffsets(tracking))
.hasSize(2)
.containsEntry(consumerReference1, d(55))
.containsEntry(consumerReference2, d(12));
assertThat(trackingWriters(tracking))
.hasSize(2)
.containsEntry(publisherReference1, d(36))
.containsEntry(publisherReference2, d(45));
tracking = toMap(get(endpoint + "?type=writer"));
assertThat(tracking).hasSize(1).containsKey("writers");
tracking = toMap(get(endpoint + "?type=all"));
assertThat(tracking).hasSize(2).containsKeys("offsets", "writers");
tracking = toMap(get(endpoint + "?type=unknown-means-all"));
assertThat(tracking).hasSize(2).containsKeys("offsets", "writers");
}
@SuppressWarnings("unchecked")
private static Map<String, Number> trackingOffsets(Map<String, Object> tracking) {
return (Map<String, Number>) tracking.get("offsets");
}
@SuppressWarnings("unchecked")
private static Map<String, Number> trackingWriters(Map<String, Object> tracking) {
return (Map<String, Number>) tracking.get("writers");
}
private static Double d(int value) {
return (double) value;
}
private static List<Message> message(Client client) {
return Collections.singletonList(
client.messageBuilder().addData("hello".getBytes(StandardCharsets.UTF_8)).build());
}
static class PermissionsTestConfiguration {
final String user;
final String endpoint;

View File

@ -75,12 +75,12 @@ init(Req, Opts) ->
undefined ->
no_supported_sub_protocol(undefined, Req);
Protocol ->
WsOpts0 = proplists:get_value(ws_opts, Opts, #{}),
WsOpts = maps:merge(#{compress => true}, WsOpts0),
case lists:member(<<"mqtt">>, Protocol) of
false ->
no_supported_sub_protocol(Protocol, Req);
true ->
WsOpts0 = proplists:get_value(ws_opts, Opts, #{}),
WsOpts = maps:merge(#{compress => true}, WsOpts0),
ShouldUseFHC = application:get_env(?APP, use_file_handle_cache, true),
case ShouldUseFHC of
true -> ?LOG_INFO("Web MQTT: file handle cache use is enabled");
@ -278,7 +278,12 @@ terminate(_Reason, _Request,
no_supported_sub_protocol(Protocol, Req) ->
%% The client MUST include mqtt in the list of WebSocket Sub Protocols it offers [MQTT-6.0.0-3].
?LOG_ERROR("Web MQTT: 'mqtt' not included in client offered subprotocols: ~tp", [Protocol]),
{ok, cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req), #state{}}.
%% Set should_use_fhc to false, because at this early stage of init no fhc
%% obtain was called, so terminate/3 should not call fhc release
%% (even if use_file_handle_cache is true)
{ok,
cowboy_req:reply(400, #{<<"connection">> => <<"close">>}, Req),
#state{should_use_fhc = false}}.
handle_data(Data, State0 = #state{}) ->
case handle_data1(Data, State0) of

View File

@ -95,7 +95,7 @@ Release notes are kept under [rabbitmq-server/release-notes](https://github.com/
and [streams](https://rabbitmq.com/streams.html#leader-election) already do. At the same time, this means
that RabbitMQ clusters now **must have a majority of nodes online at all times**, or all client operations will be refused.
In RabbitMQ 4.0 Khepri will be the only option, and **Mnesia will be completely removed**.
The target is to make Khepri the default database engine in RabbitMQ 4.0.
* Messages are now internally stored using a new common heavily AMQP 1.0-influenced container format. This is a major step towards a protocol-agnostic core:
a common format that encapsulates a sum of data types used by the protocols RabbitMQ supports, plus annotations for routng, dead-lettering state,