TD-818: Add GetExplanationForChosenRoute method (#113)

* TD-818: Analyze routes of payment

* TD-818: Add GetExplanationForChosenRoute method

* Add varset gathering and explanation for candidates

* Add attempted routes explanation

* Add varset to explanation output

* Fix tests

* Fix test

* Format

* Review fix

* Add test

* Fix

* Bump cache
This commit is contained in:
ndiezel0 2024-01-30 20:51:20 +05:00 committed by GitHub
parent fd2952e781
commit 01c646e15e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 464 additions and 69 deletions

View File

@ -38,4 +38,4 @@ jobs:
thrift-version: ${{ needs.setup.outputs.thrift-version }} thrift-version: ${{ needs.setup.outputs.thrift-version }}
run-ct-with-compose: true run-ct-with-compose: true
use-coveralls: true use-coveralls: true
cache-version: v4 cache-version: v5

View File

@ -27,7 +27,9 @@
capture_data :: undefined | hg_invoice_payment:capture_data(), capture_data :: undefined | hg_invoice_payment:capture_data(),
failure :: undefined | hg_invoice_payment:failure(), failure :: undefined | hg_invoice_payment:failure(),
timings :: undefined | hg_timings:t(), timings :: undefined | hg_timings:t(),
allocation :: undefined | hg_allocation:allocation() allocation :: undefined | hg_allocation:allocation(),
route_limits = #{} :: hg_routing:limits(),
route_scores = #{} :: hg_routing:scores()
}). }).
-record(refund_st, { -record(refund_st, {

View File

@ -30,7 +30,9 @@
). ).
-define(route_changed(Route), -define(route_changed(Route),
{invoice_payment_route_changed, #payproc_InvoicePaymentRouteChanged{route = Route}} {invoice_payment_route_changed, #payproc_InvoicePaymentRouteChanged{
route = Route
}}
). ).
-define(route_changed(Route, Candidates), -define(route_changed(Route, Candidates),
@ -40,6 +42,15 @@
}} }}
). ).
-define(route_changed(Route, Candidates, Scores, Limits),
{invoice_payment_route_changed, #payproc_InvoicePaymentRouteChanged{
route = Route,
candidates = Candidates,
scores = Scores,
limits = Limits
}}
).
-define(cash_flow_changed(CashFlow), -define(cash_flow_changed(CashFlow),
{invoice_payment_cash_flow_changed, #payproc_InvoicePaymentCashFlowChanged{ {invoice_payment_cash_flow_changed, #payproc_InvoicePaymentCashFlowChanged{
cash_flow = CashFlow cash_flow = CashFlow

View File

@ -11,6 +11,7 @@
genlib, genlib,
fault_detector_proto, fault_detector_proto,
hg_proto, hg_proto,
routing,
cowboy, cowboy,
woody, woody,
scoper, % should be before any scoper event handler usage scoper, % should be before any scoper event handler usage

View File

@ -140,7 +140,11 @@ handle_function_('RepairWithScenario', {InvoiceID, Scenario}, _Opts) ->
handle_function_('GetPaymentRoutesLimitValues', {InvoiceID, PaymentID}, _Opts) -> handle_function_('GetPaymentRoutesLimitValues', {InvoiceID, PaymentID}, _Opts) ->
_ = set_invoicing_meta(InvoiceID, PaymentID), _ = set_invoicing_meta(InvoiceID, PaymentID),
St = get_state(InvoiceID), St = get_state(InvoiceID),
hg_invoice_payment:get_limit_values(get_payment_session(PaymentID, St), hg_invoice:get_payment_opts(St)). hg_invoice_payment:get_limit_values(get_payment_session(PaymentID, St), hg_invoice:get_payment_opts(St));
handle_function_('ExplainRoute', {InvoiceID, PaymentID}, _Opts) ->
_ = set_invoicing_meta(InvoiceID, PaymentID),
St = get_state(InvoiceID),
hg_routing_explanation:get_explanation(get_payment_session(PaymentID, St), hg_invoice:get_payment_opts(St)).
ensure_started(ID, TemplateID, PartyRevision, Params, Allocation) -> ensure_started(ID, TemplateID, PartyRevision, Params, Allocation) ->
Invoice = hg_invoice:create(ID, TemplateID, PartyRevision, Params, Allocation), Invoice = hg_invoice:create(ID, TemplateID, PartyRevision, Params, Allocation),

View File

@ -102,6 +102,8 @@
-export([construct_payment_plan_id/1]). -export([construct_payment_plan_id/1]).
-export([construct_payment_plan_id/2]). -export([construct_payment_plan_id/2]).
-export([get_payer_payment_tool/1]).
%% %%
-export_type([payment_id/0]). -export_type([payment_id/0]).
@ -1982,18 +1984,22 @@ produce_routing_events(Ctx = #{error := Error}, _Revision, St) when Error =/= un
Route = hg_route:to_payment_route(hd(RollbackableCandidates)), Route = hg_route:to_payment_route(hd(RollbackableCandidates)),
Candidates = Candidates =
ordsets:from_list([hg_route:to_payment_route(R) || R <- RollbackableCandidates]), ordsets:from_list([hg_route:to_payment_route(R) || R <- RollbackableCandidates]),
RouteScores = hg_routing_ctx:route_scores(Ctx),
RouteLimits = hg_routing_ctx:route_limits(Ctx),
%% For protocol compatability we set choosen route in route_changed event. %% For protocol compatability we set choosen route in route_changed event.
%% It doesn't influence cash_flow building because this step will be %% It doesn't influence cash_flow building because this step will be
%% skipped. And all limit's 'hold' operations will be rolled back. %% skipped. And all limit's 'hold' operations will be rolled back.
%% For same purpose in cascade routing we use route from unfiltered list of %% For same purpose in cascade routing we use route from unfiltered list of
%% originally resolved candidates. %% originally resolved candidates.
[?route_changed(Route, Candidates), ?payment_rollback_started(Failure)]; [?route_changed(Route, Candidates, RouteScores, RouteLimits), ?payment_rollback_started(Failure)];
produce_routing_events(Ctx, Revision, _St) -> produce_routing_events(Ctx, Revision, _St) ->
ok = log_route_choice_meta(Ctx, Revision), ok = log_route_choice_meta(Ctx, Revision),
Route = hg_route:to_payment_route(hg_routing_ctx:choosen_route(Ctx)), Route = hg_route:to_payment_route(hg_routing_ctx:choosen_route(Ctx)),
Candidates = Candidates =
ordsets:from_list([hg_route:to_payment_route(R) || R <- hg_routing_ctx:considered_candidates(Ctx)]), ordsets:from_list([hg_route:to_payment_route(R) || R <- hg_routing_ctx:considered_candidates(Ctx)]),
[?route_changed(Route, Candidates)]. RouteScores = hg_routing_ctx:route_scores(Ctx),
RouteLimits = hg_routing_ctx:route_limits(Ctx),
[?route_changed(Route, Candidates, RouteScores, RouteLimits)].
route_args(St) -> route_args(St) ->
Opts = get_opts(St), Opts = get_opts(St),
@ -2484,9 +2490,10 @@ filter_routes_with_limit_hold(Ctx0, VS, Iter, St) ->
Ctx1 = reject_routes(limit_misconfiguration, RejectedRoutes, Ctx0), Ctx1 = reject_routes(limit_misconfiguration, RejectedRoutes, Ctx0),
hg_routing_ctx:stash_current_candidates(Ctx1). hg_routing_ctx:stash_current_candidates(Ctx1).
filter_routes_by_limit_overflow(Ctx, VS, St) -> filter_routes_by_limit_overflow(Ctx0, VS, St) ->
{_Routes, RejectedRoutes} = get_limit_overflow_routes(hg_routing_ctx:candidates(Ctx), VS, St), {_Routes, RejectedRoutes, Limits} = get_limit_overflow_routes(hg_routing_ctx:candidates(Ctx0), VS, St),
reject_routes(limit_overflow, RejectedRoutes, Ctx). Ctx1 = hg_routing_ctx:stash_route_limits(Limits, Ctx0),
reject_routes(limit_overflow, RejectedRoutes, Ctx1).
reject_routes(GroupReason, RejectedRoutes, Ctx) -> reject_routes(GroupReason, RejectedRoutes, Ctx) ->
lists:foldr( lists:foldr(
@ -2501,19 +2508,19 @@ get_limit_overflow_routes(Routes, VS, St) ->
Payment = get_payment(St), Payment = get_payment(St),
Invoice = get_invoice(Opts), Invoice = get_invoice(Opts),
lists:foldl( lists:foldl(
fun(Route, {RoutesNoOverflowIn, RejectedIn}) -> fun(Route, {RoutesNoOverflowIn, RejectedIn, LimitsIn}) ->
PaymentRoute = hg_route:to_payment_route(Route), PaymentRoute = hg_route:to_payment_route(Route),
ProviderTerms = hg_routing:get_payment_terms(PaymentRoute, VS, Revision), ProviderTerms = hg_routing:get_payment_terms(PaymentRoute, VS, Revision),
TurnoverLimits = get_turnover_limits(ProviderTerms), TurnoverLimits = get_turnover_limits(ProviderTerms),
case hg_limiter:check_limits(TurnoverLimits, Invoice, Payment, PaymentRoute) of case hg_limiter:check_limits(TurnoverLimits, Invoice, Payment, PaymentRoute) of
{ok, _} -> {ok, Limits} ->
{[Route | RoutesNoOverflowIn], RejectedIn}; {[Route | RoutesNoOverflowIn], RejectedIn, LimitsIn#{PaymentRoute => Limits}};
{error, {limit_overflow, IDs}} -> {error, {limit_overflow, IDs, Limits}} ->
RejectedRoute = hg_route:to_rejected_route(Route, {'LimitOverflow', IDs}), RejectedRoute = hg_route:to_rejected_route(Route, {'LimitOverflow', IDs}),
{RoutesNoOverflowIn, [RejectedRoute | RejectedIn]} {RoutesNoOverflowIn, [RejectedRoute | RejectedIn], LimitsIn#{PaymentRoute => Limits}}
end end
end, end,
{[], []}, {[], [], #{}},
Routes Routes
). ).
@ -2875,6 +2882,7 @@ get_payment_tool(#domain_InvoicePayment{payer = Payer}) ->
get_payment_created_at(#domain_InvoicePayment{created_at = CreatedAt}) -> get_payment_created_at(#domain_InvoicePayment{created_at = CreatedAt}) ->
CreatedAt. CreatedAt.
-spec get_payer_payment_tool(payer()) -> payment_tool().
get_payer_payment_tool(?payment_resource_payer(PaymentResource, _ContactInfo)) -> get_payer_payment_tool(?payment_resource_payer(PaymentResource, _ContactInfo)) ->
get_resource_payment_tool(PaymentResource); get_resource_payment_tool(PaymentResource);
get_payer_payment_tool(?customer_payer(_CustomerID, _, _, PaymentTool, _)) -> get_payer_payment_tool(?customer_payer(_CustomerID, _, _, PaymentTool, _)) ->
@ -2941,7 +2949,11 @@ merge_change(Change = ?risk_score_changed(RiskScore), #st{} = St, Opts) ->
risk_score = RiskScore, risk_score = RiskScore,
activity = {payment, routing} activity = {payment, routing}
}; };
merge_change(Change = ?route_changed(Route, Candidates), #st{routes = Routes} = St, Opts) -> merge_change(
Change = ?route_changed(Route, Candidates, Scores, Limits),
#st{routes = Routes, route_scores = RouteScores, route_limits = RouteLimits} = St,
Opts
) ->
_ = validate_transition([{payment, S} || S <- [routing, processing_failure]], Change, St, Opts), _ = validate_transition([{payment, S} || S <- [routing, processing_failure]], Change, St, Opts),
St#st{ St#st{
%% On route change we expect cash flow from previous attempt to be rolled back. %% On route change we expect cash flow from previous attempt to be rolled back.
@ -2951,7 +2963,9 @@ merge_change(Change = ?route_changed(Route, Candidates), #st{routes = Routes} =
trx = undefined, trx = undefined,
routes = [Route | Routes], routes = [Route | Routes],
candidate_routes = ordsets:to_list(Candidates), candidate_routes = ordsets:to_list(Candidates),
activity = {payment, cash_flow_building} activity = {payment, cash_flow_building},
route_scores = hg_maybe:apply(fun(S) -> maps:merge(RouteScores, S) end, Scores, RouteScores),
route_limits = hg_maybe:apply(fun(L) -> maps:merge(RouteLimits, L) end, Limits, RouteLimits)
}; };
merge_change(Change = ?payment_capture_started(Data), #st{} = St, Opts) -> merge_change(Change = ?payment_capture_started(Data), #st{} = St, Opts) ->
_ = validate_transition([{payment, S} || S <- [flow_waiting]], Change, St, Opts), _ = validate_transition([{payment, S} || S <- [flow_waiting]], Change, St, Opts),

View File

@ -19,6 +19,8 @@
-type change_queue() :: [hg_limiter_client:limit_change()]. -type change_queue() :: [hg_limiter_client:limit_change()].
-export_type([turnover_limit_value/0]).
-export([get_turnover_limits/1]). -export([get_turnover_limits/1]).
-export([check_limits/4]). -export([check_limits/4]).
-export([hold_payment_limits/5]). -export([hold_payment_limits/5]).
@ -60,31 +62,33 @@ get_limit_values(TurnoverLimits, Invoice, Payment, Route) ->
). ).
-spec check_limits([turnover_limit()], invoice(), payment(), route()) -> -spec check_limits([turnover_limit()], invoice(), payment(), route()) ->
{ok, [hg_limiter_client:limit()]} {ok, [turnover_limit_value()]}
| {error, {limit_overflow, [binary()]}}. | {error, {limit_overflow, [binary()], [turnover_limit_value()]}}.
check_limits(TurnoverLimits, Invoice, Payment, Route) -> check_limits(TurnoverLimits, Invoice, Payment, Route) ->
Context = gen_limit_context(Invoice, Payment, Route), Context = gen_limit_context(Invoice, Payment, Route),
{ok, Limits} = gather_limits(TurnoverLimits, Context, []),
try try
check_limits_(TurnoverLimits, Context, []) ok = check_limits_(Limits, Context),
{ok, Limits}
catch catch
throw:limit_overflow -> throw:limit_overflow ->
IDs = [T#domain_TurnoverLimit.id || T <- TurnoverLimits], IDs = [T#domain_TurnoverLimit.id || T <- TurnoverLimits],
{error, {limit_overflow, IDs}} {error, {limit_overflow, IDs, Limits}}
end. end.
check_limits_([], _, Limits) -> check_limits_([], _) ->
{ok, Limits}; ok;
check_limits_([T | TurnoverLimits], Context, Acc) -> check_limits_([TurnoverLimitValue | TLVs], Context) ->
#domain_TurnoverLimit{id = LimitID, domain_revision = Version} = T, #payproc_TurnoverLimitValue{
Clock = get_latest_clock(), limit = #domain_TurnoverLimit{
Limit = hg_limiter_client:get(LimitID, Version, Clock, Context), id = LimitID,
#limiter_Limit{ upper_boundary = UpperBoundary
amount = LimiterAmount },
} = Limit, value = LimiterAmount
UpperBoundary = T#domain_TurnoverLimit.upper_boundary, } = TurnoverLimitValue,
case LimiterAmount =< UpperBoundary of case LimiterAmount =< UpperBoundary of
true -> true ->
check_limits_(TurnoverLimits, Context, [Limit | Acc]); check_limits_(TLVs, Context);
false -> false ->
logger:notice("Limit with id ~p overflowed, amount ~p upper boundary ~p", [ logger:notice("Limit with id ~p overflowed, amount ~p upper boundary ~p", [
LimitID, LimitID,
@ -94,6 +98,15 @@ check_limits_([T | TurnoverLimits], Context, Acc) ->
throw(limit_overflow) throw(limit_overflow)
end. end.
gather_limits([], _Context, Acc) ->
{ok, Acc};
gather_limits([T | TurnoverLimits], Context, Acc) ->
#domain_TurnoverLimit{id = LimitID, domain_revision = Version} = T,
Clock = get_latest_clock(),
#limiter_Limit{amount = Amount} = hg_limiter_client:get(LimitID, Version, Clock, Context),
TurnoverLimitValue = #payproc_TurnoverLimitValue{limit = T, value = Amount},
gather_limits(TurnoverLimits, Context, [TurnoverLimitValue | Acc]).
-spec hold_payment_limits([turnover_limit()], route(), pos_integer(), invoice(), payment()) -> ok. -spec hold_payment_limits([turnover_limit()], route(), pos_integer(), invoice(), payment()) -> ok.
hold_payment_limits(TurnoverLimits, Route, Iter, Invoice, Payment) -> hold_payment_limits(TurnoverLimits, Route, Iter, Invoice, Payment) ->
ChangeIDs = [construct_payment_change_id(Route, Iter, Invoice, Payment)], ChangeIDs = [construct_payment_change_id(Route, Iter, Invoice, Payment)],

View File

@ -6229,7 +6229,7 @@ payment_cascade_success(C) ->
next_change(InvoiceID, Client), next_change(InvoiceID, Client),
?payment_ev(PaymentID, ?risk_score_changed(_)) = ?payment_ev(PaymentID, ?risk_score_changed(_)) =
next_change(InvoiceID, Client), next_change(InvoiceID, Client),
{_Route1, _CashFlow1, TrxID1, Failure1} = {Route1, _CashFlow1, TrxID1, Failure1} =
await_cascade_triggering(InvoiceID, PaymentID, Client), await_cascade_triggering(InvoiceID, PaymentID, Client),
ok = payproc_errors:match('PaymentFailure', Failure1, fun({preauthorization_failed, {card_blocked, _}}) -> ok end), ok = payproc_errors:match('PaymentFailure', Failure1, fun({preauthorization_failed, {card_blocked, _}}) -> ok end),
%% Assert payment status IS NOT failed %% Assert payment status IS NOT failed
@ -6271,7 +6271,21 @@ payment_cascade_success(C) ->
Trx Trx
), ),
%% At the end of this scenario limit must be accounted only once. %% At the end of this scenario limit must be accounted only once.
hg_limiter_helper:assert_payment_limit_amount(?LIMIT_ID4, InitialAccountedAmount + Amount, PaymentFinal, Invoice). _ = hg_limiter_helper:assert_payment_limit_amount(
?LIMIT_ID4, InitialAccountedAmount + Amount, PaymentFinal, Invoice
),
#payproc_InvoicePaymentExplanation{
explained_routes = [
#payproc_InvoicePaymentRouteExplanation{
route = Route2,
is_chosen = true
},
#payproc_InvoicePaymentRouteExplanation{
route = Route1,
is_chosen = false
}
]
} = hg_client_invoicing:explain_route(InvoiceID, PaymentID, Client).
payment_cascade_success_w_refund_fixture(Revision, _C) -> payment_cascade_success_w_refund_fixture(Revision, _C) ->
Brovider = Brovider =

View File

@ -42,6 +42,8 @@
-export([compute_terms/3]). -export([compute_terms/3]).
-export([explain_route/3]).
-export([pull_event/2]). -export([pull_event/2]).
-export([pull_event/3]). -export([pull_event/3]).
-export([pull_change/4]). -export([pull_change/4]).
@ -86,6 +88,8 @@
-type chargeback_reject_params() :: dmsl_payproc_thrift:'InvoicePaymentChargebackRejectParams'(). -type chargeback_reject_params() :: dmsl_payproc_thrift:'InvoicePaymentChargebackRejectParams'().
-type chargeback_reopen_params() :: dmsl_payproc_thrift:'InvoicePaymentChargebackReopenParams'(). -type chargeback_reopen_params() :: dmsl_payproc_thrift:'InvoicePaymentChargebackReopenParams'().
-type invoice_payment_explanation() :: dmsl_payproc_thrift:'InvoicePaymentExplanation'().
-type term_set() :: dmsl_domain_thrift:'TermSet'(). -type term_set() :: dmsl_domain_thrift:'TermSet'().
-type cash() :: undefined | dmsl_domain_thrift:'Cash'(). -type cash() :: undefined | dmsl_domain_thrift:'Cash'().
-type cart() :: undefined | dmsl_domain_thrift:'InvoiceCart'(). -type cart() :: undefined | dmsl_domain_thrift:'InvoiceCart'().
@ -300,6 +304,12 @@ compute_terms(InvoiceID, PartyRevision, Client) ->
gen_server:call(Client, {call, 'ComputeTerms', [InvoiceID, PartyRevision], otel_ctx:get_current()}) gen_server:call(Client, {call, 'ComputeTerms', [InvoiceID, PartyRevision], otel_ctx:get_current()})
). ).
-spec explain_route(invoice_id(), payment_id(), pid()) ->
invoice_payment_explanation() | woody_error:business_error().
explain_route(InvoiceID, PaymentID, Client) ->
Args = [InvoiceID, PaymentID],
map_result_error(gen_server:call(Client, {call, 'ExplainRoute', Args, otel_ctx:get_current()})).
-define(DEFAULT_NEXT_EVENT_TIMEOUT, 5000). -define(DEFAULT_NEXT_EVENT_TIMEOUT, 5000).
-spec pull_event(invoice_id(), pid()) -> -spec pull_event(invoice_id(), pid()) ->

View File

@ -1,5 +1,7 @@
-module(hg_route). -module(hg_route).
-include_lib("hellgate/include/domain.hrl").
-export([new/2]). -export([new/2]).
-export([new/4]). -export([new/4]).
-export([new/5]). -export([new/5]).
@ -20,8 +22,6 @@
%% %%
-include("domain.hrl").
-record(route, { -record(route, {
provider_ref :: dmsl_domain_thrift:'ProviderRef'(), provider_ref :: dmsl_domain_thrift:'ProviderRef'(),
terminal_ref :: dmsl_domain_thrift:'TerminalRef'(), terminal_ref :: dmsl_domain_thrift:'TerminalRef'(),

View File

@ -5,6 +5,7 @@
-include_lib("damsel/include/dmsl_domain_thrift.hrl"). -include_lib("damsel/include/dmsl_domain_thrift.hrl").
-include_lib("damsel/include/dmsl_payproc_thrift.hrl"). -include_lib("damsel/include/dmsl_payproc_thrift.hrl").
-include_lib("fault_detector_proto/include/fd_proto_fault_detector_thrift.hrl"). -include_lib("fault_detector_proto/include/fd_proto_fault_detector_thrift.hrl").
-include_lib("hellgate/include/domain.hrl").
-export([gather_routes/5]). -export([gather_routes/5]).
-export([rate_routes/1]). -export([rate_routes/1]).
@ -23,8 +24,6 @@
%% %%
-include("domain.hrl").
-type payment_terms() :: dmsl_domain_thrift:'PaymentsProvisionTerms'(). -type payment_terms() :: dmsl_domain_thrift:'PaymentsProvisionTerms'().
-type payment_institution() :: dmsl_domain_thrift:'PaymentInstitution'(). -type payment_institution() :: dmsl_domain_thrift:'PaymentInstitution'().
-type route_predestination() :: payment | recurrent_paytool | recurrent_payment. -type route_predestination() :: payment | recurrent_paytool | recurrent_payment.
@ -46,8 +45,6 @@
-type conversion_condition() :: normal | lacking. -type conversion_condition() :: normal | lacking.
-type conversion_fail_rate() :: float(). -type conversion_fail_rate() :: float().
-type condition_score() :: 0 | 1.
-type route_groups_by_priority() :: #{{availability_condition(), terminal_priority_rating()} => [fail_rated_route()]}. -type route_groups_by_priority() :: #{{availability_condition(), terminal_priority_rating()} => [fail_rated_route()]}.
-type fail_rated_route() :: {hg_route:t(), provider_status()}. -type fail_rated_route() :: {hg_route:t(), provider_status()}.
@ -57,7 +54,7 @@
-type route_choice_context() :: #{ -type route_choice_context() :: #{
chosen_route => hg_route:t(), chosen_route => hg_route:t(),
preferable_route => hg_route:t(), preferable_route => hg_route:t(),
% Contains one of the field names defined in #route_scores{} % Contains one of the field names defined in #domain_PaymentRouteScores{}
reject_reason => atom() reject_reason => atom()
}. }.
@ -76,28 +73,25 @@
-type varset() :: hg_varset:varset(). -type varset() :: hg_varset:varset().
-type revision() :: hg_domain:revision(). -type revision() :: hg_domain:revision().
-record(route_scores, { -type route_scores() :: #domain_PaymentRouteScores{}.
availability_condition :: condition_score(), -type limits() :: #{hg_route:payment_route() => [hg_limiter:turnover_limit_value()]}.
conversion_condition :: condition_score(), -type scores() :: #{hg_route:payment_route() => hg_routing:route_scores()}.
priority_rating :: terminal_priority_rating(),
pin :: integer(),
random_condition :: integer(),
availability :: float(),
conversion :: float()
}).
-type route_scores() :: #route_scores{}.
-type misconfiguration_error() :: {misconfiguration, {routing_decisions, _} | {routing_candidate, _}}. -type misconfiguration_error() :: {misconfiguration, {routing_decisions, _} | {routing_candidate, _}}.
-export_type([route_predestination/0]). -export_type([route_predestination/0]).
-export_type([route_choice_context/0]). -export_type([route_choice_context/0]).
-export_type([fail_rated_route/0]). -export_type([fail_rated_route/0]).
-export_type([route_scores/0]).
-export_type([limits/0]).
-export_type([scores/0]).
%% %%
-spec filter_by_critical_provider_status(T) -> T when T :: hg_routing_ctx:t(). -spec filter_by_critical_provider_status(T) -> T when T :: hg_routing_ctx:t().
filter_by_critical_provider_status(Ctx) -> filter_by_critical_provider_status(Ctx0) ->
RoutesFailRates = rate_routes(hg_routing_ctx:candidates(Ctx)), RoutesFailRates = rate_routes(hg_routing_ctx:candidates(Ctx0)),
RouteScores = score_routes_map(RoutesFailRates),
Ctx1 = hg_routing_ctx:stash_route_scores(RouteScores, Ctx0),
lists:foldr( lists:foldr(
fun fun
({R, {{dead, _} = AvailabilityStatus, _ConversionStatus}}, C) -> ({R, {{dead, _} = AvailabilityStatus, _ConversionStatus}}, C) ->
@ -106,7 +100,7 @@ filter_by_critical_provider_status(Ctx) ->
({_R, _ProviderStatus}, C) -> ({_R, _ProviderStatus}, C) ->
C C
end, end,
hg_routing_ctx:with_fail_rates(RoutesFailRates, Ctx), hg_routing_ctx:with_fail_rates(RoutesFailRates, Ctx1),
RoutesFailRates RoutesFailRates
). ).
@ -317,7 +311,7 @@ select_better_route_ideal(Left, Right) ->
set_ideal_score({RouteScores, PT}) -> set_ideal_score({RouteScores, PT}) ->
{ {
RouteScores#route_scores{ RouteScores#domain_PaymentRouteScores{
availability_condition = 1, availability_condition = 1,
availability = 1.0, availability = 1.0,
conversion_condition = 1, conversion_condition = 1,
@ -367,11 +361,11 @@ format_logger_metadata(Meta, Route, Revision) when
map_route_switch_reason(SameScores, SameScores) -> map_route_switch_reason(SameScores, SameScores) ->
unknown; unknown;
map_route_switch_reason(RealScores, IdealScores) when map_route_switch_reason(RealScores, IdealScores) when
is_record(RealScores, route_scores); is_record(IdealScores, route_scores) is_record(RealScores, 'domain_PaymentRouteScores'); is_record(IdealScores, 'domain_PaymentRouteScores')
-> ->
Zipped = lists:zip(tuple_to_list(RealScores), tuple_to_list(IdealScores)), Zipped = lists:zip(tuple_to_list(RealScores), tuple_to_list(IdealScores)),
DifferenceIdx = find_idx_of_difference(Zipped), DifferenceIdx = find_idx_of_difference(Zipped),
lists:nth(DifferenceIdx, record_info(fields, route_scores)). lists:nth(DifferenceIdx, record_info(fields, 'domain_PaymentRouteScores')).
find_idx_of_difference(ZippedList) -> find_idx_of_difference(ZippedList) ->
find_idx_of_difference(ZippedList, 0). find_idx_of_difference(ZippedList, 0).
@ -438,6 +432,16 @@ calc_random_condition(StartFrom, Random, [FailRatedRoute | Rest], Routes) ->
calc_random_condition(StartFrom + Weight, Random, Rest, [{NewRoute, Status} | Routes]) calc_random_condition(StartFrom + Weight, Random, Rest, [{NewRoute, Status} | Routes])
end. end.
-spec score_routes_map([fail_rated_route()]) -> #{hg_route:payment_route() => route_scores()}.
score_routes_map(Routes) ->
lists:foldl(
fun({Route, _} = FailRatedRoute, Acc) ->
Acc#{hg_route:to_payment_route(Route) => score_route(FailRatedRoute)}
end,
#{},
Routes
).
-spec score_routes([fail_rated_route()]) -> [scored_route()]. -spec score_routes([fail_rated_route()]) -> [scored_route()].
score_routes(Routes) -> score_routes(Routes) ->
[{score_route(FailRatedRoute), Route} || {Route, _} = FailRatedRoute <- Routes]. [{score_route(FailRatedRoute), Route} || {Route, _} = FailRatedRoute <- Routes].
@ -450,11 +454,11 @@ score_route({Route, ProviderStatus}) ->
{AvailabilityStatus, ConversionStatus} = ProviderStatus, {AvailabilityStatus, ConversionStatus} = ProviderStatus,
{AvailabilityCondition, Availability} = get_availability_score(AvailabilityStatus), {AvailabilityCondition, Availability} = get_availability_score(AvailabilityStatus),
{ConversionCondition, Conversion} = get_conversion_score(ConversionStatus), {ConversionCondition, Conversion} = get_conversion_score(ConversionStatus),
#route_scores{ #domain_PaymentRouteScores{
availability_condition = AvailabilityCondition, availability_condition = AvailabilityCondition,
conversion_condition = ConversionCondition, conversion_condition = ConversionCondition,
priority_rating = PriorityRate, terminal_priority_rating = PriorityRate,
pin = PinHash, route_pin = PinHash,
random_condition = RandomCondition, random_condition = RandomCondition,
availability = Availability, availability = Availability,
conversion = Conversion conversion = Conversion
@ -737,25 +741,25 @@ getv(Name, VS, Default) ->
-spec record_comparsion_test() -> _. -spec record_comparsion_test() -> _.
record_comparsion_test() -> record_comparsion_test() ->
Bigger = { Bigger = {
#route_scores{ #domain_PaymentRouteScores{
availability_condition = 1, availability_condition = 1,
availability = 0.5, availability = 0.5,
conversion_condition = 1, conversion_condition = 1,
conversion = 0.5, conversion = 0.5,
priority_rating = 1, terminal_priority_rating = 1,
pin = 0, route_pin = 0,
random_condition = 1 random_condition = 1
}, },
{42, 42} {42, 42}
}, },
Smaller = { Smaller = {
#route_scores{ #domain_PaymentRouteScores{
availability_condition = 0, availability_condition = 0,
availability = 0.1, availability = 0.1,
conversion_condition = 1, conversion_condition = 1,
conversion = 0.5, conversion = 0.5,
priority_rating = 1, terminal_priority_rating = 1,
pin = 0, route_pin = 0,
random_condition = 1 random_condition = 1
}, },
{99, 99} {99, 99}

View File

@ -18,9 +18,15 @@
-export([process/2]). -export([process/2]).
-export([with_guard/1]). -export([with_guard/1]).
-export([pipeline/2]). -export([pipeline/2]).
-export([route_limits/1]).
-export([stash_route_limits/2]).
-export([route_scores/1]).
-export([stash_route_scores/2]).
-type rejection_group() :: atom(). -type rejection_group() :: atom().
-type error() :: {atom(), _Description}. -type error() :: {atom(), _Description}.
-type route_limits() :: hg_routing:limits().
-type route_scores() :: hg_routing:scores().
-type t() :: #{ -type t() :: #{
initial_candidates := [hg_route:t()], initial_candidates := [hg_route:t()],
@ -31,7 +37,9 @@
choosen_route := hg_route:t() | undefined, choosen_route := hg_route:t() | undefined,
choice_meta := hg_routing:route_choice_context() | undefined, choice_meta := hg_routing:route_choice_context() | undefined,
stashed_candidates => [hg_route:t()], stashed_candidates => [hg_route:t()],
fail_rates => [hg_routing:fail_rated_route()] fail_rates => [hg_routing:fail_rated_route()],
route_limits => route_limits(),
route_scores => route_scores()
}. }.
-export_type([t/0]). -export_type([t/0]).
@ -152,6 +160,24 @@ rejections(#{rejections := Rejections}) ->
%% %%
-spec route_limits(t()) -> route_limits() | undefined.
route_limits(Ctx) ->
maps:get(route_limits, Ctx, undefined).
-spec stash_route_limits(route_limits(), t()) -> t().
stash_route_limits(RouteLimits, Ctx) ->
Ctx#{route_limits => RouteLimits}.
-spec route_scores(t()) -> route_scores() | undefined.
route_scores(Ctx) ->
maps:get(route_scores, Ctx, undefined).
-spec stash_route_scores(route_scores(), t()) -> t().
stash_route_scores(RouteScores, Ctx) ->
Ctx#{route_scores => RouteScores}.
%%
latest_rejected_routes(#{latest_rejection := ReasonGroup, rejections := Rejections}) -> latest_rejected_routes(#{latest_rejection := ReasonGroup, rejections := Rejections}) ->
{ReasonGroup, maps:get(ReasonGroup, Rejections, [])}. {ReasonGroup, maps:get(ReasonGroup, Rejections, [])}.

View File

@ -0,0 +1,285 @@
-module(hg_routing_explanation).
-include_lib("damsel/include/dmsl_domain_thrift.hrl").
-include_lib("damsel/include/dmsl_payproc_thrift.hrl").
-include_lib("hellgate/include/hg_invoice_payment.hrl").
%% API
-export([get_explanation/2]).
-type st() :: #st{}.
-type explanation() :: dmsl_payproc_thrift:'InvoicePaymentExplanation'().
-type route() :: hg_route:payment_route().
-type scores() :: hg_routing:scores().
-type limits() :: hg_routing:limits().
-type route_with_context() :: #{
route := route(),
scores := hg_routing:route_scores() | undefined,
limits := [hg_limiter:turnover_limit_value()] | undefined
}.
-spec get_explanation(st(), hg_invoice_payment:opts()) -> explanation().
get_explanation(
#st{
payment = Payment,
routes = Routes,
candidate_routes = CandidateRoutes,
route_scores = RouteScores,
route_limits = RouteLimits
},
Opts
) ->
case Routes of
[] ->
%% If there's no routes even tried, then no explanation can be provided
throw(#payproc_RouteNotChosen{});
[Route | AttemptedRoutes] ->
CandidateRoutesWithoutChosenRoute = exclude_chosen_route_from_candidates(CandidateRoutes, Route),
ChosenRWC = make_route_with_context(Route, RouteScores, RouteLimits),
AttemptedExplanation = maybe_explain_attempted_routes(
AttemptedRoutes, RouteScores, RouteLimits
),
CandidatesExplanation = maybe_explain_candidate_routes(
CandidateRoutesWithoutChosenRoute, RouteScores, RouteLimits, ChosenRWC
),
Varset = gather_varset(Payment, Opts),
#payproc_InvoicePaymentExplanation{
explained_routes = lists:flatten([
route_explanation(chosen, ChosenRWC, ChosenRWC),
AttemptedExplanation,
CandidatesExplanation
]),
used_varset = Varset
}
end.
exclude_chosen_route_from_candidates(CandidateRoutes, Route) when is_list(CandidateRoutes) ->
CandidateRoutes -- [Route];
exclude_chosen_route_from_candidates(_UndefinedCandidates, _Route) ->
[].
-spec make_route_with_context(route(), scores(), limits()) -> route_with_context().
make_route_with_context(Route, RouteScores, RouteLimits) ->
#{
route => Route,
scores => hg_maybe:apply(fun(A) -> maps:get(Route, A, undefined) end, RouteScores),
limits => hg_maybe:apply(fun(A) -> maps:get(Route, A, undefined) end, RouteLimits)
}.
maybe_explain_attempted_routes([], _RouteScores, _RouteLimits) ->
[];
maybe_explain_attempted_routes([AttemptedRoute | AttemptedRoutes], RouteScores, RouteLimits) ->
RouteWithContext = make_route_with_context(AttemptedRoute, RouteScores, RouteLimits),
[
route_explanation(attempted, RouteWithContext, RouteWithContext)
| maybe_explain_attempted_routes(AttemptedRoutes, RouteScores, RouteLimits)
].
maybe_explain_candidate_routes([], _RouteScores, _RouteLimits, _ChosenRWC) ->
[];
maybe_explain_candidate_routes([CandidateRoute | CandidateRoutes], RouteScores, RouteLimits, ChosenRWC) ->
RouteWithContext = make_route_with_context(CandidateRoute, RouteScores, RouteLimits),
[
route_explanation(candidate, RouteWithContext, ChosenRWC)
| maybe_explain_candidate_routes(CandidateRoutes, RouteScores, RouteLimits, ChosenRWC)
].
route_explanation(chosen, RouteWithContext, _ChosenRoute) ->
#{
route := Route,
scores := Scores,
limits := Limits
} = RouteWithContext,
#payproc_InvoicePaymentRouteExplanation{
route = Route,
is_chosen = true,
scores = Scores,
limits = Limits,
rejection_description = <<"This route was chosen.">>
};
route_explanation(attempted, RouteWithContext, _ChosenRoute) ->
#{
route := Route,
scores := Scores,
limits := Limits
} = RouteWithContext,
#payproc_InvoicePaymentRouteExplanation{
route = Route,
is_chosen = false,
scores = Scores,
limits = Limits,
rejection_description = <<"This route was attempted, but wasn't succesfull.">>
};
route_explanation(candidate, RouteWithContext, ChosenRoute) ->
#{
route := Route,
scores := Scores,
limits := Limits
} = RouteWithContext,
#payproc_InvoicePaymentRouteExplanation{
route = Route,
is_chosen = false,
scores = Scores,
limits = Limits,
rejection_description = candidate_rejection_explanation(RouteWithContext, ChosenRoute)
}.
candidate_rejection_explanation(
#{scores := RouteScores, limits := RouteLimits},
#{scores := ChosenScores}
) when RouteScores =:= undefined; RouteLimits =:= undefined; ChosenScores =:= undefined ->
<<"Not enough information to make judgement. Payment was done before relevant changes were done.">>;
candidate_rejection_explanation(
#{scores := RouteScores, limits := RouteLimits},
#{scores := ChosenScores}
) when RouteScores =:= ChosenScores ->
IfEmpty = <<"This route has the same score as the chosen route, but wasn't chosen due to order in ruleset.">>,
check_route_limits(RouteLimits, IfEmpty);
candidate_rejection_explanation(
#{scores := RouteScores, limits := RouteLimits},
#{scores := ChosenScores}
) when RouteScores > ChosenScores ->
IfEmpty = <<"No explanation for rejection can be found. Check in with developer.">>,
check_route_limits(RouteLimits, IfEmpty);
candidate_rejection_explanation(
#{scores := RouteScores, limits := RouteLimits},
#{scores := ChosenScores}
) when RouteScores < ChosenScores ->
Explanation0 = check_route_scores(RouteScores, ChosenScores),
Explanation1 = check_route_limits(RouteLimits, <<"">>),
genlib_string:join(<<" ">>, [Explanation0, Explanation1]).
check_route_limits(RouteLimits, IfEmpty) ->
case check_route_limits(RouteLimits) of
[] ->
IfEmpty;
Result ->
genlib_string:join(<<" ">>, Result)
end.
check_route_limits([]) ->
[];
check_route_limits([TurnoverLimitValue | Rest]) ->
case TurnoverLimitValue of
#payproc_TurnoverLimitValue{
limit = #domain_TurnoverLimit{
id = LimitID,
upper_boundary = UpperBoundary
},
value = Value
} when Value > UpperBoundary ->
[
format(
"Limit with id ~p was exceeded with upper_boundary being ~p and limit value being ~p.",
[LimitID, UpperBoundary, Value]
)
| check_route_limits(Rest)
];
_ ->
check_route_limits(Rest)
end.
check_route_scores(
#domain_PaymentRouteScores{
availability_condition = 0,
availability = Av
},
_ChoseScores
) ->
AvailabilityConfig = maps:get(availability, genlib_app:env(hellgate, fault_detector, #{}), #{}),
CriticalFailRate = maps:get(critical_fail_rate, AvailabilityConfig, 0.7),
format(
"Availability reached critical level with availability of ~p, while threshold is ~p.",
[1.0 - Av, CriticalFailRate]
);
check_route_scores(
#domain_PaymentRouteScores{
conversion_condition = 0,
conversion = Cv
},
_ChoseScores
) ->
ConversionConfig = maps:get(conversion, genlib_app:env(hellgate, fault_detector, #{}), #{}),
CriticalFailRate = maps:get(critical_fail_rate, ConversionConfig, 0.7),
format(
"Conversion reached critical level with conversion of ~p, while threshold is ~p.",
[1.0 - Cv, CriticalFailRate]
);
check_route_scores(
#domain_PaymentRouteScores{
terminal_priority_rating = Rating0
},
#domain_PaymentRouteScores{
terminal_priority_rating = Rating1
}
) when Rating0 < Rating1 ->
format("Priority of this route was less chosen one, where ~p < ~p.", [Rating0, Rating1]);
check_route_scores(
#domain_PaymentRouteScores{
route_pin = Pin0
},
#domain_PaymentRouteScores{
route_pin = Pin1
}
) when Pin0 < Pin1 ->
format("Pin wasn't the same as in chosen route ~p < ~p.", [Pin0, Pin1]);
check_route_scores(
#domain_PaymentRouteScores{
random_condition = Random0
},
#domain_PaymentRouteScores{
random_condition = Random1
}
) when Random0 < Random1 ->
format("Random condition wasn't the same as in chosen route ~p < ~p.", [Random0, Random1]);
check_route_scores(
#domain_PaymentRouteScores{
availability = Av0
},
#domain_PaymentRouteScores{
availability = Av1
}
) when Av0 < Av1 ->
format("Avaliability is less than in chosen route ~p < ~p.", [Av0, Av1]);
check_route_scores(
#domain_PaymentRouteScores{
conversion = Cv0
},
#domain_PaymentRouteScores{
conversion = Cv1
}
) when Cv0 < Cv1 ->
format("Conversion is less than in chosen route ~p < ~p.", [Cv0, Cv1]).
gather_varset(Payment, Opts) ->
#domain_InvoicePayment{
cost = Cost,
payer = Payer
} = Payment,
#domain_Party{
id = PartyID
} = get_party(Opts),
#domain_Shop{
id = ShopID,
category = Category
} = get_shop(Opts),
#payproc_Varset{
category = Category,
currency = Cost#domain_Cash.currency,
amount = Cost,
shop_id = ShopID,
payment_tool = hg_invoice_payment:get_payer_payment_tool(Payer),
party_id = PartyID
}.
get_party(#{party := Party}) ->
Party.
get_shop(#{party := Party, invoice := Invoice}) ->
#domain_Invoice{shop_id = ShopID} = Invoice,
hg_party:get_shop(ShopID, Party).
format(Format, Data) ->
erlang:iolist_to_binary(io_lib:format(Format, Data)).

View File

@ -0,0 +1,11 @@
{application, routing, [
{description, "Processing protocol definitions"},
{vsn, "0"},
{registered, []},
{applications, [
kernel,
stdlib,
thrift,
damsel
]}
]}.

View File

@ -21,7 +21,7 @@
{<<"ctx">>,{pkg,<<"ctx">>,<<"0.6.0">>},2}, {<<"ctx">>,{pkg,<<"ctx">>,<<"0.6.0">>},2},
{<<"damsel">>, {<<"damsel">>,
{git,"https://github.com/valitydev/damsel.git", {git,"https://github.com/valitydev/damsel.git",
{ref,"23211ff8c0c45699fa6d78afa55f304e95dbee87"}}, {ref,"decfa45d7ce4b3c948957c6ddba34742aaa9fdc5"}},
0}, 0},
{<<"dmt_client">>, {<<"dmt_client">>,
{git,"https://github.com/valitydev/dmt-client.git", {git,"https://github.com/valitydev/dmt-client.git",