Skip to content
This repository has been archived by the owner on Mar 24, 2022. It is now read-only.

Add a "bounded runtime" mode to guest execution. #612

Merged
merged 2 commits into from
Jan 11, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 13 additions & 1 deletion lucet-module/src/runtime.rs
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,18 @@
#[repr(align(8))]
pub struct InstanceRuntimeData {
pub globals_ptr: *mut i64,
pub instruction_count: u64,
/// `instruction_count_bound + instruction_count_adj` gives the total
/// instructions executed. We deconstruct the count into a signed adjustment
/// and a "bound" because we want to be able to set a runtime bound beyond
/// which we yield to the caller. We do this by beginning execution with
/// `instruction_count_adj` set to some negative value and
/// `instruction_count_bound` adjusted upward in compensation.
/// `instruction_count_adj` is incremented as execution proceeds; on each
/// increment, the Wasm code checks the sign. If the value is greater than
/// zero, then we have exceeded the bound and we must yield. At any point,
/// the `adj` value can be adjusted downward by transferring the count to
/// the `bound`.
pub instruction_count_adj: i64,
pub instruction_count_bound: i64,
pub stack_limit: u64,
}
6 changes: 6 additions & 0 deletions lucet-runtime/include/lucet_vmctx.h
Original file line number Diff line number Diff line change
Expand Up @@ -42,4 +42,10 @@ void *lucet_vmctx_get_func_from_idx(struct lucet_vmctx const *ctx, uint32_t tabl
// Mostly for tests - this conversion is builtin to lucetc
int64_t *lucet_vmctx_get_globals(struct lucet_vmctx const *ctx);

// Yield that is meant to be inserted by compiler instrumentation, transparent
// to Wasm code execution. It is intended to be invoked periodically (e.g.,
// every N instructions) to bound runtime of any particular execution slice of
// Wasm code.
void lucet_vmctx_yield_at_bound_expiration(struct lucet_vmctx const *ctx);

#endif // LUCET_VMCTX_H
207 changes: 116 additions & 91 deletions lucet-runtime/lucet-runtime-internals/src/future.rs
Original file line number Diff line number Diff line change
@@ -1,10 +1,12 @@
use crate::error::Error;
use crate::instance::{InstanceHandle, RunResult, State, TerminationDetails};
use crate::instance::{InstanceHandle, InternalRunResult, RunResult, State, TerminationDetails};
use crate::module::FunctionHandle;
use crate::val::{UntypedRetVal, Val};
use crate::vmctx::{Vmctx, VmctxInternal};
use std::any::Any;
use std::future::Future;
use std::pin::Pin;
use std::task::{Context, Poll};

/// This is the same type defined by the `futures` library, but we don't need the rest of the
/// library for this purpose.
Expand Down Expand Up @@ -75,23 +77,38 @@ impl Vmctx {
// Wrap the computation in `YieldedFuture` so that
// `Instance::run_async` can catch and run it. We will get the
// `ResumeVal` we applied to `f` above.
self.yield_impl::<YieldedFuture, ResumeVal>(YieldedFuture(f), false);
self.yield_impl::<YieldedFuture, ResumeVal>(YieldedFuture(f), false, false);
let ResumeVal(v) = self.take_resumed_val();
// We may now downcast and unbox the returned Box<dyn Any> into an `R`
// again.
*v.downcast().expect("run_async broke invariant")
}
}

/// This struct needs to be exposed publicly in order for the signature of a
/// "block_in_place" function to be writable, a concession we must make because
/// Rust does not have rank 2 types. To prevent the user from inspecting or
/// constructing the inside of this type, it is completely opaque.
pub struct Bounce<'a>(BounceInner<'a>);
/// A simple future that yields once. We use this to yield when a runtime bound is reached.
///
/// Inspired by Tokio's `yield_now()`.
struct YieldNow {
yielded: bool,
}

