[sledge] Add concurrency analysis

Summary:
This diff changes the analysis exploration algorithm from considering
only sequential executions to considering executions of the
interleaving semantics. As part of this, symbolic states are changed
so that each thread has its own registers, while all memory is shared
between them.

Currently only a simple threads interface is supported: they can be
created with `thread_create(&thread_routine)`, they can exit by
returning from `thread_routine`, and they can be joined with
`thread_join`. Current simplifications include that newly created
threads are already runnable, thread routines accept no arguments and
return no result, and no failures are possible.

The concurrent exploration algorithm gives preference to executions
which have fewer context switches, thereby performing an incremental
form of context-bounded analysis.

A form of partial-order reduction is performed, where the symbolic
states are joined across (prefixes of) executions with the same number
of context switches which reach a point where the instruction pointers
and call stacks of all threads are the same. This has the effect of
"dagifying" the concurrent execution tree by merging points after
e.g. threads perform actions that commute with each other. This is
unlike traditional partial-order reduction in that it relies on the
symbolic join to combine the results of commuting operations in a way
that the following symbolic execution can take advantage of, rather
than performing some up-front analysis to identify commuting
operations and quotienting the space of executions. The current state
of the symbolic join and execution is significantly suboptimal in this
regard.

Differential Revision: D29441149

fbshipit-source-id: cf801a6b1
master
Josh Berdine 3 years ago committed by Facebook GitHub Bot
parent 347887eebd
commit 268015fbf3

