mirror of
https://github.com/valitydev/woody_erlang.git
synced 2024-11-06 02:15:19 +00:00
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:
parent
18bdfa10eb
commit
7bebc67294
@ -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),
|
||||
|
@ -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}).
|
||||
|
@ -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,
|
||||
context = Context,
|
||||
handler = Handler,
|
||||
handler_opts = Opts,
|
||||
event_handler = EventHandler}, Function, Reason) ->
|
||||
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#{
|
||||
|
@ -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} ->
|
||||
|
@ -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,
|
||||
@ -245,25 +245,25 @@ 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,
|
||||
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()])
|
||||
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,
|
||||
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}).
|
||||
|
||||
@ -275,9 +275,9 @@ call_handle_error_fails_test(_) ->
|
||||
|
||||
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()])
|
||||
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,28 +296,28 @@ 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(_) ->
|
||||
@ -325,11 +325,11 @@ call_with_client_pool_test(_) ->
|
||||
ok = woody_client_thrift:start_pool(Pool, 10),
|
||||
Id = <<"call_with_client_pool">>,
|
||||
Gun = <<"Enforcer">>,
|
||||
Client = get_client(Id),
|
||||
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,8 +357,8 @@ 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">>,
|
||||
@ -367,17 +367,17 @@ allowed_transport_options_test(_) ->
|
||||
{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.
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user