Hrm, fewer deadlocks, but sometimes unreliable shutdown

This commit is contained in:
Scott Lystig Fritchie 2015-07-16 17:59:02 +09:00
parent f2fc5b91c2
commit d331e09923
9 changed files with 88 additions and 16 deletions

View file

@ -16,6 +16,11 @@ case PulseBuild of
{pulse_side_effect, {pulse_side_effect,
[ {does_not_exist_yet, some_func, '_'} [ {does_not_exist_yet, some_func, '_'}
, {machi_flu1_client, '_', '_'}
, {machi_projection_store, '_', '_'}
, {machi_proxy_flu1_client, '_', '_'}
, {machi_pb_translate, '_', '_'}
, {prim_file, '_', '_'} , {prim_file, '_', '_'}
, {file, '_', '_'} , {file, '_', '_'}
, {filelib, '_', '_'} , {filelib, '_', '_'}

View file

@ -2059,6 +2059,7 @@ do_repair(
repairing=[_|_]=Repairing, repairing=[_|_]=Repairing,
members_dict=MembersDict}}=_S_copy, members_dict=MembersDict}}=_S_copy,
Opts, ap_mode=RepairMode) -> Opts, ap_mode=RepairMode) ->
?V("RePaiR-~w,", [self()]),
T1 = os:timestamp(), T1 = os:timestamp(),
RepairId = proplists:get_value(repair_id, Opts, id1), RepairId = proplists:get_value(repair_id, Opts, id1),
error_logger:info_msg("Repair start: tail ~p of ~p -> ~p, ~p ID ~w\n", error_logger:info_msg("Repair start: tail ~p of ~p -> ~p, ~p ID ~w\n",

View file

@ -108,6 +108,7 @@
-include("machi.hrl"). -include("machi.hrl").
-include("machi_projection.hrl"). -include("machi_projection.hrl").
-include("machi_verbose.hrl").
-ifdef(TEST). -ifdef(TEST).
-include_lib("eunit/include/eunit.hrl"). -include_lib("eunit/include/eunit.hrl").
@ -750,8 +751,10 @@ run_middleworker_job(Fun, ArgList, WTimeout) ->
Parent = self(), Parent = self(),
MiddleWorker = MiddleWorker =
spawn(fun() -> spawn(fun() ->
?V("Goo1-~w,", [self()]),
PidsMons = PidsMons =
[spawn_monitor(fun() -> [spawn_monitor(fun() ->
?V("Goo1-~w,", [self()]),
Res = (catch Fun(Arg)), Res = (catch Fun(Arg)),
exit(Res) exit(Res)
end) || Arg <- ArgList], end) || Arg <- ArgList],

View file

@ -77,6 +77,8 @@
-include("machi.hrl"). -include("machi.hrl").
-include("machi_pb.hrl"). -include("machi_pb.hrl").
-include("machi_projection.hrl"). -include("machi_projection.hrl").
-define(V(X,Y), ok).
%% -include("machi_verbose.hrl").
-ifdef(TEST). -ifdef(TEST).
-include_lib("eunit/include/eunit.hrl"). -include_lib("eunit/include/eunit.hrl").
@ -140,6 +142,7 @@ ets_table_name(FluName) when is_atom(FluName) ->
%% list_to_atom(binary_to_list(FluName) ++ "_epoch"). %% list_to_atom(binary_to_list(FluName) ++ "_epoch").
main2(FluName, TcpPort, DataDir, Rest) -> main2(FluName, TcpPort, DataDir, Rest) ->
?V("flu-~w,", [self()]),
{Props, DbgProps} = case proplists:get_value(dbg, Rest) of {Props, DbgProps} = case proplists:get_value(dbg, Rest) of
undefined -> undefined ->
{Rest, []}; {Rest, []};
@ -220,6 +223,7 @@ start_append_server(S, AckPid) ->
%% spawn_link(fun() -> run_projection_server(S) end). %% spawn_link(fun() -> run_projection_server(S) end).
run_listen_server(#state{flu_name=FluName, tcp_port=TcpPort}=S) -> run_listen_server(#state{flu_name=FluName, tcp_port=TcpPort}=S) ->
?V("listen-~w,", [self()]),
register(make_listener_regname(FluName), self()), register(make_listener_regname(FluName), self()),
SockOpts = ?PB_PACKET_OPTS ++ SockOpts = ?PB_PACKET_OPTS ++
[{reuseaddr, true}, {mode, binary}, {active, false}], [{reuseaddr, true}, {mode, binary}, {active, false}],
@ -235,6 +239,7 @@ run_listen_server(#state{flu_name=FluName, tcp_port=TcpPort}=S) ->
run_append_server(FluPid, AckPid, #state{flu_name=Name, run_append_server(FluPid, AckPid, #state{flu_name=Name,
wedged=Wedged_p,epoch_id=EpochId}=S) -> wedged=Wedged_p,epoch_id=EpochId}=S) ->
?V("append-~w,", [self()]),
%% Reminder: Name is the "main" name of the FLU, i.e., no suffix %% Reminder: Name is the "main" name of the FLU, i.e., no suffix
register(Name, self()), register(Name, self()),
TID = ets:new(ets_table_name(Name), TID = ets:new(ets_table_name(Name),
@ -247,7 +252,7 @@ run_append_server(FluPid, AckPid, #state{flu_name=Name,
listen_server_loop(LSock, S) -> listen_server_loop(LSock, S) ->
{ok, Sock} = gen_tcp:accept(LSock), {ok, Sock} = gen_tcp:accept(LSock),
spawn_link(fun() -> net_server_loop(Sock, S) end), spawn_link(fun() -> ?V("net_server-~w,", [self()]), net_server_loop(Sock, S) end),
listen_server_loop(LSock, S). listen_server_loop(LSock, S).
append_server_loop(FluPid, #state{data_dir=DataDir,wedged=Wedged_p, append_server_loop(FluPid, #state{data_dir=DataDir,wedged=Wedged_p,
@ -258,7 +263,7 @@ append_server_loop(FluPid, #state{data_dir=DataDir,wedged=Wedged_p,
From ! wedged, From ! wedged,
append_server_loop(FluPid, S); append_server_loop(FluPid, S);
{seq_append, From, Prefix, Chunk, CSum, Extra} -> {seq_append, From, Prefix, Chunk, CSum, Extra} ->
spawn(fun() -> append_server_dispatch(From, Prefix, spawn(fun() -> ?V("appendX-~w,", [self()]), append_server_dispatch(From, Prefix,
Chunk, CSum, Extra, Chunk, CSum, Extra,
DataDir, AppendServerPid) end), DataDir, AppendServerPid) end),
append_server_loop(FluPid, S); append_server_loop(FluPid, S);
@ -292,22 +297,36 @@ append_server_loop(FluPid, #state{data_dir=DataDir,wedged=Wedged_p,
end. end.
net_server_loop(Sock, S) -> net_server_loop(Sock, S) ->
?V("~w ~w,", [self(), ?LINE]),
case gen_tcp:recv(Sock, 0, ?SERVER_CMD_READ_TIMEOUT) of case gen_tcp:recv(Sock, 0, ?SERVER_CMD_READ_TIMEOUT) of
{ok, Bin} -> {ok, Bin} ->
?V("~w ~w,", [self(), ?LINE]),
{RespBin, S2} = {RespBin, S2} =
case machi_pb:decode_mpb_ll_request(Bin) of case machi_pb:decode_mpb_ll_request(Bin) of
LL_req when LL_req#mpb_ll_request.do_not_alter == 2 -> LL_req when LL_req#mpb_ll_request.do_not_alter == 2 ->
{R, NewS} = do_pb_ll_request(LL_req, S), ?V("~w ~w,", [self(), ?LINE]),
ZARF = (catch do_pb_ll_request(LL_req, S)),
%% ?V("~w ~w ~p,", [self(), ?LINE, ZARF]),
{R, NewS} = ZARF,
%% {R, NewS} = do_pb_ll_request(LL_req, S),
?V("~w ~w,", [self(), ?LINE]),
{machi_pb:encode_mpb_ll_response(R), mode(low, NewS)}; {machi_pb:encode_mpb_ll_response(R), mode(low, NewS)};
_ -> _ ->
?V("~w ~w,", [self(), ?LINE]),
HL_req = machi_pb:decode_mpb_request(Bin), HL_req = machi_pb:decode_mpb_request(Bin),
?V("~w ~w,", [self(), ?LINE]),
1 = HL_req#mpb_request.do_not_alter, 1 = HL_req#mpb_request.do_not_alter,
?V("~w ~w,", [self(), ?LINE]),
{R, NewS} = do_pb_hl_request(HL_req, make_high_clnt(S)), {R, NewS} = do_pb_hl_request(HL_req, make_high_clnt(S)),
?V("~w ~w,", [self(), ?LINE]),
{machi_pb:encode_mpb_response(R), mode(high, NewS)} {machi_pb:encode_mpb_response(R), mode(high, NewS)}
end, end,
?V("~w ~w,", [self(), ?LINE]),
ok = gen_tcp:send(Sock, RespBin), ok = gen_tcp:send(Sock, RespBin),
?V("~w ~w,", [self(), ?LINE]),
net_server_loop(Sock, S2); net_server_loop(Sock, S2);
{error, SockError} -> {error, SockError} ->
?V("~w ~w,", [self(), ?LINE]),
Msg = io_lib:format("Socket error ~w", [SockError]), Msg = io_lib:format("Socket error ~w", [SockError]),
R = #mpb_ll_response{req_id= <<>>, R = #mpb_ll_response{req_id= <<>>,
generic=#mpb_errorresp{code=1, msg=Msg}}, generic=#mpb_errorresp{code=1, msg=Msg}},
@ -337,10 +356,13 @@ make_high_clnt(S) ->
S. S.
do_pb_ll_request(#mpb_ll_request{req_id=ReqID}, #state{pb_mode=high}=S) -> do_pb_ll_request(#mpb_ll_request{req_id=ReqID}, #state{pb_mode=high}=S) ->
?V("~w ~w,", [self(), ?LINE]),
Result = {high_error, 41, "Low protocol request while in high mode"}, Result = {high_error, 41, "Low protocol request while in high mode"},
{machi_pb_translate:to_pb_response(ReqID, unused, Result), S}; {machi_pb_translate:to_pb_response(ReqID, unused, Result), S};
do_pb_ll_request(PB_request, S) -> do_pb_ll_request(PB_request, S) ->
?V("~w ~w,", [self(), ?LINE]),
Req = machi_pb_translate:from_pb_request(PB_request), Req = machi_pb_translate:from_pb_request(PB_request),
?V("~w ~w,", [self(), ?LINE]),
{ReqID, Cmd, Result, S2} = {ReqID, Cmd, Result, S2} =
case Req of case Req of
{RqID, {LowCmd, _}=CMD} {RqID, {LowCmd, _}=CMD}
@ -348,13 +370,19 @@ do_pb_ll_request(PB_request, S) ->
LowCmd == low_wedge_status; LowCmd == low_list_files -> LowCmd == low_wedge_status; LowCmd == low_list_files ->
%% Skip wedge check for projection commands! %% Skip wedge check for projection commands!
%% Skip wedge check for these unprivileged commands %% Skip wedge check for these unprivileged commands
?V("~w ~w,", [self(), ?LINE]),
{Rs, NewS} = do_pb_ll_request3(CMD, S), {Rs, NewS} = do_pb_ll_request3(CMD, S),
?V("~w ~w,", [self(), ?LINE]),
{RqID, CMD, Rs, NewS}; {RqID, CMD, Rs, NewS};
{RqID, CMD} -> {RqID, CMD} ->
?V("~w ~w,", [self(), ?LINE]),
EpochID = element(2, CMD), % by common convention EpochID = element(2, CMD), % by common convention
?V("~w ~w,", [self(), ?LINE]),
{Rs, NewS} = do_pb_ll_request2(EpochID, CMD, S), {Rs, NewS} = do_pb_ll_request2(EpochID, CMD, S),
?V("~w ~w,", [self(), ?LINE]),
{RqID, CMD, Rs, NewS} {RqID, CMD, Rs, NewS}
end, end,
?V("~w ~w,", [self(), ?LINE]),
{machi_pb_translate:to_pb_response(ReqID, Cmd, Result), S2}. {machi_pb_translate:to_pb_response(ReqID, Cmd, Result), S2}.
do_pb_ll_request2(EpochID, CMD, S) -> do_pb_ll_request2(EpochID, CMD, S) ->
@ -404,6 +432,7 @@ do_pb_ll_request3({low_delete_migration, _EpochID, File}, S) ->
do_pb_ll_request3({low_trunc_hack, _EpochID, File}, S) -> do_pb_ll_request3({low_trunc_hack, _EpochID, File}, S) ->
{do_server_trunc_hack(File, S), S}; {do_server_trunc_hack(File, S), S};
do_pb_ll_request3({low_proj, PCMD}, S) -> do_pb_ll_request3({low_proj, PCMD}, S) ->
?V("~w ~w,", [self(), ?LINE]),
{do_server_proj_request(PCMD, S), S}. {do_server_proj_request(PCMD, S), S}.
do_pb_hl_request(#mpb_request{req_id=ReqID}, #state{pb_mode=low}=S) -> do_pb_hl_request(#mpb_request{req_id=ReqID}, #state{pb_mode=low}=S) ->
@ -717,6 +746,7 @@ write_server_find_pid(Prefix) ->
start_seq_append_server(Prefix, DataDir, AppendServerPid) -> start_seq_append_server(Prefix, DataDir, AppendServerPid) ->
proc_lib:spawn_link(fun() -> proc_lib:spawn_link(fun() ->
?V("appendY-~w,", [self()]),
%% The following is only necessary to %% The following is only necessary to
%% make nice process relationships in %% make nice process relationships in
%% 'appmon' and related tools. %% 'appmon' and related tools.

View file

@ -61,6 +61,13 @@
-behaviour(supervisor). -behaviour(supervisor).
-include("machi_verbose.hrl").
-ifdef(PULSE).
-compile({parse_transform, pulse_instrument}).
-include_lib("pulse_otp/include/pulse_otp.hrl").
-endif.
%% External API %% External API
-export([make_package_spec/4, start_flu_package/4, stop_flu_package/1]). -export([make_package_spec/4, start_flu_package/4, stop_flu_package/1]).
%% Internal API %% Internal API
@ -92,6 +99,7 @@ start_link(FluName, TcpPort, DataDir, Props) ->
[FluName, TcpPort, DataDir, Props]). [FluName, TcpPort, DataDir, Props]).
init([FluName, TcpPort, DataDir, Props0]) -> init([FluName, TcpPort, DataDir, Props0]) ->
erlang:display({flu_psup,self()}),
RestartStrategy = one_for_all, RestartStrategy = one_for_all,
MaxRestarts = 1000, MaxRestarts = 1000,
MaxSecondsBetweenRestarts = 3600, MaxSecondsBetweenRestarts = 3600,

View file

@ -28,6 +28,13 @@
-behaviour(supervisor). -behaviour(supervisor).
-include("machi_verbose.hrl").
-ifdef(PULSE).
-compile({parse_transform, pulse_instrument}).
-include_lib("pulse_otp/include/pulse_otp.hrl").
-endif.
%% API %% API
-export([start_link/0]). -export([start_link/0]).
@ -40,15 +47,23 @@ start_link() ->
supervisor:start_link({local, ?SERVER}, ?MODULE, []). supervisor:start_link({local, ?SERVER}, ?MODULE, []).
init([]) -> init([]) ->
erlang:display({flu_sup,self()}),
RestartStrategy = one_for_one, RestartStrategy = one_for_one,
MaxRestarts = 1000, MaxRestarts = 1000,
MaxSecondsBetweenRestarts = 3600, MaxSecondsBetweenRestarts = 3600,
SupFlags = {RestartStrategy, MaxRestarts, MaxSecondsBetweenRestarts}, SupFlags = {RestartStrategy, MaxRestarts, MaxSecondsBetweenRestarts},
Ps = application:get_env(machi, initial_flus, []), Ps = get_initial_flus(),
FLU_specs = [machi_flu_psup:make_package_spec(FluName, TcpPort, FLU_specs = [machi_flu_psup:make_package_spec(FluName, TcpPort,
DataDir, Props) || DataDir, Props) ||
{FluName, TcpPort, DataDir, Props} <- Ps], {FluName, TcpPort, DataDir, Props} <- Ps],
{ok, {SupFlags, FLU_specs}}. {ok, {SupFlags, FLU_specs}}.
-ifdef(PULSE).
get_initial_flus() ->
[].
-else. % PULSE
get_initial_flus() ->
application:get_env(machi, initial_flus, []).
-endif. % PULSE

View file

@ -181,7 +181,7 @@ from_pb_request(#mpb_request{req_id=ReqID,
from_pb_request(#mpb_request{req_id=ReqID}) -> from_pb_request(#mpb_request{req_id=ReqID}) ->
{ReqID, {high_error, 999966, "Unknown request"}}; {ReqID, {high_error, 999966, "Unknown request"}};
from_pb_request(_Else) -> from_pb_request(_Else) ->
io:format(user, "\nRRR from_pb_request(~p)\n", [_Else]), timer:sleep(2000), io:format(user, "\nRRR from_pb_request(~p)\n", [_Else]), %%timer:sleep(2000),
{<<>>, {high_error, 999667, "Unknown PB request"}}. {<<>>, {high_error, 999667, "Unknown PB request"}}.
from_pb_response(#mpb_ll_response{ from_pb_response(#mpb_ll_response{

View file

@ -41,11 +41,13 @@
-module(machi_projection_store). -module(machi_projection_store).
-include("machi_projection.hrl"). -include("machi_projection.hrl").
-define(V(X,Y), ok).
%% -include("machi_verbose.hrl").
-ifdef(PULSE). %% -ifdef(PULSE).
-compile({parse_transform, pulse_instrument}). %% -compile({parse_transform, pulse_instrument}).
-include_lib("pulse_otp/include/pulse_otp.hrl"). %% -include_lib("pulse_otp/include/pulse_otp.hrl").
-endif. %% -endif.
%% API %% API
-export([ -export([
@ -106,6 +108,7 @@ read_latest_projection(PidSpec, ProjType) ->
read_latest_projection(PidSpec, ProjType, Timeout) read_latest_projection(PidSpec, ProjType, Timeout)
when ProjType == 'public' orelse ProjType == 'private' -> when ProjType == 'public' orelse ProjType == 'private' ->
?V("~w ~w ~w,", [self(), ?MODULE, ?LINE]),
g_call(PidSpec, {read_latest_projection, ProjType}, Timeout). g_call(PidSpec, {read_latest_projection, ProjType}, Timeout).
%% @doc Fetch the projection record type `ProjType' for epoch number `Epoch' . %% @doc Fetch the projection record type `ProjType' for epoch number `Epoch' .
@ -170,6 +173,7 @@ g_call(PidSpec, Arg, Timeout) ->
%%%%%%%%%%%%%%%%%%%%%%%%%%% %%%%%%%%%%%%%%%%%%%%%%%%%%%
init([DataDir, NotifyWedgeStateChanges]) -> init([DataDir, NotifyWedgeStateChanges]) ->
?V("pstore-~w,", [self()]),
lclock_init(), lclock_init(),
PublicDir = machi_util:make_projection_filename(DataDir, "public"), PublicDir = machi_util:make_projection_filename(DataDir, "public"),
PrivateDir = machi_util:make_projection_filename(DataDir, "private"), PrivateDir = machi_util:make_projection_filename(DataDir, "private"),
@ -197,7 +201,9 @@ handle_call({{read_latest_projection, ProjType}, LC1}, _From, S) ->
{EpochNum, _CSum} = if ProjType == public -> S#state.max_public_epochid; {EpochNum, _CSum} = if ProjType == public -> S#state.max_public_epochid;
ProjType == private -> S#state.max_private_epochid ProjType == private -> S#state.max_private_epochid
end, end,
?V("~w ~w ~w,", [self(), ?MODULE, ?LINE]),
{Reply, NewS} = do_proj_read(ProjType, EpochNum, S), {Reply, NewS} = do_proj_read(ProjType, EpochNum, S),
?V("~w ~w ~w,", [self(), ?MODULE, ?LINE]),
{reply, {Reply, LC2}, NewS}; {reply, {Reply, LC2}, NewS};
handle_call({{read, ProjType, Epoch}, LC1}, _From, S) -> handle_call({{read, ProjType, Epoch}, LC1}, _From, S) ->
LC2 = lclock_update(LC1), LC2 = lclock_update(LC1),

View file

@ -170,7 +170,7 @@ all_list() ->
[P#p_srvr.name || {P, _Dir} <- all_list_extra()]. [P#p_srvr.name || {P, _Dir} <- all_list_extra()].
setup(Num, Seed) -> setup(Num, Seed) ->
?V("\nsetup(~w", [Num]), ?V("\nsetup(~w,~w", [self(), Num]),
All_list = lists:sublist(all_list(), Num), All_list = lists:sublist(all_list(), Num),
All_listE = lists:sublist(all_list_extra(), Num), All_listE = lists:sublist(all_list_extra(), Num),
%% shutdown_hard() has taken care of killing all relevant procs. %% shutdown_hard() has taken care of killing all relevant procs.
@ -245,7 +245,7 @@ private_stable_check() ->
{_PSimPid, _SupPid, ProxiesDict, All_listE} = get(manager_pids_hack), {_PSimPid, _SupPid, ProxiesDict, All_listE} = get(manager_pids_hack),
Res = private_projections_are_stable_check(ProxiesDict, All_listE), Res = private_projections_are_stable_check(ProxiesDict, All_listE),
if not Res -> if not Res ->
?V("BUMMER: private stable check failed!\n", []); ?QC_FMT("BUMMER: private stable check failed!\n", []);
true -> true ->
ok ok
end, end,
@ -311,6 +311,7 @@ prop_pulse(Style) when Style == new; Style == regression ->
?FORALL({Cmds0, Seed}, {gen_commands(Style), pulse:seed()}, ?FORALL({Cmds0, Seed}, {gen_commands(Style), pulse:seed()},
?IMPLIES(1 < length(Cmds0) andalso length(Cmds0) < 11, ?IMPLIES(1 < length(Cmds0) andalso length(Cmds0) < 11,
begin begin
erlang:display({prop,?MODULE,?LINE,self()}),
ok = shutdown_hard(), ok = shutdown_hard(),
%% PULSE can be really unfair, of course, including having exec_ticks %% PULSE can be really unfair, of course, including having exec_ticks
%% run where all of FLU a does its ticks then FLU b. Such a situation %% run where all of FLU a does its ticks then FLU b. Such a situation
@ -334,9 +335,13 @@ prop_pulse(Style) when Style == new; Style == regression ->
pulse:verbose([format]), pulse:verbose([format]),
{_H2, S2, Res} = pulse:run( {_H2, S2, Res} = pulse:run(
fun() -> fun() ->
{_H, _S, _R} = run_commands(?MODULE, Cmds) ?V("PROP-~w,", [self()]),
%% {_H, _S, _R} = run_commands(?MODULE, Cmds)
QAQA = run_commands(?MODULE, Cmds)
,?V("pid681=~p", [process_info(list_to_pid("<0.681.0>"))]), QAQA
end, [{seed, Seed}, end, [{seed, Seed},
{strategy, unfair}]), {strategy, unfair}]),
ok = shutdown_hard(),
{Report, PrivProjs, Diag} = S2#state.dump_state, {Report, PrivProjs, Diag} = S2#state.dump_state,
%% Report is ordered by Epoch. For each private projection %% Report is ordered by Epoch. For each private projection
@ -368,7 +373,6 @@ prop_pulse(Style) when Style == new; Style == regression ->
{false, LastRepXs} {false, LastRepXs}
end, end,
ok = shutdown_hard(),
?WHENFAIL( ?WHENFAIL(
begin begin
%% ?QC_FMT("PrivProjs = ~P\n", [PrivProjs, 50]), %% ?QC_FMT("PrivProjs = ~P\n", [PrivProjs, 50]),
@ -456,20 +460,20 @@ shutdown_hard() ->
(catch unlink(whereis(machi_partition_simulator))), (catch unlink(whereis(machi_partition_simulator))),
[begin [begin
Pid = whereis(X), Pid = whereis(X),
spawn(fun() -> (catch X:stop()) end), %%%%%%DELME deadlock source? spawn(fun() -> ?QC_FMT("shutdown-~w,", [self()]), (catch X:stop()) end),
timer:sleep(50), timer:sleep(50),
timer:sleep(10), timer:sleep(10),
(catch exit(Pid, shutdown)), (catch exit(Pid, shutdown)),
timer:sleep(1), timer:sleep(1),
(catch exit(Pid, kill)) (catch exit(Pid, kill))
end || X <- [machi_partition_simulator, machi_flu_sup] ], end || X <- [machi_partition_simulator, machi_flu_sup] ],
timer:sleep(1), timer:sleep(100),
ok. ok.
exec_ticks(Num, All_listE) -> exec_ticks(Num, All_listE) ->
Parent = self(), Parent = self(),
Pids = [spawn_link(fun() -> Pids = [spawn_link(fun() ->
%% ?V("tick-~w,", [self()]), ?V("tick-~w,", [self()]),
[begin [begin
M_name = P#p_srvr.name, M_name = P#p_srvr.name,
%% Max = 10, %% Max = 10,