Skip to content

Commit

Permalink
propagate comments forward for no_bs_start_match2 validation errors
Browse files Browse the repository at this point in the history
(See the parent commit for some a primer on the problem and test
cases.)

This commit is refactors a fix for a `beam_validator` failure we
are currently solving by hard-coding a register. In the test case:

    parse_float2(<<".", Rest/binary>>) ->
        parse_digits2([], Rest);
    parse_float2(Bin) -> {[], Bin}.

    parse_digits2(Foo, Bin) ->
        parse_digits2(Foo, [], Bin).

    parse_digits2(Foo, Bar, Bin) ->
        parse_digits2(Foo, Bar, Bin, []).

    parse_digits2(
      Foo, Bar, <<Digit/integer, Rest/binary>>, Acc)
      when is_integer(Digit) andalso Digit >= 48 andalso Digit =< 57 ->
        parse_digits2(Foo, Bar, Rest, [Digit | Acc]);
    parse_digits2(_Foo, _Bar, Rest, Acc) ->
        {lists:reverse(Acc), Rest}.

Prior to this commit, `beam_validator` gave the following validation
error:

    {{'kfun__tx_funs__-allowed_bs_match_digit_parser2_test/0-fun-0-__33998213',
      tx_funs__parse_float2,1},
      {{call_only,2,{f,7}},10,no_bs_start_match2}}}

Which happens because `parse_float2/1` calls `parse_digits2/2`
with an argument which is annotated as accepting a match context,
but the parameter in `parse_digits2/2` is not yet annotated as
accepting a match context.

In order to fix this scenario, we would need to determine which
parameter of `parse_digits2/2` (and later which of `parse_digits2/3`
needs to accept a match context, since that information is not
available in the `beam_validator` failure message.

Lets look at the disasm for that function:

    {function,tx_funs__parse_float2,1,4,
     [{label,3},
      {line,
       [{location,
         "/home/michael/dev-tools/code/khepri/test/tx_funs.erl",
         279}]},
      {func_info,
       {atom,
        'kfun__tx_funs__-allowed_bs_match_digit_parser2_test/0-fun-0-__33998213'},
       {atom,tx_funs__parse_float2},
       1},
      {label,4},
      {'%',{var_info,{x,0},[accepts_match_context]}},
      {bs_start_match4,{atom,no_fail},1,{x,0},{x,0}},
      {test,bs_match_string,
       {f,5},
       [{x,0},8,{string,<<".">>}]},
      {move,{x,0},{x,1}},
      {move,nil,{x,0}},
      {call_only,2,{f,7}},
      {label,5},
      {bs_get_tail,{x,0},{x,0},1},
      {test_heap,3,1},
      {put_tuple2,{x,0},{list,[nil,{x,0}]}},
      return]}

We want to pull forward the annotation (5th instruction, 1-indexed)
for `{x,0}` to the definition of the function being called ({f,7}).
First we lookup the function code for the failing function by Name
and Arity (`find_comments_for_call/4`), then determine the typing
of the branch in which the call occurs: in this case by scanning the
instructions between {label,4} (the start of the branch) and
instruction 10 (the call_only; we get this index from the
`beam_validator` failure message), gathering annotations and
following {move,Src,Dst} instructions.

Once we have the annotations leading up to the call, we can add
them to right after the callee's entrypoint. The `merge_comments/2`
and `add_comments_to_code/4` functions have been refactored to
allow merging in multiple comments at once, which may be necessary
if we're propagating the typing for multiple parameters.
  • Loading branch information
the-mikedavis committed Mar 18, 2022
1 parent 2033e19 commit ec87b2d
Showing 1 changed file with 145 additions and 64 deletions.
209 changes: 145 additions & 64 deletions src/khepri_fun.erl
Original file line number Diff line number Diff line change
Expand Up @@ -80,10 +80,14 @@
to_standalone_arg/2,
handle_compilation_error/2,
handle_validation_error/3,
add_comment_and_retry/5,
add_comment_to_function/5,
add_comment_to_code/3,
add_comment_to_code/4,
add_comments_and_retry/5,
add_comments_to_function/5,
add_comments_to_code/3,
add_comments_to_code/4,
find_comments_in_branch/2,
find_comments_in_branch/4,
split_comments/1,
split_comments/2,
merge_comments/2]}).

-type fun_info() :: #{arity => arity(),
Expand Down Expand Up @@ -426,17 +430,20 @@ handle_compilation_error(Asm, Error) ->