impl YieldNow {
fn new() -> Self {
Self { yielded: false }
}
}

enum BounceInner<'a> {
Done(UntypedRetVal),
More(BoxFuture<'a, ResumeVal>),
impl Future for YieldNow {
type Output = ();
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<()> {
if self.yielded {
Poll::Ready(())
} else {
self.yielded = true;
cx.waker().wake_by_ref();
Poll::Pending
}
}
}

impl InstanceHandle {
Expand All @@ -101,51 +118,59 @@ impl InstanceHandle {
/// that use `Vmctx::block_on` and provides the trampoline that `.await`s those futures on
/// behalf of the guest.
///
/// If `runtime_bound` is provided, it will also pause the Wasm execution and yield a future
/// that resumes it after (approximately) that many Wasm opcodes have executed.
cfallin marked this conversation as resolved.
Show resolved Hide resolved
///
/// # `Vmctx` Restrictions
///
/// This method permits the use of `Vmctx::block_on`, but disallows all other uses of `Vmctx::
/// yield_val_expecting_val` and family (`Vmctx::yield_`, `Vmctx::yield_expecting_val`,
/// `Vmctx::yield_val`).
pub async fn run_async<'a>(
&'a mut self,
entrypoint: &'a str,
args: &'a [Val],
runtime_bound: Option<u64>,
) -> Result<UntypedRetVal, Error> {
let func = self.module.get_export_func(entrypoint)?;
self.run_async_internal(func, args, runtime_bound).await
}

/// Run the module's [start function][start], if one exists.
///
/// # Blocking thread
///
/// The `wrap_blocking` argument is a function that is called with a closure that runs the Wasm
/// program. Since Wasm may execute for an arbitrarily long time without `await`ing, we need to
/// make sure that it runs on a thread that is allowed to block.
///
/// This argument is designed with [`tokio::task::block_in_place`][tokio] in mind. The odd type
/// is a concession to the fact that we don't have rank 2 types in Rust, and so must fall back
/// to trait objects in order to be able to take an argument that is itself a function that
/// takes a closure.
///
/// In order to provide an appropriate function, you may have to wrap the library function in
/// another closure so that the types are compatible. For example:
///
/// ```no_run
/// # async fn f() {
/// # let instance: lucet_runtime_internals::instance::InstanceHandle = unimplemented!();
/// fn block_in_place<F, R>(f: F) -> R
/// where
/// F: FnOnce() -> R,
/// {
/// // ...
/// # f()
/// }
/// If there is no start function in the module, this does nothing.
///
/// instance.run_async("entrypoint", &[], |f| block_in_place(f)).await.unwrap();
/// # }
/// ```
/// All of the other restrictions on the start function, what it may do, and
/// the requirement that it must be invoked first, are described in the
/// documentation for `Instance::run_start()`. This async version of that
/// function satisfies the requirement to run the start function first, as
/// long as the async function fully returns (not just yields).
///
/// [tokio]: https://docs.rs/tokio/0.2.21/tokio/task/fn.block_in_place.html
pub async fn run_async<'a, F>(
/// This method is similar to `Instance::run_start()`, except that it bounds
/// runtime between async future yields (invocations of `.poll()` on the
/// underlying generated future) if `runtime_bound` is provided. This
/// behaves the same way as `Instance::run_async()`.
pub async fn run_async_start<'a>(
&'a mut self,
entrypoint: &'a str,
runtime_bound: Option<u64>,
) -> Result<(), Error> {
if let Some(start) = self.module.get_start_func()? {
if !self.is_not_started() {
return Err(Error::StartAlreadyRun);
}
self.run_async_internal(start, &[], runtime_bound).await?;
}
Ok(())
}

/// Shared async run-loop implementation for both `run_async()` and
/// `run_start_async()`.
async fn run_async_internal<'a>(
&'a mut self,
func: FunctionHandle,
args: &'a [Val],
wrap_blocking: F,
) -> Result<UntypedRetVal, Error>
where
F: for<'b> Fn(&mut (dyn FnMut() -> Result<Bounce<'b>, Error>)) -> Result<Bounce<'b>, Error>,
cfallin marked this conversation as resolved.
Show resolved Hide resolved
{
runtime_bound: Option<u64>,
) -> Result<UntypedRetVal, Error> {
if self.is_yielded() {
return Err(Error::Unsupported(
"cannot run_async a yielded instance".to_owned(),
Expand All @@ -156,55 +181,55 @@ impl InstanceHandle {
let mut resume_val: Option<ResumeVal> = None;
loop {
// Run the WebAssembly program
let bounce = wrap_blocking(&mut || {
let run_result = if self.is_yielded() {
// A previous iteration of the loop stored the ResumeVal in
// `resume_val`, send it back to the guest ctx and continue
// running:
self.resume_with_val_impl(
resume_val
.take()
.expect("is_yielded implies resume_value is some"),
true,
)
} else {
// This is the first iteration, call the entrypoint:
let func = self.module.get_export_func(entrypoint)?;
self.run_func(func, args, true)
};
match run_result? {
RunResult::Returned(rval) => {
// Finished running, return UntypedReturnValue
return Ok(Bounce(BounceInner::Done(rval)));
}
RunResult::Yielded(yval) => {
// Check if the yield came from Vmctx::block_on:
if yval.is::<YieldedFuture>() {
let YieldedFuture(future) = *yval.downcast::<YieldedFuture>().unwrap();
// Rehydrate the lifetime from `'static` to `'a`, which
// is morally the same lifetime as was passed into
// `Vmctx::block_on`.
Ok(Bounce(BounceInner::More(
future as BoxFuture<'a, ResumeVal>,
)))
} else {
// Any other yielded value is not supported - die with an error.
Err(Error::Unsupported(
"cannot yield anything besides a future in Instance::run_async"
.to_owned(),
))
}
}
let run_result = if self.is_yielded() {
// A previous iteration of the loop stored the ResumeVal in
// `resume_val`, send it back to the guest ctx and continue
// running:
self.resume_with_val_impl(
resume_val
.take()
.expect("is_yielded implies resume_value is some"),
true,
runtime_bound,
)
} else if self.is_bound_expired() {
self.resume_bounded(
runtime_bound.expect("should have bound if guest had expired bound"),
)
} else {
// This is the first iteration, call the entrypoint:
self.run_func(func, args, true, runtime_bound)
};
match run_result? {
InternalRunResult::Normal(RunResult::Returned(rval)) => {
// Finished running, return UntypedReturnValue
return Ok(rval);
}
})?;
match bounce {
Bounce(BounceInner::Done(rval)) => return Ok(rval),
Bounce(BounceInner::More(fut)) => {
// await on the computation. Store its result in
// `resume_val`.
resume_val = Some(fut.await);
InternalRunResult::Normal(RunResult::Yielded(yval)) => {
// Check if the yield came from Vmctx::block_on:
if yval.is::<YieldedFuture>() {
let YieldedFuture(future) = *yval.downcast::<YieldedFuture>().unwrap();
// Rehydrate the lifetime from `'static` to `'a`, which
// is morally the same lifetime as was passed into
// `Vmctx::block_on`.
let future = future as BoxFuture<'a, ResumeVal>;

// await on the computation. Store its result in
// `resume_val`.
resume_val = Some(future.await);
// Now we want to `Instance::resume_with_val` and start
// this cycle over.
} else {
// Any other yielded value is not supported - die with an error.
return Err(Error::Unsupported(
"cannot yield anything besides a future in Instance::run_async"
.to_owned(),
));
}
}
InternalRunResult::BoundExpired => {
// Await on a simple future that yields once then is ready.
YieldNow::new().await
}
}
}
Expand Down
Loading