@ -160,8 +160,10 @@ let assign reg exp q =
|>
[%Trace.retn fun {pf} r -> pf "{%a}" pp r]
let resolve_int _ _ _ = []
(** block if [e] is known to be false; skip otherwise *)
let exec_assume q e =
let exec_assume _ q e =
match apron_texpr_of_llair_exp e q with
| Some e ->
let cond =
@ -171,14 +173,14 @@ let exec_assume q e =
| _ -> Some q
(** existentially quantify killed register [r] out of state [q] *)
let exec_kill r q =
let exec_kill _ r q =
let apron_v = apron_var_of_reg r in
if Environment.mem_var (Abstract1.env q) apron_v then
Abstract1.forget_array (Lazy.force man) q [|apron_v|] false
else q
(** perform a series [move_vec] of reg:=exp moves at state [q] *)
let exec_move move_vec q =
let exec_move _ move_vec q =
let defs, uses =
IArray.fold move_vec (Llair.Reg.Set.empty, Llair.Reg.Set.empty)
~f:(fun (r, e) (defs, uses) ->
@ -189,17 +191,17 @@ let exec_move move_vec q =
todo "overwritten variables in Domain_itv" () ;
IArray.fold ~f:(fun (r, e) q -> assign r e q) move_vec q
let exec_inst i q =
let exec_inst tid i q =
match (i : Llair.inst) with
| Move {reg_exps; loc= _} -> Ok (exec_move reg_exps q)
| Move {reg_exps; loc= _} -> Ok (exec_move tid reg_exps q)
| Store {ptr; exp; len= _; loc= _} -> (
match Llair.Reg.of_exp ptr with
| Some reg -> Ok (assign reg exp q)
| None -> Ok q )
| Load {reg; ptr; len= _; loc= _} -> Ok (assign reg ptr q)
| Nondet {reg= Some reg; msg= _; loc= _} -> Ok (exec_kill reg q)
| Nondet {reg= Some reg; msg= _; loc= _} -> Ok (exec_kill tid reg q)
| Nondet {reg= None; msg= _; loc= _} | Alloc _ | Free _ -> Ok q
| Intrinsic {reg= Some reg; _} -> Ok (exec_kill reg q)
| Intrinsic {reg= Some reg; _} -> Ok (exec_kill tid reg q)
| Intrinsic {reg= None; _} -> Ok q
| Abort {loc} ->
Error
@ -208,13 +210,15 @@ let exec_inst i q =
; pp_action= Fun.flip Llair.Inst.pp i
; pp_state= Fun.flip pp q }
let enter_scope _ _ q = q
type from_call = {areturn: Llair.Reg.t option; caller_q: t}
[@@deriving sexp_of]
let recursion_beyond_bound = `prune
(** existentially quantify locals *)
let post locals _ (q : t) =
let post _ locals _ (q : t) =
let locals =
Llair.Reg.Set.fold locals [] ~f:(fun r a ->
let v = apron_var_of_reg r in
@ -224,7 +228,7 @@ let post locals _ (q : t) =
Abstract1.forget_array (Lazy.force man) q locals false
(** drop caller-local variables, add returned value to caller state *)
let retn _ freturn {areturn; caller_q} callee_q =
let retn tid _ freturn {areturn; caller_q} callee_q =
match (areturn, freturn) with
| Some aret, Some fret ->
let env_fret_only =
@ -245,14 +249,14 @@ let retn _ freturn {areturn; caller_q} callee_q =
Abstract1.rename_array man result
[|apron_var_of_reg fret|]
[|apron_var_of_reg aret|]
| Some aret, None -> exec_kill aret caller_q
| Some aret, None -> exec_kill tid aret caller_q
| None, _ -> caller_q
(** map actuals to formals (via temporary registers), stash constraints on
caller-local variables. Note that this exploits the non-relational-ness
of Box to ignore all variables other than the formal/actual params/
returns; this will not be possible if extended to a relational domain *)
let call ~summaries ~globals:_ ~actuals ~areturn ~formals ~freturn:_
let call ~summaries _ ~globals:_ ~actuals ~areturn ~formals ~freturn:_
~locals:_ q =
if summaries then
todo "Summaries not yet implemented for interval analysis" ()
@ -285,10 +289,10 @@ let call ~summaries ~globals:_ ~actuals ~areturn ~formals ~freturn:_
(q''', {areturn; caller_q= q})
let dnf q = [q]
let resolve_callee _ _ _ = []
let resolve_callee _ _ _ _ = []
type summary = t
let pp_summary = pp
let apply_summary _ _ = None
let create_summary ~locals:_ ~formals:_ q = (q, q)
let create_summary _ ~locals:_ ~formals:_ q = (q, q)

@ -29,6 +29,31 @@ __attribute__((noreturn)) void __llair_unreachable();
/* macos version */
#define __assert_rtn(function, file, line, assertion) abort()
/*
* threads
*/
typedef int thread_t;
typedef void (*thread_create_routine)();
thread_t sledge_thread_create(thread_create_routine entry);
void sledge_thread_join(thread_t thread);
typedef uint32_t error_t;
error_t thread_create(thread_t** t, thread_create_routine entry) {
thread_t child = sledge_thread_create(entry);
**t = child;
return 0;
}
error_t thread_join(thread_t* t) {
sledge_thread_join(*t);
return 0;
}
#ifdef __cplusplus
}
#endif

@ -38,10 +38,13 @@ module type QueueS = sig
val add : elt -> t -> t
(** add an element *)
val pop : t -> (elt * elt list * t) option
(** [pop q] is [None] if [q] is empty and otherwise is [Some (e, es, q')]
where [e] is the selected element in [q], any elements [es] have the
same destination as [e], and [q'] is [q] without [e] and [es]. *)
val top : t -> (elt * elt list * t) option
(** [top q] is [None] if [q] is empty and otherwise is [Some (e, es, q')]
where [e] is the selected element in [q] and any elements [es] have
the same destination as [e]. [q'] is equivalent to [q] but possibly
more compactly represented. *)
val remove : elt -> elt list -> t -> t
end
(** Type of a queue implementation, which is parameterized over elements. *)
@ -71,18 +74,26 @@ module PriorityQueue (Elt : Elt) : QueueS with type elt = Elt.t = struct
if removed' == removed then {queue= FHeap.add queue elt; removed}
else {queue; removed= removed'}
let rec pop {queue; removed} =
let* top, queue = FHeap.pop queue in
let removed' = Elts.remove top removed in
if removed' != removed then pop {queue; removed= removed'}
let rec top {queue; removed} =
let* next = FHeap.top queue in
let removed' = Elts.remove next removed in
if removed' != removed then
let queue' = FHeap.remove_top_exn queue in
top {queue= queue'; removed= removed'}
else
let elts, removed =
FHeap.fold queue ~init:([], removed') ~f:(fun (elts, removed) elt ->
if Elt.equal_destination top elt && not (Elts.mem elt removed)
then (elt :: elts, Elts.add elt removed)
else (elts, removed) )
let elts =
FHeap.fold queue ~init:[] ~f:(fun elts elt ->
if Elt.equal_destination next elt && not (Elts.mem elt removed)
then elt :: elts
else elts )
in
Some (top, elts, {queue; removed})
Some (next, elts, {queue; removed})
let remove top elts {queue; removed} =
assert (Elt.equal top (FHeap.top_exn queue)) ;
let queue = FHeap.remove_top_exn queue in
let removed = Elts.add_list elts removed in
{queue; removed}
end
module RandomQueue (Elt : Elt) : QueueS with type elt = Elt.t = struct
@ -190,7 +201,7 @@ module RandomQueue (Elt : Elt) : QueueS with type elt = Elt.t = struct
/. Float.of_int elt_weight
; last= Add_or_pop_frontier }
let pop q =
let _pop q =
let num_recent = RAL.length q.recent in
if num_recent > 0 then
let elt, recent =
@ -230,13 +241,15 @@ module RandomQueue (Elt : Elt) : QueueS with type elt = Elt.t = struct
~finish:(fun _ ->
assert (M.is_empty q.frontier) ;
None )
let top _ = todo "concurrent sampling analysis" ()
let remove _ = todo "concurrent sampling analysis" ()
end
module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
module Stack : sig
type t
val pp : t pp
val empty : t
val push_call : Llair.func Llair.call -> D.from_call -> t -> t
val pop_return : t -> (D.from_call * Llair.jump * t) option
@ -341,43 +354,150 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
let equal_as_inlined_location = [%compare.equal: as_inlined_location]
end
(** Instruction Pointer. Functions are treated as-if-inlined by including
a call stack in each instruction pointer, effectively copying the
control-flow graph for each calling context. *)
type ip = {ip: Llair.IP.t; stk: Stack.t}
(** Instruction Pointer, of a single thread so includes thread id.
Functions are treated as-if-inlined by including a call stack in each
instruction pointer, effectively copying the control-flow graph for
each calling context. *)
type ip = {ip: Llair.IP.t; stk: Stack.t; tid: ThreadID.t}
(** Instruction Pointer *)
(** Instruction Pointer of a single thread *)
module IP : sig
type t = ip [@@deriving compare, equal, sexp_of]
val pp : t pp
end = struct
type t = ip = {ip: Llair.IP.t; stk: Stack.as_inlined_location}
type t = ip =
{ip: Llair.IP.t; stk: Stack.as_inlined_location; tid: ThreadID.t}
[@@deriving compare, equal, sexp_of]
let pp ppf {ip} = Llair.IP.pp ppf ip
end
(** A control-flow transition. An edge from block [src] to
[dst = {ip; stk}] represents a transition with call stack [stk] from
(the terminator of) block [src] to the instruction pointer [ip]. *)
type edge = {dst: IP.t; src: Llair.Block.t} [@@deriving sexp_of]
(** Representation of a single thread, including identity and scheduling
state *)
module Thread : sig
type t = Runnable of IP.t | Terminated of ThreadID.t
[@@deriving equal, sexp_of]
val compare : t Ord.t
val compare_without_tid : t Ord.t
val pp : t pp
val id : t -> ThreadID.t
end = struct
(** Because [ip] needs to include [tid], this is represented as a sum of
products, but it may be more natural to think in terms of the
isomorphic representation using a product of a sum such as
[(Runnable of ... | Terminated ...) * ThreadID.t]. *)
type t = Runnable of IP.t | Terminated of ThreadID.t
[@@deriving sexp_of]
let pp ppf = function
| Runnable ip -> IP.pp ppf ip
| Terminated tid -> Format.fprintf ppf "T%i" tid
let id = function Runnable {tid} -> tid | Terminated tid -> tid
(* Note: Threads.inactive relies on comparing tid last *)
let compare_aux compare_tid x y =
let open Ord.Infix in
if x == y then 0
else
match (x, y) with
| Runnable x, Runnable y ->
Llair.IP.compare x.ip y.ip
<?> (Stack.compare_as_inlined_location, x.stk, y.stk)
<?> (compare_tid, x.tid, y.tid)
| Runnable _, _ -> -1
| _, Runnable _ -> 1
| Terminated x_tid, Terminated y_tid -> compare_tid x_tid y_tid
let compare = compare_aux ThreadID.compare
let equal = [%compare.equal: t]
let compare_without_tid = compare_aux (fun _ _ -> 0)
end
(** Set of threads *)
module Threads : sig
type t [@@deriving compare, equal, sexp_of]
type inactive [@@deriving sexp_of]
val compare_inactive : inactive Ord.t
val compare_inactive_tids : inactive Ord.t
val init : t
val create : Llair.block -> t -> ThreadID.t * t
val after_step : Thread.t -> t -> t * inactive
val join : ThreadID.t -> t -> t option
val fold : t -> 's -> f:(Thread.t -> 's -> 's) -> 's
end = struct
module M = Map.Make (ThreadID)
type t = Thread.t M.t [@@deriving compare, equal, sexp_of]
type inactive = Thread.t array [@@deriving sexp_of]
let compare_inactive = Ord.array Thread.compare_without_tid
let compare_inactive_tids = Ord.(array (ThreadID.compare >|= Thread.id))
let inactive active_id threads =
let a = Iter.to_array (M.values (M.remove active_id threads)) in
Array.sort ~cmp:Thread.compare a ;
a
let init = M.empty
let fold threads s ~f =
M.fold threads s ~f:(fun ~key:_ ~data s -> f data s)
let create entry threads =
let ip = Llair.IP.mk entry in
let max_tid =
match M.max_binding threads with
| Some (tid, _) -> tid
| None -> ThreadID.init
in
let tid = max_tid + 1 in
let thread = Thread.Runnable {ip; stk= Stack.empty; tid} in
(tid, M.add ~key:tid ~data:thread threads)
let after_step active threads =
let tid = Thread.id active in
let inactive = inactive tid threads in
let threads = M.add ~key:tid ~data:active threads in
(threads, inactive)
let join tid threads =
match M.find tid threads with
| Some (Thread.Terminated _) -> Some (M.remove tid threads)
| _ -> None
end
(** A control-flow transition of a single thread. In the common case an
edge from block [src] to [dst = Runnable {ip; stk; tid}] represents a
transition by thread [tid] with call stack [stk] from (usually the
terminator of) block [src] to the instruction pointer [ip]. If a
scheduling point is encountered within a block, the represented
transition need not originate from the terminator of [src]. Edges can
also represent transitions that produce threads in non-[Runnable]
scheduling states, determined by the form of [dst]. *)
type edge = {dst: Thread.t; src: Llair.Block.t} [@@deriving sexp_of]
module Edge = struct
type t = edge [@@deriving sexp_of]
let pp fs {dst; src= {sort_index; lbl}} =
Format.fprintf fs "%a <-- #%i %%%s" IP.pp dst sort_index lbl
Format.fprintf fs "%a <-t%i- #%i %%%s" Thread.pp dst (Thread.id dst)
sort_index lbl
(** Each retreating edge has a depth for each calling context, except
for recursive calls. Recursive call edges are instead compared
without considering their stacks. Bounding the depth of edges
therefore has the effect of bounding the number of recursive calls
in any calling context. *)
let compare x y =
let compare_aux compare_tid x y =
let open Ord.Infix in
if x == y then 0
else
match (x, y) with
| {dst= Runnable x_t}, {dst= Runnable y_t} ->
let is_rec_call = function
| {Llair.term= Call {recursive= true}} -> true
| _ -> false
@ -386,11 +506,18 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
if is_rec_call x.src then 0
else Stack.compare_as_inlined_location stk1 stk2
in
Llair.IP.compare x.dst.ip y.dst.ip
Llair.IP.compare x_t.ip y_t.ip
<?> (Llair.Block.compare, x.src, y.src)
<?> (compare_stk, x.dst.stk, y.dst.stk)
<?> (compare_stk, x_t.stk, y_t.stk)
<?> (compare_tid, x_t.tid, y_t.tid)
| {dst= Runnable _}, _ -> -1
| _, {dst= Runnable _} -> 1
| {dst= Terminated x_tid}, {dst= Terminated y_tid} ->
Llair.Block.compare x.src y.src <?> (compare_tid, x_tid, y_tid)
let compare = compare_aux ThreadID.compare
let equal = [%compare.equal: t]
let compare_without_tid = compare_aux (fun _ _ -> 0)
end
module Depths = struct
@ -408,19 +535,24 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
| `Both (d1, d2) -> Some (Int.max d1 d2) )
end
type switches = int [@@deriving compare, equal, sexp_of]
(** Abstract memory, control, and history state, with a slot used for the
current "control position", such as an instruction pointer. Consists
of a symbolic [state], plus a coarse abstraction of the preceding
execution history in the form of [depths] representing the number of
times retreating edges have been crossed. *)
of a symbolic [state] and a scheduling state of the [threads], plus a
coarse abstraction of the preceding execution history in the form of
the number of context [switches] and [depths] representing the number
of times retreating edges have been crossed. *)
type 'a memory_control_history =
{ ctrl: 'a (** current control position *)
; state: D.t (** symbolic memory and register state *)
; threads: Threads.t (** scheduling state of the threads *)
; switches: switches (** count of preceding context switches *)
; depths: Depths.t (** count of retreating edge crossings *) }
[@@deriving sexp_of]
(** An abstract machine state consists of the instruction pointer plus the
memory, control, and history state. *)
(** An abstract machine state consists of the instruction pointer of the
active thread plus the memory, control, and history state. *)
type ams = IP.t memory_control_history [@@deriving sexp_of]
(** A unit of analysis work is an abstract machine state from which
@ -429,14 +561,19 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
type work = edge memory_control_history
(** An element of the frontier of execution is a control-flow [edge] that
has been executed, yielding a memory, control, and history state. *)
has been executed by the thread [Thread.id edge.dst], yielding a
memory, control, and history state. The [threads] indicates the
scheduling state of the point after the transition indicated by the
edge. *)
type elt = elt_ctrl memory_control_history [@@deriving sexp_of]
and elt_ctrl =
{ edge: Edge.t
; depth: int
(** pre-computed depth of [edge], for use by e.g. [Elt.compare] *)
}
; inactive: Threads.inactive
(** pre-computed summary of inactive thread scheduling states, for
use by e.g. [Elt.compare] *) }
module Work : sig
type t
@ -450,30 +587,49 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
module Elt = struct
type t = elt [@@deriving sexp_of]
let pp ppf {ctrl= {edge; depth}} =
Format.fprintf ppf "%i: %a" depth Edge.pp edge
let pp ppf {ctrl= {edge; depth}; switches} =
Format.fprintf ppf "%i,%i: %a" switches depth Edge.pp edge
let compare x y =
let open Ord.Infix in
if x == y then 0
else
( (Int.compare >|= fun x -> x.ctrl.depth)
@? (Edge.compare >|= fun x -> x.ctrl.edge)
( (Int.compare >|= fun x -> x.switches)
@? (Int.compare >|= fun x -> x.ctrl.depth)
@? (Edge.compare_without_tid >|= fun x -> x.ctrl.edge)
@? (Threads.compare_inactive >|= fun x -> x.ctrl.inactive)
@? (ThreadID.compare >|= fun x -> Thread.id x.ctrl.edge.dst)
@? (Threads.compare_inactive_tids >|= fun x -> x.ctrl.inactive)
@? (Depths.compare >|= fun x -> x.depths)
@? (D.compare >|= fun x -> x.state) )
x y
let equal = [%compare.equal: t]
let equal_destination x y = IP.equal x.ctrl.edge.dst y.ctrl.edge.dst
let equal_destination x y = Threads.equal x.threads y.threads
let dnf x = List.map ~f:(fun state -> {x with state}) (D.dnf x.state)
end
module Queue = Queue (Elt)
(** State and history projection of abstract machine states.
[StateHistory] represents the subset of [ams] fields that can be
joined across several executions. *)
module StateHistory = struct
(** Concurrent state and history projection of abstract machine states.
Abstract machine states with the same [switches], [ip], and
[threads] fields have, as far as the scheduler is concerned, the
same concurrent state and history and can be joined. *)
module ConcSH = struct
module T = struct
type t = switches * IP.t * Threads.t
[@@deriving compare, equal, sexp_of]
end
include T
module Map = Map.Make (T)
end
(** Sequential state and history projection of abstract machine states.
Complementary to [ConcSH], [SeqSH] represents the subset of [ams]
fields that can be joined across several executions that share the
same abstract concurrent state and history. *)
module SeqSH = struct
module T = struct
type t = D.t * Depths.t [@@deriving compare, equal, sexp_of]
end
@ -494,29 +650,62 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
(D.joinN states, depths)
end
(** Sequential states indexed by concurrent states. When sequential
states and histories are joined across executions that reach the
same abstract concurrent state and history, there are multiple
successor executions corresponding to which thread is selected to
execute. Executing some such successors can lead to additional
executions that reach the original abstract concurrent state and
history. These new executions also need to be joined with the old
ones. To handle this, the successors of a join are enumerated
lazily, returning them one-by-one from the scheduler and adding them
to the analysis worklist. The "cursor" that maintains the current
progress of this enumeration is a set of sequential states that is
indexed by concurrent states. *)
module Cursor = struct
type t = SeqSH.Set.t ConcSH.Map.t
let empty = ConcSH.Map.empty
let add = ConcSH.Map.add
let find = ConcSH.Map.find
end
(** Analysis exploration state *)
type t = Queue.t
type t = Queue.t * Cursor.t
let prune depth {ctrl= edge} wl =
[%Trace.info " %i: %a" depth Edge.pp edge] ;
Report.hit_bound Config.bound ;
wl
let enqueue depth ({ctrl= edge; depths} as elt) queue =
[%Trace.info " %i: %a@ | %a" depth Edge.pp edge Queue.pp queue] ;
let enqueue depth ({ctrl= {dst} as edge; threads; depths} as elt)
(queue, cursor) =
[%Trace.info
" %i,%i: %a@ | %a" elt.switches depth Edge.pp edge Queue.pp queue] ;
let depths = Depths.add ~key:edge ~data:depth depths in
let queue = Queue.add {elt with ctrl= {edge; depth}; depths} queue in
let threads, inactive = Threads.after_step dst threads in
let queue =
Queue.add
{elt with ctrl= {edge; depth; inactive}; threads; depths}
queue
in
(queue, cursor)
let init state curr =
let depth = 0 in
let ip = Llair.IP.mk curr in
let stk = Stack.empty in
let prev = curr in
let edge = {dst= {ip; stk}; src= prev} in
let tid = ThreadID.init in
let edge = {dst= Runnable {ip; stk; tid}; src= prev} in
let threads = Threads.init in
let switches = 0 in
let depths = Depths.empty in
let queue = Queue.create () in
enqueue depth {ctrl= edge; state; depths} queue
let cursor = Cursor.empty in
enqueue depth
{ctrl= edge; state; threads; switches; depths}
(queue, cursor)
let add ~retreating ({ctrl= edge; depths} as elt) wl =
let depth = Option.value (Depths.find edge depths) ~default:0 in
@ -524,21 +713,73 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
if depth > Config.bound && Config.bound >= 0 then prune depth elt wl
else enqueue depth elt wl
let dequeue queue =
let+ ({ctrl= {edge= {dst}}; state; depths} as top), elts, queue =
Queue.pop queue
module Succs = struct
module M = ConcSH.Map
let empty = M.empty
let add = M.add_multi
let find_first m ~f =
let exception Stop in
let found = ref None in
let hit_end = ref true in
( try
M.iteri m ~f:(fun ~key ~data ->
match !found with
| None -> (
match f ~key ~data with
| None -> ()
| Some r -> found := Some r )
| Some _ ->
hit_end := false ;
raise_notrace Stop )
with Stop -> () ) ;
(!found, !hit_end)
end
let rec dequeue (queue, cursor) =
let* ({threads} as top), elts, queue = Queue.top queue in
let succs =
List.fold (top :: elts) Succs.empty ~f:(fun incoming succs ->
let {ctrl= {edge= {dst}}; state; switches; depths} = incoming in
let incoming_tid = Thread.id dst in
Threads.fold threads succs ~f:(fun active succs ->
match active with
| Terminated _ -> succs
| Runnable ({tid} as ip) ->
let switches =
if tid = incoming_tid then switches else switches + 1
in
[%Trace.info
" %i: %a [%a]@ | %a" top.ctrl.depth Edge.pp top.ctrl.edge Stack.pp
dst.stk Queue.pp queue] ;
let state, depths =
StateHistory.join
(List.fold
~f:(fun {state; depths} -> StateHistory.Set.add (state, depths))
elts
(StateHistory.Set.of_ (state, depths)))
Succs.add ~key:(switches, ip, threads)
~data:(state, depths) succs ) )
in
let found, hit_end =
Succs.find_first succs
~f:(fun ~key:(switches, ip, threads) ~data:incoming ->
let next = (switches, ip, threads) in
let curr = SeqSH.Set.of_list incoming in
let+ next_states =
match Cursor.find next cursor with
| Some already_done ->
let next_states = SeqSH.Set.diff curr already_done in
if SeqSH.Set.is_empty next_states then None
else Some next_states
| None -> Some curr
in
( next
, next_states
, Cursor.add ~key:next ~data:next_states cursor ) )
in
({ctrl= dst; state; depths}, queue)
let queue = if hit_end then Queue.remove top elts queue else queue in
match found with
| Some ((switches, ip, threads), next_states, cursor) ->
[%Trace.info
" %i,%i: %a@ | %a" switches top.ctrl.depth Edge.pp top.ctrl.edge
Queue.pp queue] ;
let state, depths = SeqSH.join next_states in
Some
({ctrl= ip; state; threads; switches; depths}, (queue, cursor))
| None -> dequeue (queue, cursor)
let rec run ~f wl =
match dequeue wl with
@ -556,30 +797,31 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
(List.pp "@," D.pp_summary)
data ) )]
let exec_jump jump ({ctrl= {ip; stk}} as ams) wl =
let exec_jump jump ({ctrl= {ip; stk; tid}} as ams) wl =
let src = Llair.IP.block ip in
let {Llair.dst; retreating} = jump in
let ip = Llair.IP.mk dst in
let edge = {dst= {ip; stk}; src} in
let edge = {dst= Runnable {ip; stk; tid}; src} in
Work.add ~retreating {ams with ctrl= edge} wl
let exec_skip_func areturn return ({ctrl= {ip}; state} as ams) wl =
let exec_skip_func areturn return ({ctrl= {ip; tid}; state} as ams) wl =
Report.unknown_call (Llair.IP.block ip).term ;
let state = Option.fold ~f:D.exec_kill areturn state in
let state = Option.fold ~f:(D.exec_kill tid) areturn state in
exec_jump return {ams with state} wl
let exec_call globals call ({ctrl= {stk}; state} as ams) wl =
let exec_call globals call ({ctrl= {stk; tid}; state} as ams) wl =
let Llair.{callee; actuals; areturn; return; recursive} = call in
let Llair.{name; formals; freturn; locals; entry} = callee in
[%Trace.call fun {pf} ->
pf " @[<2>@ %a from %a with state@]@;<1 2>%a" Llair.Func.pp_call call
Llair.Function.pp return.dst.parent.name D.pp state]
pf " t%i@[<2>@ %a from %a with state@]@;<1 2>%a" tid
Llair.Func.pp_call call Llair.Function.pp return.dst.parent.name
D.pp state]
;
let dnf_states =
if Config.function_summaries then D.dnf state else [state]
in
let domain_call =
D.call ~globals ~actuals ~areturn ~formals ~freturn ~locals
D.call tid ~globals ~actuals ~areturn ~formals ~freturn ~locals
in
List.fold dnf_states wl ~f:(fun state wl ->
match
@ -596,7 +838,7 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
let ip = Llair.IP.mk entry in
let stk = Stack.push_call call from_call stk in
let src = Llair.IP.block ams.ctrl.ip in
let edge = {dst= {ip; stk}; src} in
let edge = {dst= Runnable {ip; stk; tid}; src} in
Work.add ~retreating:recursive {ams with ctrl= edge; state} wl
| Some post -> exec_jump return {ams with state= post} wl )
|>
@ -610,17 +852,17 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
let globals = Domain_used_globals.by_function Config.globals name in
exec_call globals call ams wl
let exec_return exp ({ctrl= {ip; stk}; state} as ams) wl =
let exec_return exp ({ctrl= {ip; stk; tid}; state} as ams) wl =
let block = Llair.IP.block ip in
let func = block.parent in
let Llair.{name; formals; freturn; locals} = func in
[%Trace.call fun {pf} -> pf " @ from: %a" Llair.Function.pp name]
[%Trace.call fun {pf} -> pf " t%i@ from: %a" tid Llair.Function.pp name]
;
let summarize post_state =
if not Config.function_summaries then post_state
else
let function_summary, post_state =
D.create_summary ~locals ~formals post_state
D.create_summary tid ~locals ~formals post_state
in
Llair.Function.Tbl.add_multi ~key:name ~data:function_summary
summary_table ;
@ -631,40 +873,45 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
let exit_state =
match (freturn, exp) with
| Some freturn, Some return_val ->
D.exec_move (IArray.of_ (freturn, return_val)) pre_state
D.exec_move tid (IArray.of_ (freturn, return_val)) pre_state
| None, None -> pre_state
| _ -> violates Llair.Func.invariant func
in
( match Stack.pop_return stk with
| Some (from_call, retn_site, stk) ->
let post_state = summarize (D.post locals from_call exit_state) in
let retn_state = D.retn formals freturn from_call post_state in
let post_state =
summarize (D.post tid locals from_call exit_state)
in
let retn_state = D.retn tid formals freturn from_call post_state in
exec_jump retn_site
{ams with ctrl= {ams.ctrl with stk}; state= retn_state}
wl
| None ->
if Config.function_summaries then
summarize exit_state |> (ignore : D.t -> unit) ;
Work.add ~retreating:false
{ams with ctrl= {dst= Terminated tid; src= block}}
wl )
|>
[%Trace.retn fun {pf} _ -> pf ""]
let exec_throw exc ({ctrl= {ip; stk}; state} as ams) wl =
let exec_throw exc ({ctrl= {ip; stk; tid}; state} as ams) wl =
let func = (Llair.IP.block ip).parent in
let Llair.{name; formals; freturn; fthrow; locals} = func in
[%Trace.call fun {pf} -> pf "@ from %a" Llair.Function.pp name]
;
let unwind formals scope from_call state =
D.retn formals (Some fthrow) from_call (D.post scope from_call state)
D.retn tid formals (Some fthrow) from_call
(D.post tid scope from_call state)
in
let pre_state = state in
( match Stack.pop_throw stk ~unwind pre_state with
| Some (from_call, retn_site, stk, unwind_state) ->
let exit_state =
D.exec_move (IArray.of_ (fthrow, exc)) unwind_state
D.exec_move tid (IArray.of_ (fthrow, exc)) unwind_state
in
let post_state = D.post locals from_call exit_state in
let retn_state = D.retn formals freturn from_call post_state in
let post_state = D.post tid locals from_call exit_state in
let retn_state = D.retn tid formals freturn from_call post_state in
exec_jump retn_site
{ams with ctrl= {ams.ctrl with stk}; state= retn_state}
wl
@ -672,21 +919,39 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
|>
[%Trace.retn fun {pf} _ -> pf ""]
let exec_assume cond jump ({state} as ams) wl =
match D.exec_assume state cond with
let exec_assume cond jump ({ctrl= {tid}; state} as ams) wl =
match D.exec_assume tid state cond with
| Some state -> exec_jump jump {ams with state} wl
| None ->
[%Trace.info " infeasible %a@\n@[%a@]" Llair.Exp.pp cond D.pp state] ;
wl
let resolve_callee (pgm : Llair.program) callee state =
let exec_thread_create reg {Llair.entry; locals} return
({ctrl= {tid}; state; threads} as ams) wl =
let child_tid, threads = Threads.create entry threads in
let child =
Llair.Exp.integer (Llair.Reg.typ reg) (Z.of_int child_tid)
in
let state = D.exec_move tid (IArray.of_ (reg, child)) state in
let state = D.enter_scope child_tid locals state in
exec_jump return {ams with state; threads} wl
let exec_thread_join thread return ({ctrl= {tid}; state; threads} as ams)
wl =
List.fold (D.resolve_int tid state thread) wl ~f:(fun join_tid wl ->
match Threads.join join_tid threads with
| Some threads -> exec_jump return {ams with threads} wl
| None -> wl )
let resolve_callee (pgm : Llair.program) tid callee state =
let lookup name = Llair.Func.find name pgm.functions in
D.resolve_callee lookup callee state
D.resolve_callee lookup tid callee state
let exec_term pgm ({ctrl= {ip}; state} as ams) wl =
let exec_term pgm ({ctrl= {ip; tid}; state} as ams) wl =
let block = Llair.IP.block ip in
let term = block.term in
[%Trace.info " @\n@[%a@]@\n%a" D.pp state Llair.Term.pp block.term] ;
[%Trace.info
" t%i@\n@[%a@]@\n%a" tid D.pp state Llair.Term.pp block.term] ;
Report.step_term block ;
match (term : Llair.term) with
| Switch {key; tbl; els} ->
@ -706,9 +971,21 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
~parent:(Llair.Function.name jump.dst.parent.name)
~name:jump.dst.lbl))
jump ams wl )
| Call call -> exec_call call ams wl
| Call ({callee; actuals; areturn; return} as call) -> (
match
(Llair.Function.name callee.name, IArray.to_array actuals, areturn)
with
| "sledge_thread_create", [|callee|], Some reg -> (
match resolve_callee pgm tid callee state with
| [] -> exec_skip_func areturn return ams wl
| callees ->
List.fold callees wl ~f:(fun callee wl ->
exec_thread_create reg callee return ams wl ) )
| "sledge_thread_join", [|thread|], None ->
exec_thread_join thread return ams wl
| _ -> exec_call call ams wl )
| ICall ({callee; areturn; return} as call) -> (
match resolve_callee pgm callee state with
match resolve_callee pgm tid callee state with
| [] -> exec_skip_func areturn return ams wl
| callees ->
List.fold callees wl ~f:(fun callee wl ->
@ -717,15 +994,19 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
| Throw {exc} -> exec_throw exc ams wl
| Unreachable -> wl
let rec exec_ip pgm ({ctrl= {ip}; state} as ams) wl =
let rec exec_ip pgm ({ctrl= {ip; stk; tid}; state} as ams) wl =
match Llair.IP.inst ip with
| Some inst -> (
[%Trace.info " @\n@[%a@]@\n%a" D.pp state Llair.Inst.pp inst] ;
[%Trace.info " t%i@\n@[%a@]@\n%a" tid D.pp state Llair.Inst.pp inst] ;
Report.step_inst ip ;
match D.exec_inst inst state with
match D.exec_inst tid inst state with
| Ok state ->
let ip = Llair.IP.succ ip in
exec_ip pgm {ams with ctrl= {ams.ctrl with ip}; state} wl
if Llair.IP.is_schedule_point ip then
let src = Llair.IP.block ip in
let edge = {dst= Runnable {ip; stk; tid}; src} in
Work.add ~retreating:false {ams with ctrl= edge; state} wl
else exec_ip pgm {ams with ctrl= {ams.ctrl with ip}; state} wl
| Error alarm ->
Report.alarm alarm ;
wl )
@ -742,8 +1023,8 @@ module Make (Config : Config) (D : Domain) (Queue : Queue) = struct
let actuals = IArray.empty in
let areturn = None in
let state, _ =
D.call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals
(D.init pgm.globals)
D.call ThreadID.init ~summaries ~globals ~actuals ~areturn ~formals
~freturn ~locals (D.init pgm.globals)
in
Work.init state entry

@ -14,15 +14,18 @@ module type Domain = sig
val join : t -> t -> t
val joinN : t list -> t
val dnf : t -> t list
val exec_assume : t -> Llair.Exp.t -> t option
val exec_kill : Llair.Reg.t -> t -> t
val exec_move : (Llair.Reg.t * Llair.Exp.t) iarray -> t -> t
val exec_inst : Llair.inst -> t -> t Or_alarm.t
val exec_assume : ThreadID.t -> t -> Llair.Exp.t -> t option
val exec_kill : ThreadID.t -> Llair.Reg.t -> t -> t
val exec_move : ThreadID.t -> (Llair.Reg.t * Llair.Exp.t) iarray -> t -> t
val exec_inst : ThreadID.t -> Llair.inst -> t -> t Or_alarm.t
val resolve_int : ThreadID.t -> t -> Llair.Exp.t -> int list
val enter_scope : ThreadID.t -> Llair.Reg.Set.t -> t -> t
type from_call [@@deriving sexp_of]
val call :
summaries:bool
-> ThreadID.t
-> globals:Llair.Global.Set.t
-> actuals:Llair.Exp.t iarray
-> areturn:Llair.Reg.t option
@ -32,11 +35,22 @@ module type Domain = sig
-> t
-> t * from_call
val post : Llair.Reg.Set.t -> from_call -> t -> t
val retn : Llair.Reg.t iarray -> Llair.Reg.t option -> from_call -> t -> t
val post : ThreadID.t -> Llair.Reg.Set.t -> from_call -> t -> t
val retn :
ThreadID.t
-> Llair.Reg.t iarray
-> Llair.Reg.t option
-> from_call
-> t
-> t
val resolve_callee :
(string -> Llair.func option) -> Llair.Exp.t -> t -> Llair.func list
(string -> Llair.func option)
-> ThreadID.t
-> Llair.Exp.t
-> t
-> Llair.func list
val recursion_beyond_bound : [`skip | `prune]
@ -45,7 +59,11 @@ module type Domain = sig
val pp_summary : summary pp
val create_summary :
locals:Llair.Reg.Set.t -> formals:Llair.Reg.t iarray -> t -> summary * t
ThreadID.t
-> locals:Llair.Reg.Set.t
-> formals:Llair.Reg.t iarray
-> t
-> summary * t
val apply_summary : t -> summary -> t option
end

@ -14,7 +14,8 @@ module type State_domain_sig = sig
include Domain
val create_summary :
locals:Llair.Reg.Set.t
ThreadID.t
-> locals:Llair.Reg.Set.t
-> formals:Llair.Reg.t iarray
-> entry:t
-> current:t
@ -43,29 +44,34 @@ module Make (State_domain : State_domain_sig) = struct
let entrys, currents = List.split rs in
(State_domain.joinN entrys, State_domain.joinN currents)
let exec_assume (entry, current) cnd =
let+ next = State_domain.exec_assume current cnd in
let resolve_int tid (_, current) = State_domain.resolve_int tid current
let exec_assume tid (entry, current) cnd =
let+ next = State_domain.exec_assume tid current cnd in
(entry, next)
let exec_kill reg (entry, current) =
(entry, State_domain.exec_kill reg current)
let exec_kill tid reg (entry, current) =
(entry, State_domain.exec_kill tid reg current)
let exec_move reg_exps (entry, current) =
(entry, State_domain.exec_move reg_exps current)
let exec_move tid reg_exps (entry, current) =
(entry, State_domain.exec_move tid reg_exps current)
let exec_inst inst (entry, current) =
let exec_inst tid inst (entry, current) =
let open Or_alarm.Import in
let+ next = State_domain.exec_inst inst current in
let+ next = State_domain.exec_inst tid inst current in
(entry, next)
let enter_scope tid regs (entry, current) =
(entry, State_domain.enter_scope tid regs current)
type from_call =
{state_from_call: State_domain.from_call; caller_entry: State_domain.t}
[@@deriving sexp_of]
let recursion_beyond_bound = State_domain.recursion_beyond_bound
let call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals
(entry, current) =
let call ~summaries tid ~globals ~actuals ~areturn ~formals ~freturn
~locals (entry, current) =
[%Trace.call fun {pf} ->
pf
"@ @[<v>@[actuals: (@[%a@])@ formals: (@[%a@])@]@ locals: \
@ -77,7 +83,7 @@ module Make (State_domain : State_domain_sig) = struct
State_domain.pp current]
;
let caller_current, state_from_call =
State_domain.call ~summaries ~globals ~actuals ~areturn ~formals
State_domain.call tid ~summaries ~globals ~actuals ~areturn ~formals
~freturn ~locals current
in
( (caller_current, caller_current)
@ -85,33 +91,35 @@ module Make (State_domain : State_domain_sig) = struct
|>
[%Trace.retn fun {pf} (reln, _) -> pf "@,%a" pp reln]
let post locals {state_from_call; caller_entry} (_, current) =
let post tid locals {state_from_call; caller_entry} (_, current) =
[%Trace.call fun {pf} -> pf "@ locals: %a" Llair.Reg.Set.pp locals]
;
(caller_entry, State_domain.post locals state_from_call current)
(caller_entry, State_domain.post tid locals state_from_call current)
|>
[%Trace.retn fun {pf} -> pf "%a" pp]
let retn formals freturn {caller_entry; state_from_call} (_, current) =
let retn tid formals freturn {caller_entry; state_from_call} (_, current)
=
[%Trace.call fun {pf} -> pf "@ %a" State_domain.pp current]
;
(caller_entry, State_domain.retn formals freturn state_from_call current)
( caller_entry
, State_domain.retn tid formals freturn state_from_call current )
|>
[%Trace.retn fun {pf} -> pf "%a" pp]
let dnf (entry, current) =
List.map ~f:(fun c -> (entry, c)) (State_domain.dnf current)
let resolve_callee f e (_, current) =
State_domain.resolve_callee f e current
let resolve_callee f tid e (_, current) =
State_domain.resolve_callee f tid e current
type summary = State_domain.summary
let pp_summary = State_domain.pp_summary
let create_summary ~locals ~formals (entry, current) =
let create_summary tid ~locals ~formals (entry, current) =
let fs, next =
State_domain.create_summary ~locals ~formals ~entry ~current
State_domain.create_summary tid ~locals ~formals ~entry ~current
in
(fs, (entry, next))

@ -14,7 +14,8 @@ module type State_domain_sig = sig
include Domain
val create_summary :
locals:Llair.Reg.Set.t
ThreadID.t
-> locals:Llair.Reg.Set.t
-> formals:Llair.Reg.t iarray
-> entry:t
-> current:t

@ -29,7 +29,7 @@ let init globals =
| _ -> violates Llair.GlobalDefn.invariant global
in
let len = Term.integer (Z.of_int siz) in
let cnt = X.term seq in
let cnt = X.term ThreadID.init seq in
Sh.star q (Sh.seg {loc; bas= loc; len; siz= len; cnt})
| _ -> q )
@ -49,18 +49,24 @@ let joinN qs =
let dnf = Sh.dnf
let exec_assume q b =
Exec.assume q (X.formula b)
let resolve_int tid q e =
match Term.get_z (Context.normalize q.Sh.ctx (X.term tid e)) with
| Some z -> [Z.to_int z]
| None -> []
let exec_assume tid q b =
Exec.assume q (X.formula tid b)
|> simplify
|> fun q -> if Sh.is_unsat_dnf q then None else Some q
let exec_kill r q = Exec.kill q (X.reg r) |> simplify
let exec_kill tid r q = Exec.kill q (X.reg tid r) |> simplify
let exec_move res q =
Exec.move q (IArray.map res ~f:(fun (r, e) -> (X.reg r, X.term e)))
let exec_move tid res q =
Exec.move q
(IArray.map res ~f:(fun (r, e) -> (X.reg tid r, X.term tid e)))
|> simplify
let exec_inst inst pre =
let exec_inst tid inst pre =
let alarm kind =
{ Alarm.kind
; loc= Llair.Inst.loc inst
@ -75,24 +81,33 @@ let exec_inst inst pre =
| Move {reg_exps; _} ->
Ok
(Exec.move pre
(IArray.map reg_exps ~f:(fun (r, e) -> (X.reg r, X.term e))))
(IArray.map reg_exps ~f:(fun (r, e) ->
(X.reg tid r, X.term tid e) )))
| Load {reg; ptr; len; _} ->
Exec.load pre ~reg:(X.reg reg) ~ptr:(X.term ptr) ~len:(X.term len)
Exec.load pre ~reg:(X.reg tid reg) ~ptr:(X.term tid ptr)
~len:(X.term tid len)
|> or_alarm
| Store {ptr; exp; len; _} ->
Exec.store pre ~ptr:(X.term ptr) ~exp:(X.term exp) ~len:(X.term len)
Exec.store pre ~ptr:(X.term tid ptr) ~exp:(X.term tid exp)
~len:(X.term tid len)
|> or_alarm
| Alloc {reg; num; len; _} ->
Exec.alloc pre ~reg:(X.reg reg) ~num:(X.term num) ~len |> or_alarm
| Free {ptr; _} -> Exec.free pre ~ptr:(X.term ptr) |> or_alarm
| Nondet {reg; _} -> Ok (Exec.nondet pre (Option.map ~f:X.reg reg))
Exec.alloc pre ~reg:(X.reg tid reg) ~num:(X.term tid num) ~len
|> or_alarm
| Free {ptr; _} -> Exec.free pre ~ptr:(X.term tid ptr) |> or_alarm
| Nondet {reg; _} -> Ok (Exec.nondet pre (Option.map ~f:(X.reg tid) reg))
| Abort _ -> Error (alarm Abort)
| Intrinsic {reg; name; args; _} ->
let areturn = Option.map ~f:X.reg reg in
let actuals = IArray.map ~f:X.term args in
let areturn = Option.map ~f:(X.reg tid) reg in
let actuals = IArray.map ~f:(X.term tid) args in
Exec.intrinsic pre areturn name actuals |> or_alarm )
|> Or_alarm.map ~f:simplify
let enter_scope tid regs q =
let vars = X.regs tid regs in
assert (Var.Set.disjoint vars q.Sh.us) ;
Sh.extend_us vars q
let value_determined_by ctx us a =
List.exists (Context.class_of ctx a) ~f:(fun b ->
Term.Set.subset (Term.Set.of_iter (Term.atoms b)) ~of_:us )
@ -127,10 +142,13 @@ let and_eqs sub formals actuals q =
in
IArray.fold2_exn ~f:and_eq formals actuals q
let localize_entry globals actuals formals freturn locals shadow pre entry =
let localize_entry tid globals actuals formals freturn locals shadow pre
entry =
(* Add the formals here to do garbage collection and then get rid of them *)
let formals_set = Var.Set.of_iter (IArray.to_iter formals) in
let freturn_locals = X.regs (Llair.Reg.Set.add_option freturn locals) in
let freturn_locals =
X.regs tid (Llair.Reg.Set.add_option freturn locals)
in
let wrt =
Term.Set.of_iter
(Iter.append
@ -157,7 +175,8 @@ type from_call = {areturn: Var.t option; unshadow: Var.Subst.t; frame: Sh.t}
(** Express formula in terms of formals instead of actuals, and enter scope
of locals: rename formals to fresh vars in formula and actuals, add
equations between each formal and actual, and quantify fresh vars. *)
let call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals q =
let call ~summaries tid ~globals ~actuals ~areturn ~formals ~freturn ~locals
q =
[%Trace.call fun {pf} ->
pf "@ @[<hv>locals: {@[%a@]}@ globals: {@[%a@]}@ q: %a@]"
Llair.Reg.Set.pp locals Llair.Global.Set.pp globals pp q ;
@ -166,17 +185,19 @@ let call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals q =
let fv_actuals =
actuals
|> IArray.to_iter
|> Iter.map ~f:X.term
|> Iter.map ~f:(X.term tid)
|> Iter.flat_map ~f:Term.vars
in
not
(Option.exists areturn ~f:(fun modif ->
Iter.exists ~f:(Var.equal (X.reg modif)) fv_actuals )) )]
Iter.exists ~f:(Var.equal (X.reg tid modif)) fv_actuals )) )]
;
let actuals = IArray.map ~f:X.term actuals in
let areturn = Option.map ~f:X.reg areturn in
let formals = IArray.map ~f:X.reg formals in
let freturn_locals = X.regs (Llair.Reg.Set.add_option freturn locals) in
let actuals = IArray.map ~f:(X.term tid) actuals in
let areturn = Option.map ~f:(X.reg tid) areturn in
let formals = IArray.map ~f:(X.reg tid) formals in
let freturn_locals =
X.regs tid (Llair.Reg.Set.add_option freturn locals)
in
let modifs = Var.Set.of_option areturn in
(* quantify modifs, their current values will be overwritten and so should
not be saved and restored on return *)
@ -197,7 +218,8 @@ let call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals q =
( if not summaries then (entry, {areturn; unshadow; frame= Sh.emp})
else
let q, frame =
localize_entry globals actuals formals freturn locals shadow q entry
localize_entry tid globals actuals formals freturn locals shadow q
entry
in
(q, {areturn; unshadow; frame}) )
|>
@ -206,18 +228,18 @@ let call ~summaries ~globals ~actuals ~areturn ~formals ~freturn ~locals q =
frame pp entry]
(** Leave scope of locals: existentially quantify locals. *)
let post locals _ q =
let post tid locals _ q =
[%Trace.call fun {pf} ->
pf "@ @[<hv>locals: {@[%a@]}@ q: %a@]" Llair.Reg.Set.pp locals Sh.pp q]
;
Sh.exists (X.regs locals) q |> simplify
Sh.exists (X.regs tid locals) q |> simplify
|>
[%Trace.retn fun {pf} -> pf "%a" Sh.pp]
(** Express in terms of actuals instead of formals: existentially quantify
formals, and apply inverse of fresh variables for formals renaming to
restore the shadowed variables. *)
let retn formals freturn {areturn; unshadow; frame} q =
let retn tid formals freturn {areturn; unshadow; frame} q =
[%Trace.call fun {pf} ->
pf "@ @[<v>formals: {@[%a@]}%a%a@ unshadow: %a@ q: %a@ frame: %a@]"
(IArray.pp ", " Llair.Reg.pp)
@ -228,9 +250,9 @@ let retn formals freturn {areturn; unshadow; frame} q =
areturn Var.Subst.pp unshadow pp q pp frame]
;
let formals =
Var.Set.of_iter (Iter.map ~f:X.reg (IArray.to_iter formals))
Var.Set.of_iter (Iter.map ~f:(X.reg tid) (IArray.to_iter formals))
in
let freturn = Option.map ~f:X.reg freturn in
let freturn = Option.map ~f:(X.reg tid) freturn in
let q =
match areturn with
| Some areturn -> (
@ -259,8 +281,8 @@ let retn formals freturn {areturn; unshadow; frame} q =
|>
[%Trace.retn fun {pf} -> pf "%a" pp]
let resolve_callee lookup ptr (q : Sh.t) =
Context.class_of q.ctx (X.term ptr)
let resolve_callee lookup tid ptr (q : Sh.t) =
Context.class_of q.ctx (X.term tid ptr)
|> List.find_map ~f:(X.lookup_func lookup)
|> Option.to_list
@ -272,16 +294,16 @@ let pp_summary fs {xs; foot; post} =
Format.fprintf fs "@[<v>xs: @[%a@]@ foot: %a@ post: %a @]" Var.Set.pp xs
pp foot pp post
let create_summary ~locals ~formals ~entry ~current:(post : Sh.t) =
let create_summary tid ~locals ~formals ~entry ~current:(post : Sh.t) =
[%Trace.call fun {pf} ->
pf "@ formals %a@ entry: %a@ current: %a"
(IArray.pp ",@ " Llair.Reg.pp)
formals pp entry pp post]
;
let formals =
Var.Set.of_iter (Iter.map ~f:X.reg (IArray.to_iter formals))
Var.Set.of_iter (Iter.map ~f:(X.reg tid) (IArray.to_iter formals))
in
let locals = X.regs locals in
let locals = X.regs tid locals in
let foot = Sh.exists locals entry in
let foot, subst = Sh.freshen ~wrt:(Var.Set.union foot.us post.us) foot in
let restore_formals q =

@ -11,7 +11,8 @@ open Domain_intf
include Domain
val create_summary :
locals:Llair.Reg.Set.t
ThreadID.t
-> locals:Llair.Reg.Set.t
-> formals:Llair.Reg.t iarray
-> entry:t
-> current:t

@ -13,25 +13,27 @@ let pp fs () = Format.pp_print_string fs "()"
let init _ = ()
let join () () = ()
let joinN _ = ()
let exec_assume () _ = Some ()
let exec_kill _ () = ()
let exec_move _ () = ()
let exec_inst _ () = Ok ()
let resolve_int _ _ _ = []
let exec_assume _ () _ = Some ()
let exec_kill _ _ () = ()
let exec_move _ _ () = ()
let exec_inst _ _ () = Ok ()
let enter_scope _ _ () = ()
type from_call = unit [@@deriving compare, equal, sexp]
let call ~summaries:_ ~globals:_ ~actuals:_ ~areturn:_ ~formals:_ ~freturn:_
~locals:_ _ =
let call ~summaries:_ _ ~globals:_ ~actuals:_ ~areturn:_ ~formals:_
~freturn:_ ~locals:_ _ =
((), ())
let recursion_beyond_bound = `skip
let post _ _ () = ()
let retn _ _ _ _ = ()
let post _ _ _ () = ()
let retn _ _ _ _ _ = ()
let dnf () = [()]
let resolve_callee _ _ _ = []
let resolve_callee _ _ _ _ = []
type summary = unit
let pp_summary fs () = Format.pp_print_string fs "()"
let create_summary ~locals:_ ~formals:_ _ = ((), ())
let create_summary _ ~locals:_ ~formals:_ _ = ((), ())
let apply_summary _ _ = Some ()

@ -19,9 +19,10 @@ let init globals =
let join l r = Llair.Global.Set.union l r
let joinN = function [] -> empty | x :: xs -> List.fold ~f:join xs x
let enter_scope _ _ state = state
let recursion_beyond_bound = `skip
let post _ _ state = state
let retn _ _ from_call post = Llair.Global.Set.union from_call post
let post _ _ _ state = state
let retn _ _ _ from_call post = Llair.Global.Set.union from_call post
let dnf t = [t]
let used_globals exp s =
@ -30,13 +31,14 @@ let used_globals exp s =
| Some g -> Llair.Global.Set.add g s
| None -> s )
let exec_assume st exp = Some (used_globals exp st)
let exec_kill _ st = st
let resolve_int _ _ _ = []
let exec_assume _ st exp = Some (used_globals exp st)
let exec_kill _ _ st = st
let exec_move reg_exps st =
let exec_move _ reg_exps st =
IArray.fold ~f:(fun (_, rhs) -> used_globals rhs) reg_exps st
let exec_inst inst st =
let exec_inst _ inst st =
[%Trace.call fun {pf} -> pf "@ pre:{%a} %a" pp st Llair.Inst.pp inst]
;
Ok (Llair.Inst.fold_exps ~f:used_globals inst st)
@ -47,18 +49,18 @@ let exec_inst inst st =
type from_call = t [@@deriving sexp]
(* Set abstract state to bottom (i.e. empty set) at function entry *)
let call ~summaries:_ ~globals:_ ~actuals ~areturn:_ ~formals:_ ~freturn:_
let call ~summaries:_ _ ~globals:_ ~actuals ~areturn:_ ~formals:_ ~freturn:_
~locals:_ st =
(empty, IArray.fold ~f:used_globals actuals st)
let resolve_callee _ _ _ = []
let resolve_callee _ _ _ _ = []
(* A function summary is the set of global registers accessed by that
function and its transitive callees *)
type summary = t
let pp_summary = pp
let create_summary ~locals:_ ~formals:_ state = (state, state)
let create_summary _ ~locals:_ ~formals:_ state = (state, state)
let apply_summary st summ = Some (Llair.Global.Set.union st summ)
(** Query *)

