Revert changes to add proc_name via gen_server2, instead do it explicitly with a helper macro.

This commit is contained in:
Simon MacMullen 2014-01-10 15:17:36 +00:00
parent 7c3f256df2
commit f7834aa144
12 changed files with 33 additions and 44 deletions

View File

@ -111,3 +111,5 @@
-define(INVALID_HEADERS_KEY, <<"x-invalid-headers">>).
-define(ROUTING_HEADERS, [<<"CC">>, <<"BCC">>]).
-define(DELETED_HEADER, <<"BCC">>).
-define(store_proc_name(N), rabbit_misc:store_proc_name(?MODULE, N)).

View File

@ -81,10 +81,6 @@
%% process as sys:get_status/1 would). Pass through a function which
%% can be invoked on the state, get back the result. The state is not
%% modified.
%%
%% 10) The Options parameter to start / start_link can include
%% {proc_name, Name}. This name is stored in the process dictionary
%% for later debugging.
%% All modifications are (C) 2009-2013 GoPivotal, Inc.
@ -287,7 +283,7 @@ behaviour_info(_Other) ->
%%% Name ::= {local, atom()} | {global, atom()}
%%% Mod ::= atom(), callback module implementing the 'real' server
%%% Args ::= term(), init arguments (to Mod:init/1)
%%% Options ::= [{timeout, Timeout} | {debug, [Flag]} | {proc_name, term()}]
%%% Options ::= [{timeout, Timeout} | {debug, [Flag]}]
%%% Flag ::= trace | log | {logfile, File} | statistics | debug
%%% (debug == log && statistics)
%%% Returns: {ok, Pid} |
@ -467,10 +463,6 @@ init_it(Starter, Parent, Name0, Mod, Args, Options) ->
mod = Mod,
queue = Queue,
debug = Debug }),
case opt(proc_name, Options) of
{ok, ProcName} -> put(process_name, {Mod, ProcName});
false -> ok
end,
case catch Mod:init(Args) of
{ok, State} ->
proc_lib:init_ack(Starter, {ok, self()}),
@ -909,17 +901,9 @@ common_noreply(_Name, _NState, [] = _Debug) ->
common_noreply(Name, NState, Debug) ->
sys:handle_debug(Debug, fun print_event/3, Name, {noreply, NState}).
common_become(Name, OldMod, NewMod, NState, Debug) ->
case get(process_name) of
{OldMod, ProcName} -> put(process_name, {NewMod, ProcName});
_ -> ok
end,
common_become0(Name, NewMod, NState, Debug).
common_become0(_Name, _Mod, _NState, [] = _Debug) ->
common_become(_Name, _Mod, _NState, [] = _Debug) ->
[];
common_become0(Name, Mod, NState, Debug) ->
common_become(Name, Mod, NState, Debug) ->
sys:handle_debug(Debug, fun print_event/3, Name, {become, Mod, NState}).
handle_msg({'$gen_call', From, Msg}, GS2State = #gs2_state { mod = Mod,
@ -951,7 +935,6 @@ handle_msg(Msg, GS2State = #gs2_state { mod = Mod, state = State }) ->
handle_common_reply(Reply, Msg, GS2State).
handle_common_reply(Reply, Msg, GS2State = #gs2_state { name = Name,
mod = Mod0,
debug = Debug}) ->
case Reply of
{noreply, NState} ->
@ -965,14 +948,14 @@ handle_common_reply(Reply, Msg, GS2State = #gs2_state { name = Name,
time = Time1,
debug = Debug1});
{become, Mod, NState} ->
Debug1 = common_become(Name, Mod0, Mod, NState, Debug),
Debug1 = common_become(Name, Mod, NState, Debug),
loop(find_prioritisers(
GS2State #gs2_state { mod = Mod,
state = NState,
time = infinity,
debug = Debug1 }));
{become, Mod, NState, Time1} ->
Debug1 = common_become(Name, Mod0, Mod, NState, Debug),
Debug1 = common_become(Name, Mod, NState, Debug),
loop(find_prioritisers(
GS2State #gs2_state { mod = Mod,
state = NState,

View File

@ -517,8 +517,7 @@ table_definitions() ->
[{Name, [?TABLE_MATCH | Attributes]}].
start_link(GroupName, Module, Args, TxnFun) ->
gen_server2:start_link(
?MODULE, [GroupName, Module, Args, TxnFun], [{proc_name, GroupName}]).
gen_server2:start_link(?MODULE, [GroupName, Module, Args, TxnFun], []).
leave(Server) ->
gen_server2:cast(Server, leave).
@ -543,6 +542,7 @@ forget_group(GroupName) ->
ok.
init([GroupName, Module, Args, TxnFun]) ->
put(process_name, {?MODULE, GroupName}),
{MegaSecs, Secs, MicroSecs} = now(),
random:seed(MegaSecs, Secs, MicroSecs),
Self = make_member(GroupName),

View File

@ -99,8 +99,7 @@
%%----------------------------------------------------------------------------
start_link(Q) ->
gen_server2:start_link(?MODULE, Q, [{proc_name, Q#amqqueue.name}]).
start_link(Q) -> gen_server2:start_link(?MODULE, Q, []).
info_keys() -> ?INFO_KEYS.
@ -108,6 +107,7 @@ info_keys() -> ?INFO_KEYS.
init(Q) ->
process_flag(trap_exit, true),
?store_proc_name(Q#amqqueue.name),
{ok, init_state(Q#amqqueue{pid = self()}), hibernate,
{backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN, ?DESIRED_HIBERNATE}}.

View File

@ -119,8 +119,7 @@ start_link(Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User,
VHost, Capabilities, CollectorPid, Limiter) ->
gen_server2:start_link(
?MODULE, [Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol,
User, VHost, Capabilities, CollectorPid, Limiter],
[{proc_name, {ConnName, Channel}}]).
User, VHost, Capabilities, CollectorPid, Limiter], []).
do(Pid, Method) ->
do(Pid, Method, none).
@ -195,6 +194,7 @@ force_event_refresh() ->
init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
Capabilities, CollectorPid, LimiterPid]) ->
process_flag(trap_exit, true),
?store_proc_name({ConnName, Channel}),
ok = pg_local:join(rabbit_channels, self()),
State = #ch{state = starting,
protocol = Protocol,

View File

@ -117,6 +117,8 @@
-module(rabbit_limiter).
-include("rabbit.hrl").
-behaviour(gen_server2).
-export([start_link/1]).
@ -194,8 +196,7 @@
%% API
%%----------------------------------------------------------------------------
start_link(ProcName) ->
gen_server2:start_link(?MODULE, [], [{proc_name, ProcName}]).
start_link(ProcName) -> gen_server2:start_link(?MODULE, [ProcName], []).
new(Pid) ->
%% this a 'call' to ensure that it is invoked at most once.
@ -322,7 +323,8 @@ update_credit(CTag, Credit, Drain, Credits) ->
%% gen_server callbacks
%%----------------------------------------------------------------------------
init([]) -> {ok, #lim{}}.
init([ProcName]) -> ?store_proc_name(ProcName),
{ok, #lim{}}.
prioritise_call(get_prefetch_limit, _From, _Len, _State) -> 9;
prioritise_call(_Msg, _From, _Len, _State) -> 0.

View File

@ -310,8 +310,7 @@
%%----------------------------------------------------------------------------
start_link(Queue, GM, DeathFun, DepthFun) ->
gen_server2:start_link(?MODULE, [Queue, GM, DeathFun, DepthFun],
[{proc_name, Queue#amqqueue.name}]).
gen_server2:start_link(?MODULE, [Queue, GM, DeathFun, DepthFun], []).
get_gm(CPid) ->
gen_server2:call(CPid, get_gm, infinity).
@ -324,6 +323,7 @@ ensure_monitoring(CPid, Pids) ->
%% ---------------------------------------------------------------------------
init([#amqqueue { name = QueueName } = Q, GM, DeathFun, DepthFun]) ->
?store_proc_name(QueueName),
GM1 = case GM of
undefined ->
{ok, GM2} = gm:start_link(

View File

@ -71,8 +71,7 @@
%%----------------------------------------------------------------------------
start_link(Q) -> gen_server2:start_link(
?MODULE, Q, [{proc_name, Q#amqqueue.name}]).
start_link(Q) -> gen_server2:start_link(?MODULE, Q, []).
set_maximum_since_use(QPid, Age) ->
gen_server2:cast(QPid, {set_maximum_since_use, Age}).
@ -80,6 +79,7 @@ set_maximum_since_use(QPid, Age) ->
info(QPid) -> gen_server2:call(QPid, info, infinity).
init(Q) ->
?store_proc_name(Q#amqqueue.name),
{ok, {not_started, Q}, hibernate,
{backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN,
?DESIRED_HIBERNATE}}.
@ -617,6 +617,7 @@ promote_me(From, #state { q = Q = #amqqueue { name = QName },
KS1 = lists:foldl(fun (ChPid0, KS0) ->
pmon:demonitor(ChPid0, KS0)
end, KS, AwaitGmDown),
rabbit_misc:store_proc_name(rabbit_amqqueue_process, QName),
rabbit_amqqueue_process:init_with_backing_queue_state(
Q1, rabbit_mirror_queue_master, MasterState, RateTRef, Deliveries, KS1,
MTC).

View File

@ -84,7 +84,7 @@
master_prepare(Ref, QName, Log, SPids) ->
MPid = self(),
spawn_link(fun () ->
rabbit_misc:store_proc_name(?MODULE, QName),
?store_proc_name(QName),
syncer(Ref, Log, MPid, SPids)
end).

View File

@ -16,7 +16,7 @@
-module(rabbit_queue_collector).
-behaviour(gen_server2).
-behaviour(gen_server).
-export([start_link/1, register/2, delete_all/1]).
@ -41,17 +41,18 @@
%%----------------------------------------------------------------------------
start_link(ProcName) ->
gen_server2:start_link(?MODULE, [], [{proc_name, ProcName}]).
gen_server:start_link(?MODULE, [ProcName], []).
register(CollectorPid, Q) ->
gen_server2:call(CollectorPid, {register, Q}, infinity).
gen_server:call(CollectorPid, {register, Q}, infinity).
delete_all(CollectorPid) ->
gen_server2:call(CollectorPid, delete_all, infinity).
gen_server:call(CollectorPid, delete_all, infinity).
%%----------------------------------------------------------------------------
init([]) ->
init([ProcName]) ->
?store_proc_name(ProcName),
{ok, #state{monitors = pmon:new(), delete_from = undefined}}.
%%--------------------------------------------------------------------------
@ -79,7 +80,7 @@ handle_info({'DOWN', _MRef, process, DownPid, _Reason},
State = #state{monitors = QMons, delete_from = Deleting}) ->
QMons1 = pmon:erase(DownPid, QMons),
case Deleting =/= undefined andalso pmon:is_empty(QMons1) of
true -> gen_server2:reply(Deleting, ok);
true -> gen_server:reply(Deleting, ok);
false -> ok
end,
{noreply, State#state{monitors = QMons1}}.

View File

@ -214,7 +214,7 @@ start_connection(Parent, HelperSup, Deb, Sock, SockTransform) ->
erlang:send_after(?HANDSHAKE_TIMEOUT * 1000, self(), handshake_timeout),
{PeerHost, PeerPort, Host, Port} =
socket_op(Sock, fun (S) -> rabbit_net:socket_ends(S, inbound) end),
rabbit_misc:store_proc_name(?MODULE, list_to_binary(Name)),
?store_proc_name(list_to_binary(Name)),
State = #v1{parent = Parent,
sock = ClientSock,
connection = #connection{

View File

@ -144,7 +144,7 @@ system_code_change(Misc, _Module, _OldVsn, _Extra) ->
enter_mainloop(Identity, State) ->
Deb = sys:debug_options([]),
rabbit_misc:store_proc_name(?MODULE, Identity),
?store_proc_name(Identity),
mainloop(Deb, State).
mainloop(Deb, State) ->