Refactor woody_clinet, woody_server_thrift_handler behaviour

* woody_clinet:client -> context
* woody_client:new -> new_context
* woody_server_thrift_handler behaviour: RpcId is not available as handle_function/handle_error argument anymore.
  It should be fetched from the provided Context value: woody_client:get_rpc_id(Context).
This commit is contained in:
Anton Belyaev 2016-05-05 14:18:49 +03:00
parent 18bdfa10eb
commit 7bebc67294
5 changed files with 190 additions and 179 deletions

View File

@ -8,15 +8,16 @@
-include("woody_defs.hrl").
%% API
-export([new/2]).
-export([make_child_client/2]).
-export([next/1]).
-export([new_context/2]).
-export([get_rpc_id/1]).
-export([make_child_context/2]).
-export([call/3]).
-export([call_safe/3]).
-export([call_async/5]).
-export_type([client/0, options/0, result_ok/0, result_error/0]).
-export_type([context/0, options/0, result_ok/0, result_error/0]).
-define(ROOT_REQ_PARENT_ID, <<"undefined">>).
@ -27,30 +28,31 @@
-export([init/1]).
%% behaviour definition
-callback call(client(), request(), options()) -> result_ok() | no_return().
-callback call(context(), request(), options()) -> result_ok() | no_return().
%%
%% API
%%
-type client() :: #{ %% all elements are mandatory
-type context() :: #{ %% all elements are mandatory
root_rpc => boolean(),
span_id => woody_t:req_id() | undefined,
trace_id => woody_t:req_id(),
parent_id => woody_t:req_id(),
event_handler => woody_t:handler(),
seq => non_neg_integer()
seq => non_neg_integer(),
rpc_id => woody_t:rpc_id() | undefined
}.
-type class() :: throw | error | exit.
-type stacktrace() :: list().
-type result_ok() :: {ok | {ok, _Response}, client()}.
-type result_ok() :: {ok | {ok, _Response}, context()}.
-type result_error() ::
{{exception , woody_client_thrift:except_thrift()} , client()} |
{{error , woody_client_thrift:error_protocol()} , client()} |
{{error , woody_client_thrift_http_transport:error()} , client()} |
{{error , {class(), _Reason, stacktrace()}} , client()}.
{{exception , woody_client_thrift:except_thrift()} , context()} |
{{error , woody_client_thrift:error_protocol()} , context()} |
{{error , woody_client_thrift_http_transport:error()} , context()} |
{{error , {class(), _Reason, stacktrace()}} , context()}.
-type request() :: any().
@ -62,61 +64,62 @@
-type callback() :: fun((result_ok() | result_error()) -> _).
-spec new(woody_t:req_id(), woody_t:handler()) -> client().
new(ReqId, EventHandler) ->
-spec new_context(woody_t:req_id(), woody_t:handler()) -> context().
new_context(ReqId, EventHandler) ->
#{
root_rpc => true,
span_id => ReqId,
trace_id => ReqId,
parent_id => ?ROOT_REQ_PARENT_ID,
seq => 0,
event_handler => EventHandler
event_handler => EventHandler,
rpc_id => undefined
}.
-spec make_child_client(woody_t:rpc_id(), woody_t:handler()) -> client().
make_child_client(#{span_id := ReqId, trace_id := TraceId}, EventHandler) ->
-spec make_child_context(woody_t:rpc_id(), woody_t:handler()) -> context().
make_child_context(RpcId = #{span_id := ReqId, trace_id := TraceId}, EventHandler) ->
#{
root_rpc => false,
span_id => undefined,
trace_id => TraceId,
parent_id => ReqId,
seq => 0,
event_handler => EventHandler
event_handler => EventHandler,
rpc_id => RpcId
}.
-spec next(client()) -> client().
next(Client = #{root_rpc := true}) ->
Client;
next(Client = #{root_rpc := false, seq := Seq}) ->
NextSeq = Seq +1,
Client#{span_id => make_req_id(NextSeq), seq => NextSeq}.
-spec get_rpc_id(context()) -> woody_t:rpc_id() | undefined | no_return().
get_rpc_id(#{rpc_id := RpcId}) ->
RpcId;
get_rpc_id(_) ->
error(badarg).
-spec call(client(), request(), options()) -> result_ok() | no_return().
call(Client, Request, Options) ->
-spec call(context(), request(), options()) -> result_ok() | no_return().
call(Context, Request, Options) ->
ProtocolHandler = woody_t:get_protocol_handler(client, Options),
ProtocolHandler:call(next(Client), Request, Options).
ProtocolHandler:call(next(Context), Request, Options).
-spec call_safe(client(), request(), options()) -> result_ok() | result_error().
call_safe(Client, Request, Options) ->
try call(Client, Request, Options)
-spec call_safe(context(), request(), options()) -> result_ok() | result_error().
call_safe(Context, Request, Options) ->
try call(Context, Request, Options)
catch
%% valid thrift exception
throw:{Except = ?except_thrift(_), Client1} ->
{Except, Client1};
throw:{Except = ?except_thrift(_), Context1} ->
{Except, Context1};
%% rpc send failed
error:{TError = ?error_transport(_), Client1} ->
{{error, TError}, Client1};
error:{TError = ?error_transport(_), Context1} ->
{{error, TError}, Context1};
%% thrift protocol error
error:{PError = ?error_protocol(_), Client1} ->
{{error, PError}, Client1};
error:{PError = ?error_protocol(_), Context1} ->
{{error, PError}, Context1};
%% what else could have happened?
Class:Reason ->
{{error, {Class, Reason, erlang:get_stacktrace()}}, Client}
{{error, {Class, Reason, erlang:get_stacktrace()}}, Context}
end.
-spec call_async(woody_t:sup_ref(), callback(), client(), request(), options()) ->
{ok, pid(), client()} | {error, _}.
call_async(Sup, Callback, Client, Request, Options) ->
-spec call_async(woody_t:sup_ref(), callback(), context(), request(), options()) ->
{ok, pid(), context()} | {error, _}.
call_async(Sup, Callback, Context, Request, Options) ->
_ = woody_t:get_protocol_handler(client, Options),
SupervisorSpec = #{
id => {?MODULE, woody_clients_sup},
@ -131,19 +134,19 @@ call_async(Sup, Callback, Client, Request, Options) ->
{error, {already_started, Pid}} -> Pid
end,
supervisor:start_child(ClientSup,
[Callback, Client, Request, Options]).
[Callback, Context, Request, Options]).
%%
%% Internal API
%%
-spec init_call_async(callback(), client(), request(), options()) -> {ok, pid(), client()}.
init_call_async(Callback, Client, Request, Options) ->
proc_lib:start_link(?MODULE, do_call_async, [Callback, Client, Request, Options]).
-spec init_call_async(callback(), context(), request(), options()) -> {ok, pid(), context()}.
init_call_async(Callback, Context, Request, Options) ->
proc_lib:start_link(?MODULE, do_call_async, [Callback, Context, Request, Options]).
-spec do_call_async(callback(), client(), request(), options()) -> _.
do_call_async(Callback, Client, Request, Options) ->
proc_lib:init_ack({ok, self(), next(Client)}),
Callback(call_safe(Client, Request, Options)).
-spec do_call_async(callback(), context(), request(), options()) -> _.
do_call_async(Callback, Context, Request, Options) ->
proc_lib:init_ack({ok, self(), next(Context)}),
Callback(call_safe(Context, Request, Options)).
%%
%% Supervisor callbacks
@ -170,6 +173,13 @@ init(woody_client_sup) ->
%%
%% Internal functions
%%
-spec next(context()) -> context().
next(Context = #{root_rpc := true}) ->
Context;
next(Context = #{root_rpc := false, seq := Seq}) ->
NextSeq = Seq +1,
Context#{span_id => make_req_id(NextSeq), seq => NextSeq}.
-spec make_req_id(non_neg_integer()) -> woody_t:req_id().
make_req_id(Seq) ->
BinSeq = genlib:to_binary(Seq),

View File

@ -38,12 +38,12 @@ start_pool(Name, PoolSize) when is_integer(PoolSize) ->
stop_pool(Name) ->
woody_client_thrift_http_transport:stop_client_pool(Name).
-spec call(woody_client:client(), request(), woody_client:options()) ->
-spec call(woody_client:context(), request(), woody_client:options()) ->
woody_client:result_ok() | no_return().
call(Client = #{event_handler := EventHandler},
call(Context = #{event_handler := EventHandler},
{Service = {_, ServiceName}, Function, Args}, TransportOpts)
->
RpcId = maps:with([span_id, trace_id, parent_id], Client),
RpcId = maps:with([span_id, trace_id, parent_id], Context),
woody_event_handler:handle_event(EventHandler, ?EV_CALL_SERVICE, RpcId#{
service => ServiceName,
function => Function,
@ -54,7 +54,7 @@ call(Client = #{event_handler := EventHandler},
do_call(
make_thrift_client(RpcId, Service, TransportOpts, EventHandler),
Function, Args
), RpcId, Client
), RpcId, Context
).
@ -91,40 +91,40 @@ do_call(Client, Function, Args) ->
format_return({ok, ok}, RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, ok, ?thrift_cast),
{ok, Client};
{ok, Context};
format_return({ok, Result}, RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, ok, Result),
{{ok, Result}, Client};
{{ok, Result}, Context};
%% In case a server violates the requirements and sends
%% #TAppiacationException{} with http status code 200.
format_return({exception, Result = #'TApplicationException'{}}, RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, error, Result),
error({?error_transport(server_error), Client});
error({?error_transport(server_error), Context});
%% Service threw valid thrift exception
format_return(Exception = ?except_thrift(_), RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, ok, Exception),
throw({Exception, Client});
throw({Exception, Context});
format_return({error, Error = ?error_transport(_)}, RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, error, Error),
error({Error, Client});
error({Error, Context});
format_return({error, Error}, RpcId,
Client = #{event_handler := EventHandler})
Context = #{event_handler := EventHandler})
->
?log_rpc_result(EventHandler, RpcId, error, Error),
error({?error_protocol(Error), Client}).
error({?error_protocol(Error), Context}).

View File

@ -1,7 +1,7 @@
-module(woody_server_thrift_handler).
%% API
-export([start/6]).
-export([start/5]).
-include_lib("thrift/include/thrift_constants.hrl").
-include_lib("thrift/include/thrift_protocol.hrl").
@ -16,13 +16,13 @@
-type args() :: tuple().
-export_type([handler_opts/0, args/0, result/0, error_reason/0]).
-callback handle_function(woody_t:func(), args(), woody_t:rpc_id(),
woody_client:client(), handler_opts())
-callback handle_function(woody_t:func(), args(),
woody_client:context(), handler_opts())
->
ok | {ok, result()} | {error, result()} | no_return().
-callback handle_error(woody_t:func(), error_reason(), woody_t:rpc_id(),
woody_client:client(), handler_opts())
-callback handle_error(woody_t:func(), error_reason(),
woody_client:context(), handler_opts())
-> _.
%%
@ -41,8 +41,7 @@
-define(error_protocol_send , send_error).
-record(state, {
rpc_id :: woody_t:rpc_id(),
woody_client :: woody_client:client(),
context :: woody_client:context(),
service :: woody_t:service(),
handler :: woody_t:handler(),
handler_opts :: handler_opts(),
@ -58,19 +57,18 @@
-type event_handler() :: woody_t:handler().
-type transport_handler() :: woody_t:handler().
-spec start(thrift_transport:t_transport(), woody_t:rpc_id(), woody_client:client(),
-spec start(thrift_transport:t_transport(), woody_client:context(),
thrift_handler(), event_handler(), transport_handler())
->
{ok | noreply | {error, _Reason}, cowboy_req:req()}.
start(Transport, RpcId, WoodyClient, {Service, Handler, Opts},
start(Transport, Context, {Service, Handler, Opts},
EventHandler, TransportHandler)
->
{ok, Protocol} = thrift_binary_protocol:new(Transport,
[{strict_read, true}, {strict_write, true}]
),
{Result, Protocol1} = process(#state{
rpc_id = RpcId,
woody_client = WoodyClient,
context = Context,
service = Service,
handler = Handler,
handler_opts = Opts,
@ -154,17 +152,17 @@ try_call_handler(Function, Args, State, SeqId) ->
end.
call_handler(Function,Args, #state{
rpc_id = RpcId,
woody_client = WoodyClient,
context = Context,
handler = Handler,
handler_opts = Opts,
service = {_, ServiceName},
event_handler = EventHandler})
->
RpcId = woody_client:get_rpc_id(Context),
woody_event_handler:handle_event(EventHandler, ?EV_INVOKE_SERVICE_HANDLER, RpcId#{
service => ServiceName, function => Function, args => Args, options => Opts
}),
Result = Handler:handle_function(Function, Args, RpcId, WoodyClient, Opts),
Result = Handler:handle_function(Function, Args, Context, Opts),
?log_rpc_result(EventHandler, ok, RpcId#{result => Result}),
Result.
@ -193,11 +191,12 @@ handle_success(State = #state{service = Service}, Function, Result, SeqId) ->
end.
handle_function_catch(State = #state{
rpc_id = RpcId,
context = Context,
service = Service,
event_handler = EventHandler
}, Function, Class, Reason, Stack, SeqId)
->
RpcId = woody_client:get_rpc_id(Context),
ReplyType = get_function_info(Service, Function, reply_type),
case {Class, Reason} of
_Error when ReplyType =:= oneway_void ->
@ -283,24 +282,26 @@ prepare_response({State, {error, Reason}}, FunctionName) ->
{handle_protocol_error(State, FunctionName, Reason), State#state.protocol}.
handle_protocol_error(State = #state{
rpc_id = RpcId,
context = Context,
protocol_stage = Stage,
transport_handler = Trans,
event_handler = EventHandler}, Function, Reason)
->
RpcId = woody_client:get_rpc_id(Context),
call_error_handler(State, Function, Reason),
woody_event_handler:handle_event(EventHandler, ?EV_THRIFT_ERROR,
RpcId#{stage => Stage, reason => Reason}),
format_protocol_error(Reason, Trans).
call_error_handler(#state{
rpc_id = RpcId,
woody_client = WoodyClient,
handler = Handler,
handler_opts = Opts,
event_handler = EventHandler}, Function, Reason) ->
context = Context,
handler = Handler,
handler_opts = Opts,
event_handler = EventHandler}, Function, Reason)
->
RpcId = woody_client:get_rpc_id(Context),
try
Handler:handle_error(Function, Reason, RpcId, WoodyClient, Opts)
Handler:handle_error(Function, Reason, Context, Opts)
catch
Class:Error ->
woody_event_handler:handle_event(EventHandler, ?EV_INTERNAL_ERROR, RpcId#{

View File

@ -83,7 +83,7 @@ check_callback(Callback, Module) ->
proplists:get_value(Callback, Module:module_info(exports)).
validate_handler(Handler) when is_atom(Handler) ->
[check_callback(F, 5, Handler) || F <- [handle_function, handle_error]],
[check_callback(F, 4, Handler) || F <- [handle_function, handle_error]],
Handler.
get_socket_transport(Ip, Port, Options) ->
@ -304,9 +304,9 @@ get_body(Req, ServerOpts) ->
end.
do_handle(RpcId, Body, ThriftHander, EventHandler, Req) ->
WoodyClient = woody_client:make_child_client(RpcId, EventHandler),
Context = woody_client:make_child_context(RpcId, EventHandler),
Transport = make_transport(Req, RpcId, Body, EventHandler),
case woody_server_thrift_handler:start(Transport, RpcId, WoodyClient, ThriftHander,
case woody_server_thrift_handler:start(Transport, Context, ThriftHander,
EventHandler, ?MODULE)
of
{ok, Req1} ->

View File

@ -15,8 +15,8 @@
-export([init/1]).
%% woody_server_thrift_handler callbacks
-export([handle_function/5]).
-export([handle_error/5]).
-export([handle_function/4]).
-export([handle_error/4]).
%% woody_event_handler callbacks
-export([handle_event/2]).
@ -215,18 +215,18 @@ call_handler_throw_test(_) ->
call_safe_handler_throw_unexpected_test(_) ->
Id = <<"call_safe_handler_throw_unexpected">>,
Current = genlib_map:get(<<"Rocket Launcher">>, ?WEAPONS),
Client = get_client(Id),
Expect = {{error, ?error_transport(server_error)}, Client},
Expect = call_safe(Client, weapons, switch_weapon,
Context = make_context(Id),
Expect = {{error, ?error_transport(server_error)}, Context},
Expect = call_safe(Context, weapons, switch_weapon,
[Current, next, 1, self_to_bin()]),
{ok, _} = receive_msg({Id, Current}).
call_handler_throw_unexpected_test(_) ->
Id = <<"call_handler_throw_unexpected">>,
Current = genlib_map:get(<<"Rocket Launcher">>, ?WEAPONS),
Client = get_client(Id),
Expect = {?error_transport(server_error), Client},
try call(Client, weapons, switch_weapon, [Current, next, 1, self_to_bin()])
Context = make_context(Id),
Expect = {?error_transport(server_error), Context},
try call(Context, weapons, switch_weapon, [Current, next, 1, self_to_bin()])
catch
error:Expect -> ok
end,
@ -244,26 +244,26 @@ call_handler_error_test(_) ->
call_safe_client_transport_error_test(_) ->
Gun = 'Wrong Type of Mega Destroyer',
Id = <<"call_safe_client_transport_error">>,
Client = get_client(Id),
{{error, ?error_protocol(_)}, Client} = call_safe(Client,
Id = <<"call_safe_client_transport_error">>,
Context = make_context(Id),
{{error, ?error_protocol(_)}, Context} = call_safe(Context,
weapons, get_weapon, [Gun, self_to_bin()]).
call_client_transport_error_test(_) ->
Gun = 'Wrong Type of Mega Destroyer',
Id = <<"call_client_transport_error">>,
Client = get_client(Id),
try call(Client, weapons, get_weapon, [Gun, self_to_bin()])
Id = <<"call_client_transport_error">>,
Context = make_context(Id),
try call(Context, weapons, get_weapon, [Gun, self_to_bin()])
catch
error:{?error_protocol(_), Client} -> ok
error:{?error_protocol(_), Context} -> ok
end.
call_safe_server_transport_error_test(_) ->
Id = <<"call_safe_server_transport_error">>,
Armor = <<"Helmet">>,
Client = get_client(Id),
Expect = {{error, ?error_transport(server_error)}, Client},
Expect = call_safe(Client, powerups, get_powerup,
Id = <<"call_safe_server_transport_error">>,
Armor = <<"Helmet">>,
Context = make_context(Id),
Expect = {{error, ?error_transport(server_error)}, Context},
Expect = call_safe(Context, powerups, get_powerup,
[Armor, self_to_bin()]),
{ok, _} = receive_msg({Id, Armor}).
@ -274,10 +274,10 @@ call_handle_error_fails_test(_) ->
do_call_server_transport_error(<<"call_handle_error_fails">>).
do_call_server_transport_error(Id) ->
Armor = <<"Helmet">>,
Client = get_client(Id),
Expect = {?error_transport(server_error), Client},
try call(Client, powerups, get_powerup, [Armor, self_to_bin()])
Armor = <<"Helmet">>,
Context = make_context(Id),
Expect = {?error_transport(server_error), Context},
try call(Context, powerups, get_powerup, [Armor, self_to_bin()])
catch
error:Expect -> ok
end,
@ -286,9 +286,9 @@ do_call_server_transport_error(Id) ->
call_oneway_void_test(_) ->
Id = <<"call_oneway_void_test">>,
Armor = <<"Helmet">>,
Client = get_client(Id),
Expect = {ok, Client},
Expect = call(Client, powerups, like_powerup, [Armor, self_to_bin()]),
Context = make_context(Id),
Expect = {ok, Context},
Expect = call(Context, powerups, like_powerup, [Armor, self_to_bin()]),
{ok, _} = receive_msg({Id, Armor}).
call_async_ok_test(C) ->
@ -296,40 +296,40 @@ call_async_ok_test(C) ->
Pid = self(),
Callback = fun(Res) -> collect(Res, Pid) end,
Id1 = <<"call_async_ok1">>,
Client1 = get_client(Id1),
{ok, Pid1, Client1} = get_weapon(Client1, Sup, Callback, <<"Impact Hammer">>),
Context1 = make_context(Id1),
{ok, Pid1, Context1} = get_weapon(Context1, Sup, Callback, <<"Impact Hammer">>),
Id2 = <<"call_async_ok2">>,
Client2 = get_client(Id2),
{ok, Pid2, Client2} = get_weapon(Client2, Sup, Callback, <<"Flak Cannon">>),
{ok, Pid1} = receive_msg({Client1,
Context2 = make_context(Id2),
{ok, Pid2, Context2} = get_weapon(Context2, Sup, Callback, <<"Flak Cannon">>),
{ok, Pid1} = receive_msg({Context1,
genlib_map:get(<<"Impact Hammer">>, ?WEAPONS)}),
{ok, Pid2} = receive_msg({Client2,
{ok, Pid2} = receive_msg({Context2,
genlib_map:get(<<"Flak Cannon">>, ?WEAPONS)}).
get_weapon(Client, Sup, Cb, Gun) ->
call_async(Client, weapons, get_weapon, [Gun, <<>>], Sup, Cb).
get_weapon(Context, Sup, Cb, Gun) ->
call_async(Context, weapons, get_weapon, [Gun, <<>>], Sup, Cb).
collect({{ok, Result}, Client}, Pid) ->
send_msg(Pid, {Client, Result}).
collect({{ok, Result}, Context}, Pid) ->
send_msg(Pid, {Context, Result}).
span_ids_sequence_test(_) ->
Id = <<"span_ids_sequence">>,
Current = genlib_map:get(<<"Enforcer">>, ?WEAPONS),
Client = get_client(Id),
Expect = {{ok, genlib_map:get(<<"Ripper">>, ?WEAPONS)}, Client},
Expect = call(Client, weapons, switch_weapon,
Context = make_context(Id),
Expect = {{ok, genlib_map:get(<<"Ripper">>, ?WEAPONS)}, Context},
Expect = call(Context, weapons, switch_weapon,
[Current, next, 1, self_to_bin()]).
call_with_client_pool_test(_) ->
Pool = guns,
ok = woody_client_thrift:start_pool(Pool, 10),
Id = <<"call_with_client_pool">>,
Gun = <<"Enforcer">>,
Client = get_client(Id),
ok = woody_client_thrift:start_pool(Pool, 10),
Id = <<"call_with_client_pool">>,
Gun = <<"Enforcer">>,
Context = make_context(Id),
{Url, Service} = get_service_endpoint(weapons),
Expect = {{ok, genlib_map:get(Gun, ?WEAPONS)}, Client},
Expect = {{ok, genlib_map:get(Gun, ?WEAPONS)}, Context},
Expect = woody_client:call(
Client,
Context,
{Service, get_weapon, [Gun, self_to_bin()]},
#{url => Url, pool => Pool}
),
@ -357,27 +357,27 @@ make_thrift_multiplexed_client(Id, ServiceName, {Url, Service}) ->
[{strict_read, true}, {strict_write, true}]
),
{ok, Protocol1} = thrift_multiplexed_protocol:new(Protocol, ServiceName),
{ok, Client} = thrift_client:new(Protocol1, Service),
Client.
{ok, Context} = thrift_client:new(Protocol1, Service),
Context.
allowed_transport_options_test(_) ->
Id = <<"allowed_transport_options">>,
Gun = <<"Enforcer">>,
Id = <<"allowed_transport_options">>,
Gun = <<"Enforcer">>,
Args = [Gun, self_to_bin()],
{Url, Service} = get_service_endpoint(weapons),
Pool = guns,
ok = woody_client_thrift:start_pool(Pool, 1),
Client = get_client(Id),
Context = make_context(Id),
Options = #{url => Url, pool => Pool, ssl_options => [], connect_timeout => 0},
{{error, ?error_transport(connect_timeout)}, Client} = woody_client:call_safe(
Client,
{{error, ?error_transport(connect_timeout)}, Context} = woody_client:call_safe(
Context,
{Service, get_weapon, Args},
Options
),
BadOpt = #{custom_option => 'fire!'},
ErrorBadOpt = {badarg, {unsupported_options, BadOpt}},
{{error, {error, ErrorBadOpt, _}}, Client} = woody_client:call_safe(
Client,
{{error, {error, ErrorBadOpt, _}}, Context} = woody_client:call_safe(
Context,
{Service, get_weapon, Args},
maps:merge(Options, BadOpt)
),
@ -427,15 +427,15 @@ init(_) ->
%% Weapons
handle_function(switch_weapon, {CurrentWeapon, Direction, Shift, To},
_RpcId = #{span_id := SpanId, trace_id := TraceId},
WoodyClient = #{parent_id := SpanId, trace_id := TraceId}, _Opts)
Context = #{ parent_id := SpanId, trace_id := TraceId,
rpc_id := #{span_id := SpanId, trace_id := TraceId}}, _Opts)
->
send_msg(To, {SpanId, CurrentWeapon}),
switch_weapon(CurrentWeapon, Direction, Shift, WoodyClient);
switch_weapon(CurrentWeapon, Direction, Shift, Context);
handle_function(get_weapon, {Name, To},
#{span_id := SpanId, trace_id := TraceId},
#{parent_id := SpanId, trace_id := TraceId}, _Opts)
_Context = #{ parent_id := SpanId, trace_id := TraceId,
rpc_id := #{span_id := SpanId, trace_id := TraceId}}, _Opts)
->
send_msg(To,{SpanId, Name}),
Res = case genlib_map:get(Name, ?WEAPONS) of
@ -448,26 +448,26 @@ handle_function(get_weapon, {Name, To},
%% Powerups
handle_function(get_powerup, {Name, To},
#{span_id := SpanId, trace_id := TraceId},
#{parent_id := SpanId, trace_id := TraceId}, _Opts)
_Context = #{ parent_id := SpanId, trace_id := TraceId,
rpc_id := #{span_id := SpanId, trace_id := TraceId}}, _Opts)
->
send_msg(To, {SpanId, Name}),
{ok, genlib_map:get(Name, ?POWERUPS, powerup_unknown)};
handle_function(like_powerup, {Name, To},
#{span_id := SpanId, trace_id := TraceId},
#{parent_id := SpanId, trace_id := TraceId}, _Opts)
_Context = #{ parent_id := SpanId, trace_id := TraceId,
rpc_id := #{span_id := SpanId, trace_id := TraceId}}, _Opts)
->
send_msg(To, {SpanId, Name}),
ok.
handle_error(get_powerup, _,
#{trace_id := TraceId, span_id := SpanId = <<"call_handle_error_fails">>},
#{trace_id := TraceId, parent_id := SpanId}, _Opts)
_Context = #{ trace_id := TraceId, span_id := SpanId = <<"call_handle_error_fails">>,
rpc_id := #{trace_id := TraceId, parent_id := SpanId}}, _Opts)
->
error(no_more_powerups);
handle_error(_Function, _Reason,
_RpcId = #{span_id := SpanId, trace_id := TraceId},
_WoodyClient = #{parent_id := SpanId, trace_id := TraceId}, _Opts)
_Context = #{ parent_id := SpanId, trace_id := TraceId,
rpc_id := #{span_id := SpanId, trace_id := TraceId}}, _Opts)
->
ok.
@ -481,27 +481,27 @@ handle_event(Type, Meta) ->
%%
%% internal functions
%%
get_client(ReqId) ->
woody_client:new(ReqId, ?MODULE).
make_context(ReqId) ->
woody_client:new_context(ReqId, ?MODULE).
call(Client, ServiceName, Function, Args) ->
do_call(call, Client, ServiceName, Function, Args).
call(Context, ServiceName, Function, Args) ->
do_call(call, Context, ServiceName, Function, Args).
call_safe(Client, ServiceName, Function, Args) ->
do_call(call_safe, Client, ServiceName, Function, Args).
call_safe(Context, ServiceName, Function, Args) ->
do_call(call_safe, Context, ServiceName, Function, Args).
do_call(Call, Client, ServiceName, Function, Args) ->
do_call(Call, Context, ServiceName, Function, Args) ->
{Url, Service} = get_service_endpoint(ServiceName),
woody_client:Call(
Client,
Context,
{Service, Function, Args},
#{url => Url}
).
call_async(Client, ServiceName, Function, Args, Sup, Callback) ->
call_async(Context, ServiceName, Function, Args, Sup, Callback) ->
{Url, Service} = get_service_endpoint(ServiceName),
woody_client:call_async(Sup, Callback,
Client,
Context,
{Service, Function, Args},
#{url => Url}
).
@ -518,18 +518,18 @@ get_service_endpoint(powerups) ->
}.
gun_test_basic(CallFun, Id, Gun, {ExpectStatus, ExpectRes}, WithMsg) ->
Client = get_client(Id),
Expect = {{ExpectStatus, ExpectRes}, Client},
Expect = ?MODULE:CallFun(Client, weapons, get_weapon, [Gun, self_to_bin()]),
Context = make_context(Id),
Expect = {{ExpectStatus, ExpectRes}, Context},
Expect = ?MODULE:CallFun(Context, weapons, get_weapon, [Gun, self_to_bin()]),
case WithMsg of
true -> {ok, _} = receive_msg({Id, Gun});
_ -> ok
end.
gun_catch_test_basic(Id, Gun, {Class, Exception}, WithMsg) ->
Client = get_client(Id),
Expect = {Exception, Client},
try call(Client, weapons, get_weapon, [Gun, self_to_bin()])
Context = make_context(Id),
Expect = {Exception, Context},
try call(Context, weapons, get_weapon, [Gun, self_to_bin()])
catch
Class:Expect -> ok
end,
@ -538,8 +538,8 @@ gun_catch_test_basic(Id, Gun, {Class, Exception}, WithMsg) ->
_ -> ok
end.
switch_weapon(CurrentWeapon, Direction, Shift, WoodyClient) ->
case call_safe(WoodyClient, weapons, get_weapon,
switch_weapon(CurrentWeapon, Direction, Shift, Context) ->
case call_safe(Context, weapons, get_weapon,
[new_weapon_name(CurrentWeapon, Direction, Shift), self_to_bin()])
of
{{ok, Weapon}, _} ->
@ -547,9 +547,9 @@ switch_weapon(CurrentWeapon, Direction, Shift, WoodyClient) ->
{{exception, #weapon_failure{
code = <<"weapon_error">>,
reason = <<"out of ammo">>
}}, NextClient} ->
ok = validate_next_client(NextClient, WoodyClient),
switch_weapon(CurrentWeapon, Direction, Shift + 1, NextClient)
}}, NextContex} ->
ok = validate_next_context(NextContex, Context),
switch_weapon(CurrentWeapon, Direction, Shift + 1, NextContex)
end.
new_weapon_name(#weapon{slot_pos = Pos}, next, Shift) ->
@ -562,7 +562,7 @@ new_weapon_name(Pos) when is_integer(Pos), Pos >= 0, Pos < 10 ->
new_weapon_name(_) ->
throw(?pos_error).
validate_next_client(#{seq := NextSeq}, #{seq := Seq}) ->
validate_next_context(#{seq := NextSeq}, #{seq := Seq}) ->
NextSeq = Seq + 1,
ok.