@ -550,6 +550,27 @@ module IP = struct
let block ip = ip.block
let is_schedule_point ip =
match inst ip with
| Some (Load _ | Store _ | Free _) -> true
| Some (Move _ | Alloc _ | Nondet _ | Abort _) -> false
| Some (Intrinsic {name; _}) -> (
match name with
| `calloc | `malloc | `mallocx | `nallocx -> false
| `_ZN5folly13usingJEMallocEv | `aligned_alloc | `dallocx | `mallctl
|`mallctlbymib | `mallctlnametomib | `malloc_stats_print
|`malloc_usable_size | `memcpy | `memmove | `memset
|`posix_memalign | `rallocx | `realloc | `sallocx | `sdallocx
|`strlen | `xallocx ->
true )
| None -> (
match ip.block.term with
| Call {callee; _} -> (
match Function.name callee.name with
| "sledge_thread_join" -> true
| _ -> false )
| _ -> false )
let pp ppf {block; index} =
Format.fprintf ppf "#%i%t %%%s" block.sort_index
(fun ppf -> if index <> 0 then Format.fprintf ppf "+%i" index)

@ -204,6 +204,7 @@ module IP : sig
val block : t -> block
val inst : t -> inst option
val succ : t -> t
val is_schedule_point : t -> bool
module Tbl : HashTable.S with type key := t
end