handle_validation_error(
Asm,
{_FailingFun,
{{Call, 1 = _Arity, {f, EntryLabel}},
_,
{{_, Name, Arity},
{{Call, _Arity, {f, EntryLabel}},
CallIndex,
no_bs_start_match2}},
Error) when Call =:= call orelse Call =:= call_only ->
%% FIXME: We currently hard-code the register to annotate because we don't
%% have an immediate way to tell which one should be annotated.
Register = {x, 0},
Comment = {'%', {var_info, Register, [accepts_match_context]}},
{_GeneratedModuleName,
_Exports,
_Attributes,
Functions,
_Labels} = Asm,
#function{code = Instructions} = lookup_function1(Functions, Name, Arity),
Comments = find_comments_in_branch(Instructions, CallIndex),
Location = {'after', {label, EntryLabel}},
add_comment_and_retry(Asm, Error, EntryLabel, Location, Comment);
add_comments_and_retry(Asm, Error, EntryLabel, Location, Comments);
handle_validation_error(
Asm,
{FailingFun,
Expand All @@ -445,22 +452,91 @@ handle_validation_error(
{bad_type, {needed, NeededType}, {actual, any}}}},
Error) ->
VarInfo = {var_info, Var, [{type, NeededType}]},
Comment = {'%', VarInfo},
Comments = [{'%', VarInfo}],
Location = {before, FailingInstruction},
add_comment_and_retry(Asm, Error, FailingFun, Location, Comment);
add_comments_and_retry(Asm, Error, FailingFun, Location, Comments);
handle_validation_error(Asm, _ValidationFailure, Error) ->
throw({compilation_failure, Error, Asm}).

add_comment_and_retry(
Asm, Error, FailingFun, Location, Comment) ->
%% Looks up the comments for all variables within a branch up as they
%% appear at the given instruction index.
%%
%% For example, consider the instructions for this function:
%%
%% ...
%% {label, 4},
%% {'%', {var_info, {x, 0}, [accepts_match_context]}},
%% {bs_start_match4, {atom, no_fail}, 1, {x, 0}, {x, 0}},
%% {test, bs_match_string, {f, 5}, [{x, 0}, 8, {string, <<".">>}]},
%% {move, {x, 0}, {x, 1}},
%% {move, nil, {x, 0}},
%% {call_only, 2, {f, 7},
%% {label, 5},
%% ...
%%
%% When given these instructions and the index for the `call_only'
%% instruction, this function should return:
%%
%% [{'%', {var_info, {x, 1}, [accepts_match_context]}}]
%%
%% Notice that this comment applies to `{x, 1}' instead of `{x, 0}' as it
%% appears in the original comment, since this is the typing of the variables
%% at the time of the `call_only' instruction.
%%
%% To construct these comments, we fold through the instructions in order
%% and track the comments. There are some special-case instructions to
%% consider - `label/1' and `move/2'.
%%
%% A `label/1' instruction begins a branch within the instructions. Each
%% branch has independent typing, so any comments that appear between two
%% labels only apply to the variables within that branch. So in this function,
%% `label/1' clears any types we've gathered. In the above example
%% instructions, we only care about the typing between `{label, 4}' and
%% `{label, 5}'.
%%
%% `move/2' moves the some value from a source register or literal `Src' to
%% a destination register `Dst'. When handling a `move/2', if any comments
%% exist for `Src', we move the comments for `Src' to `Dst'. If any comments
%% exist for `Dst`, they are discarded. In the example instructions,
%% `{move, {x, 0}, {x, 1}}' moves the comment for `{x, 0}' to `{x, 1}'.
find_comments_in_branch(Instructions, Index) ->
%% `beam_validator' instruction counter is 1-indexed
find_comments_in_branch(Instructions, Index, 1, #{}).

find_comments_in_branch(
_Instructions, Index, Index, VarInfos) ->
[{'%', {var_info, Var, Info}} || {Var, Info} <- maps:to_list(VarInfos)];
find_comments_in_branch(
[{'%', {var_info, Var, Info}} | Rest], Index, Counter, VarInfos) ->
VarInfos1 = maps:put(Var, Info, VarInfos),
find_comments_in_branch(Rest, Index, Counter + 1, VarInfos1);
find_comments_in_branch(
[{move, Src, Dst} | Rest], Index, Counter, VarInfos) ->
VarInfos1 = case VarInfos of
#{Src := Info} -> maps:put(Dst, Info, VarInfos);
_ -> maps:remove(Dst, VarInfos)
end,
VarInfos2 = maps:remove(Src, VarInfos1),
find_comments_in_branch(Rest, Index, Counter + 1, VarInfos2);
find_comments_in_branch(
[{label, _Label} | Rest], Index, Counter, _VarInfos) ->
%% Each branch (separated by labels) has independent typing
find_comments_in_branch(Rest, Index, Counter + 1, #{});
find_comments_in_branch(
[_Instruction | Rest], Index, Counter, VarInfos) ->
find_comments_in_branch(
Rest, Index, Counter + 1, VarInfos).

add_comments_and_retry(
Asm, Error, FailingFun, Location, Comments) ->
{GeneratedModuleName,
Exports,
Attributes,
Functions,
Labels} = Asm,
try
Functions1 = add_comment_to_function(
Functions, FailingFun, Location, Comment, []),
Functions1 = add_comments_to_function(
Functions, FailingFun, Location, Comments, []),
Asm1 = {GeneratedModuleName,
Exports,
Attributes,
Expand All @@ -472,59 +548,64 @@ add_comment_and_retry(
throw({compilation_failure, Error, Asm})
end.

add_comment_to_function(
add_comments_to_function(
[#function{name = Name, arity = Arity, code = Code} = Function | Rest],
{_GeneratedModuleName, Name, Arity},
Location, Comment, Result) ->
Code1 = add_comment_to_code(Code, Location, Comment),
Location, Comments, Result) ->
Code1 = add_comments_to_code(Code, Location, Comments),
Function1 = Function#function{code = Code1},
lists:reverse(Result) ++ [Function1 | Rest];
add_comment_to_function(
add_comments_to_function(
[#function{entry = EntryLabel, code = Code} = Function | Rest],
EntryLabel, Location, Comment, Result) ->
Code1 = add_comment_to_code(Code, Location, Comment),
EntryLabel, Location, Comments, Result) ->
Code1 = add_comments_to_code(Code, Location, Comments),
Function1 = Function#function{code = Code1},
lists:reverse(Result) ++ [Function1 | Rest];
add_comment_to_function(
[Function | Rest], FailingFun, Location, Comment, Result) ->
add_comment_to_function(
Rest, FailingFun, Location, Comment, [Function | Result]).

add_comment_to_code(Code, Location, Comment) ->
add_comment_to_code(Code, Location, Comment, []).

add_comment_to_code(
[Instruction | Rest], {before, Instruction},
{'%', {var_info, Var, _}} = Comment, Result) ->
case Result of
[{'%', {var_info, Var, _}} = ExistingComment | Result1] ->
Comment1 = merge_comments(Comment, ExistingComment),
lists:reverse(Result1) ++ [Comment1, Instruction | Rest];
_ ->
lists:reverse(Result) ++ [Comment, Instruction | Rest]
end;
add_comment_to_code(
[Instruction | Rest], {'after', Instruction},
{'%', {var_info, Var, _}} = Comment, Result) ->
case Rest of
[{'%', {var_info, Var, _}} = ExistingComment | Rest1] ->
Comment1 = merge_comments(Comment, ExistingComment),
lists:reverse(Result) ++ [Instruction, Comment1 | Rest1];
_ ->
lists:reverse(Result) ++ [Instruction, Comment | Rest]
end;
add_comment_to_code(
[Instruction | Rest], Location, Comment, Result) ->
add_comment_to_code(Rest, Location, Comment, [Instruction | Result]).

merge_comments(Comment, Comment) ->
throw(duplicate_annotations);
merge_comments(
{'%', {var_info, Var, Attributes1}},
{'%', {var_info, Var, Attributes2}}) ->
{'%', {var_info, Var, Attributes1 ++ Attributes2}};
merge_comments(Comment, _Instruction) ->
Comment.
add_comments_to_function(
[Function | Rest], FailingFun, Location, Comments, Result) ->
add_comments_to_function(
Rest, FailingFun, Location, Comments, [Function | Result]).

add_comments_to_code(Code, Location, Comments) ->
add_comments_to_code(Code, Location, Comments, []).

add_comments_to_code(
[Instruction | Rest], {before, Instruction}, Comments, Result) ->
{ExistingComments, Result1} = split_comments(Result),
Comments1 = merge_comments(Comments, ExistingComments),
lists:reverse(Result1) ++ Comments1 ++ [Instruction | Rest];
add_comments_to_code(
[Instruction | Rest], {'after', Instruction}, Comments, Result) ->
{ExistingComments, Rest1} = split_comments(Rest),
Comments1 = merge_comments(Comments, ExistingComments),
lists:reverse(Result) ++ [Instruction | Comments1] ++ Rest1;
add_comments_to_code(
[Instruction | Rest], Location, Comments, Result) ->
add_comments_to_code(Rest, Location, Comments, [Instruction | Result]).

split_comments(Instructions) ->
split_comments(Instructions, []).

split_comments([{'%', _} = Comment | Rest], Comments) ->
split_comments(Rest, [Comment | Comments]);
split_comments(Rest, Comments) ->
{lists:reverse(Comments), Rest}.

merge_comments(Comments, ExistingComments) ->
ExistingCommentsMap = maps:from_list(
[{Var, Info} ||
{'%', {var_info, Var, Info}} <-
ExistingComments]),
lists:map(fun({'%', {var_info, Var, Info}} = Annotation) ->
case ExistingCommentsMap of
#{Var := Info} ->
throw(duplicate_annotations);
#{Var := ExistingInfo} ->
{'%', {var_info, Var, Info ++ ExistingInfo}};
_ ->
Annotation
end
end, Comments).

-spec exec(StandaloneFun, Args) -> Ret when
StandaloneFun :: standalone_fun(),
Expand Down

0 comments on commit ec87b2d

Please sign in to comment.