@ -10,7 +10,7 @@ module T = Term
module F = Formula
module RegTbl = HashTable.Make (struct
type t = Llair.Reg.t [@@deriving equal, hash]
type t = Llair.Reg.t * ThreadID.t [@@deriving equal, hash]
end)
let reg_tbl : Var.t RegTbl.t = RegTbl.create ()
@ -23,62 +23,66 @@ let lookup_func lookup term =
let uconst name = T.apply (Funsym.uninterp name) [||]
let global g = uconst (Llair.Global.name g)
let reg r =
RegTbl.find_or_add reg_tbl r ~default:(fun () ->
let reg tid r =
RegTbl.find_or_add reg_tbl (r, tid) ~default:(fun () ->
let name = Llair.Reg.name r in
let id = Llair.Reg.id r in
let id = 1 + RegTbl.length reg_tbl in
Var.identified ~name ~id )
let regs rs = Var.Set.of_iter (Iter.map ~f:reg (Llair.Reg.Set.to_iter rs))
let regs tid rs =
Var.Set.of_iter (Iter.map ~f:(reg tid) (Llair.Reg.Set.to_iter rs))
let uap0 f = T.apply f [||]
let uap1 f a = T.apply f [|a|]
let uap2 f a b = T.apply f [|a; b|]
let lit2 p a b = F.lit p [|a; b|]
let nlit2 p a b = F.not_ (lit2 p a b)
let rec ap_ttt : 'a. (T.t -> T.t -> 'a) -> _ -> _ -> 'a =
fun f a b -> f (term a) (term b)
and ap_ttf (f : T.t -> T.t -> F.t) a b = F.inject (ap_ttt f a b)
let rec ap_ttt : 'a. (T.t -> T.t -> 'a) -> _ -> _ -> _ -> 'a =
fun f tid a b -> f (term tid a) (term tid b)
and ap_ttf (f : T.t -> T.t -> F.t) tid a b = F.inject (ap_ttt f tid a b)
and ap_fff (f : F.t -> F.t -> F.t) a b =
F.inject (f (formula a) (formula b))
and ap_fff (f : F.t -> F.t -> F.t) tid a b =
F.inject (f (formula tid a) (formula tid b))
and ap_uut : 'a. (T.t -> T.t -> 'a) -> _ -> _ -> _ -> 'a =
fun f typ a b ->
and ap_uut : 'a. (T.t -> T.t -> 'a) -> _ -> _ -> _ -> _ -> 'a =
fun f typ tid a b ->
let bits = Llair.Typ.bit_size_of typ in
let unsigned x = uap1 (Unsigned bits) x in
f (unsigned (term a)) (unsigned (term b))
f (unsigned (term tid a)) (unsigned (term tid b))
and ap_uuf (f : T.t -> T.t -> F.t) typ a b = F.inject (ap_uut f typ a b)
and ap_uuf (f : T.t -> T.t -> F.t) typ tid a b =
F.inject (ap_uut f typ tid a b)
and term : Llair.Exp.t -> T.t =
fun e ->
and term : ThreadID.t -> Llair.Exp.t -> T.t =
fun tid e ->
let imp p q = F.or_ (F.not_ p) q in
let nimp p q = F.and_ p (F.not_ q) in
let if_ p q = F.or_ p (F.not_ q) in
let nif p q = F.and_ (F.not_ p) q in
match e with
(* formulas *)
| Ap2 (Eq, Integer {bits= 1; _}, p, q) -> ap_fff F.iff p q
| Ap2 (Dq, Integer {bits= 1; _}, p, q) -> ap_fff F.xor p q
| Ap2 ((Gt | Ugt), Integer {bits= 1; _}, p, q) -> ap_fff nimp p q
| Ap2 ((Lt | Ult), Integer {bits= 1; _}, p, q) -> ap_fff nif p q
| Ap2 ((Ge | Uge), Integer {bits= 1; _}, p, q) -> ap_fff if_ p q
| Ap2 ((Le | Ule), Integer {bits= 1; _}, p, q) -> ap_fff imp p q
| Ap2 (Add, Integer {bits= 1; _}, p, q) -> ap_fff F.xor p q
| Ap2 (Sub, Integer {bits= 1; _}, p, q) -> ap_fff F.xor p q
| Ap2 (Mul, Integer {bits= 1; _}, p, q) -> ap_fff F.and_ p q
| Ap2 (Eq, Integer {bits= 1; _}, p, q) -> ap_fff F.iff tid p q
| Ap2 (Dq, Integer {bits= 1; _}, p, q) -> ap_fff F.xor tid p q
| Ap2 ((Gt | Ugt), Integer {bits= 1; _}, p, q) -> ap_fff nimp tid p q
| Ap2 ((Lt | Ult), Integer {bits= 1; _}, p, q) -> ap_fff nif tid p q
| Ap2 ((Ge | Uge), Integer {bits= 1; _}, p, q) -> ap_fff if_ tid p q
| Ap2 ((Le | Ule), Integer {bits= 1; _}, p, q) -> ap_fff imp tid p q
| Ap2 (Add, Integer {bits= 1; _}, p, q) -> ap_fff F.xor tid p q
| Ap2 (Sub, Integer {bits= 1; _}, p, q) -> ap_fff F.xor tid p q
| Ap2 (Mul, Integer {bits= 1; _}, p, q) -> ap_fff F.and_ tid p q
(* div and rem are not formulas even if bits=1 due to division by 0 *)
| Ap2 (And, Integer {bits= 1; _}, p, q) -> ap_fff F.and_ p q
| Ap2 (Or, Integer {bits= 1; _}, p, q) -> ap_fff F.or_ p q
| Ap2 (Xor, Integer {bits= 1; _}, p, q) -> ap_fff F.xor p q
| Ap2 ((Shl | Lshr), Integer {bits= 1; _}, p, q) -> ap_fff nimp p q
| Ap2 (Ashr, Integer {bits= 1; _}, p, q) -> ap_fff F.or_ p q
| Ap2 (And, Integer {bits= 1; _}, p, q) -> ap_fff F.and_ tid p q
| Ap2 (Or, Integer {bits= 1; _}, p, q) -> ap_fff F.or_ tid p q
| Ap2 (Xor, Integer {bits= 1; _}, p, q) -> ap_fff F.xor tid p q
| Ap2 ((Shl | Lshr), Integer {bits= 1; _}, p, q) -> ap_fff nimp tid p q
| Ap2 (Ashr, Integer {bits= 1; _}, p, q) -> ap_fff F.or_ tid p q
| Ap3 (Conditional, Integer {bits= 1; _}, cnd, pos, neg) ->
F.inject
(F.cond ~cnd:(formula cnd) ~pos:(formula pos) ~neg:(formula neg))
(F.cond ~cnd:(formula tid cnd) ~pos:(formula tid pos)
~neg:(formula tid neg))
(* terms *)
| Reg _ -> T.var (reg (Llair.Reg.of_exp e |> Option.get_exn))
| Reg _ -> T.var (reg tid (Llair.Reg.of_exp e |> Option.get_exn))
| Global {name; typ= _} | Function {name; typ= _} -> uconst name
| Label {parent; name} ->
uap0 (Funsym.uninterp ("label_" ^ parent ^ "_" ^ name))
@ -89,58 +93,59 @@ and term : Llair.Exp.t -> T.t =
| _ | (exception Invalid_argument _) ->
uap0 (Funsym.uninterp ("float_" ^ data)) )
| Ap1 (Signed {bits}, _, e) ->
let a = term e in
let a = term tid e in
if bits = 1 then
match F.project a with
| Some fml -> F.inject fml
| _ -> uap1 (Signed bits) a
else uap1 (Signed bits) a
| Ap1 (Unsigned {bits}, _, e) ->
let a = term e in
let a = term tid e in
if bits = 1 then
match F.project a with
| Some fml -> F.inject fml
| _ -> uap1 (Unsigned bits) a
else uap1 (Unsigned bits) a
| Ap1 (Convert {src}, dst, e) when Llair.Typ.equivalent src dst -> term e
| Ap1 (Convert {src= Float _}, Float _, e) -> term e
| Ap1 (Convert {src}, dst, e) when Llair.Typ.equivalent src dst ->
term tid e
| Ap1 (Convert {src= Float _}, Float _, e) -> term tid e
| Ap1 (Convert {src}, dst, e) ->
let s =
Format.asprintf "convert_%a_of_%a" Llair.Typ.pp dst Llair.Typ.pp src
in
uap1 (Funsym.uninterp s) (term e)
| Ap2 (Eq, _, d, e) -> ap_ttf F.eq d e
| Ap2 (Dq, _, d, e) -> ap_ttf F.dq d e
| Ap2 (Gt, _, d, e) -> ap_ttf F.gt d e
| Ap2 (Lt, _, d, e) -> ap_ttf F.lt d e
| Ap2 (Ge, _, d, e) -> ap_ttf F.ge d e
| Ap2 (Le, _, d, e) -> ap_ttf F.le d e
| Ap2 (Ugt, typ, d, e) -> ap_uuf F.gt typ d e
| Ap2 (Ult, typ, d, e) -> ap_uuf F.lt typ d e
| Ap2 (Uge, typ, d, e) -> ap_uuf F.ge typ d e
| Ap2 (Ule, typ, d, e) -> ap_uuf F.le typ d e
| Ap2 (Ord, _, d, e) -> ap_ttf (lit2 (Predsym.uninterp "ord")) d e
| Ap2 (Uno, _, d, e) -> ap_ttf (nlit2 (Predsym.uninterp "ord")) d e
| Ap2 (Add, _, d, e) -> ap_ttt T.add d e
| Ap2 (Sub, _, d, e) -> ap_ttt T.sub d e
| Ap2 (Mul, _, d, e) -> ap_ttt T.mul d e
| Ap2 (Div, _, d, e) -> ap_ttt T.div d e
| Ap2 (Rem, _, d, e) -> ap_ttt (uap2 Rem) d e
| Ap2 (Udiv, typ, d, e) -> ap_uut T.div typ d e
| Ap2 (Urem, typ, d, e) -> ap_uut (uap2 Rem) typ d e
| Ap2 (And, _, d, e) -> ap_ttt (uap2 BitAnd) d e
| Ap2 (Or, _, d, e) -> ap_ttt (uap2 BitOr) d e
| Ap2 (Xor, _, d, e) -> ap_ttt (uap2 BitXor) d e
| Ap2 (Shl, _, d, e) -> ap_ttt (uap2 BitShl) d e
| Ap2 (Lshr, _, d, e) -> ap_ttt (uap2 BitLshr) d e
| Ap2 (Ashr, _, d, e) -> ap_ttt (uap2 BitAshr) d e
uap1 (Funsym.uninterp s) (term tid e)
| Ap2 (Eq, _, d, e) -> ap_ttf F.eq tid d e
| Ap2 (Dq, _, d, e) -> ap_ttf F.dq tid d e
| Ap2 (Gt, _, d, e) -> ap_ttf F.gt tid d e
| Ap2 (Lt, _, d, e) -> ap_ttf F.lt tid d e
| Ap2 (Ge, _, d, e) -> ap_ttf F.ge tid d e
| Ap2 (Le, _, d, e) -> ap_ttf F.le tid d e
| Ap2 (Ugt, typ, d, e) -> ap_uuf F.gt typ tid d e
| Ap2 (Ult, typ, d, e) -> ap_uuf F.lt typ tid d e
| Ap2 (Uge, typ, d, e) -> ap_uuf F.ge typ tid d e
| Ap2 (Ule, typ, d, e) -> ap_uuf F.le typ tid d e
| Ap2 (Ord, _, d, e) -> ap_ttf (lit2 (Predsym.uninterp "ord")) tid d e
| Ap2 (Uno, _, d, e) -> ap_ttf (nlit2 (Predsym.uninterp "ord")) tid d e
| Ap2 (Add, _, d, e) -> ap_ttt T.add tid d e
| Ap2 (Sub, _, d, e) -> ap_ttt T.sub tid d e
| Ap2 (Mul, _, d, e) -> ap_ttt T.mul tid d e
| Ap2 (Div, _, d, e) -> ap_ttt T.div tid d e
| Ap2 (Rem, _, d, e) -> ap_ttt (uap2 Rem) tid d e
| Ap2 (Udiv, typ, d, e) -> ap_uut T.div typ tid d e
| Ap2 (Urem, typ, d, e) -> ap_uut (uap2 Rem) typ tid d e
| Ap2 (And, _, d, e) -> ap_ttt (uap2 BitAnd) tid d e
| Ap2 (Or, _, d, e) -> ap_ttt (uap2 BitOr) tid d e
| Ap2 (Xor, _, d, e) -> ap_ttt (uap2 BitXor) tid d e
| Ap2 (Shl, _, d, e) -> ap_ttt (uap2 BitShl) tid d e
| Ap2 (Lshr, _, d, e) -> ap_ttt (uap2 BitLshr) tid d e
| Ap2 (Ashr, _, d, e) -> ap_ttt (uap2 BitAshr) tid d e
| Ap3 (Conditional, _, cnd, thn, els) ->
T.ite ~cnd:(formula cnd) ~thn:(term thn) ~els:(term els)
T.ite ~cnd:(formula tid cnd) ~thn:(term tid thn) ~els:(term tid els)
| Ap1 (Select idx, typ, rcd) ->
let off, len = Llair.Typ.offset_length_of_elt typ idx in
let off = T.integer (Z.of_int off) in
let len = T.integer (Z.of_int len) in
T.extract ~seq:(term rcd) ~off ~len
T.extract ~seq:(term tid rcd) ~off ~len
| Ap2 (Update idx, typ, rcd, elt) ->
let oI, lI = Llair.Typ.offset_length_of_elt typ idx in
let oJ = oI + lI in
@ -149,10 +154,10 @@ and term : Llair.Exp.t -> T.t =
let len1 = T.integer (Z.of_int lI) in
let off2 = T.integer (Z.of_int oJ) in
let len2 = T.integer (Z.of_int (Llair.Typ.size_of typ - oI - lI)) in
let seq = term rcd in
let seq = term tid rcd in
T.concat
[| T.extract ~seq ~off:off0 ~len:len0
; T.sized ~seq:(term elt) ~siz:len1
; T.sized ~seq:(term tid elt) ~siz:len1
; T.extract ~seq ~off:off2 ~len:len2 |]
| ApN (Record, typ, elts) ->
let elt_siz i =
@ -160,7 +165,7 @@ and term : Llair.Exp.t -> T.t =
in
T.concat
(Array.mapi (IArray.to_array elts) ~f:(fun i elt ->
T.sized ~seq:(term elt) ~siz:(elt_siz i) ))
| Ap1 (Splat, _, byt) -> T.splat (term byt)
T.sized ~seq:(term tid elt) ~siz:(elt_siz i) ))
| Ap1 (Splat, _, byt) -> T.splat (term tid byt)
and formula e = F.dq0 (term e)
and formula tid e = F.dq0 (term tid e)

@ -9,7 +9,7 @@ open Fol
val lookup_func : (string -> 'a option) -> Term.t -> 'a option
val global : Llair.Global.t -> Term.t
val reg : Llair.Reg.t -> Var.t
val regs : Llair.Reg.Set.t -> Var.Set.t
val term : Llair.Exp.t -> Term.t
val formula : Llair.Exp.t -> Formula.t
val reg : ThreadID.t -> Llair.Reg.t -> Var.t
val regs : ThreadID.t -> Llair.Reg.Set.t -> Var.Set.t
val term : ThreadID.t -> Llair.Exp.t -> Term.t
val formula : ThreadID.t -> Llair.Exp.t -> Formula.t

@ -562,7 +562,7 @@ let%test_module _ =
let%test "unsigned boolean overflow" =
Formula.equal Formula.tt
(Llair_to_Fol.formula
(Llair_to_Fol.formula ThreadID.init
Llair.(
Exp.uge
(Exp.integer Typ.bool Z.minus_one)
@ -570,7 +570,7 @@ let%test_module _ =
let pp_exp e =
Format.printf "@\nexp= %a; term= %a@." Llair.Exp.pp e Term.pp
(Llair_to_Fol.term e)
(Llair_to_Fol.term ThreadID.init e)
let ( !! ) i = Llair.(Exp.integer Typ.siz (Z.of_int i))

@ -0,0 +1,10 @@
(*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This source code is licensed under the MIT license found in the
* LICENSE file in the root directory of this source tree.
*)
type t = int [@@deriving compare, equal, hash, sexp_of]
let init = 0

@ -0,0 +1,10 @@
(*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This source code is licensed under the MIT license found in the
* LICENSE file in the root directory of this source tree.
*)
type t = int [@@deriving compare, equal, hash, sexp_of]
val init : t

@ -0,0 +1,19 @@
/*
* Copyright (c) Facebook, Inc. and its affiliates.
*
* This source code is licensed under the MIT license found in the
* LICENSE file in the root directory of this source tree.
*/
int count = 0;
void child_routine() { count++; }
int main() {
thread_t child_data;
thread_t* child = &child_data;
error_t err = thread_create(&child, &child_routine);
count++;
err = thread_join(child);
return count;
}
Loading…
Cancel
Save