Implement shared host functions. (#2625)

* Implement defining host functions at the Config level.

This commit introduces defining host functions at the `Config` rather than with
`Func` tied to a `Store`.

The intention here is to enable a host to define all of the functions once
with a `Config` and then use a `Linker` (or directly with
`Store::get_host_func`) to use the functions when instantiating a module.

This should help improve the performance of use cases where a `Store` is
short-lived and redefining the functions at every module instantiation is a
noticeable performance hit.

This commit adds `add_to_config` to the code generation for Wasmtime's `Wasi`
type.

The new method adds the WASI functions to the given config as host functions.

This commit adds context functions to `Store`: `get` to get a context of a
particular type and `set` to set the context on the store.

For safety, `set` cannot replace an existing context value of the same type.

`Wasi::set_context` was added to set the WASI context for a `Store` when using
`Wasi::add_to_config`.

* Add `Config::define_host_func_async`.

* Make config "async" rather than store.

This commit moves the concept of "async-ness" to `Config` rather than `Store`.

Note: this is a breaking API change for anyone that's already adopted the new
async support in Wasmtime.

Now `Config::new_async` is used to create an "async" config and any `Store`
associated with that config is inherently "async".

This is needed for async shared host functions to have some sanity check during their
execution (async host functions, like "async" `Func`, need to be called with
the "async" variants).

* Update async function tests to smoke async shared host functions.

This commit updates the async function tests to also smoke the shared host
functions, plus `Func::wrap0_async`.

This also changes the "wrap async" method names on `Config` to
`wrap$N_host_func_async` to slightly better match what is on `Func`.

* Move the instance allocator into `Engine`.

This commit moves the instantiated instance allocator from `Config` into
`Engine`.

This makes certain settings in `Config` no longer order-dependent, which is how
`Config` should ideally be.

This also removes the confusing concept of the "default" instance allocator,
instead opting to construct the on-demand instance allocator when needed.

This does alter the semantics of the instance allocator as now each `Engine`
gets its own instance allocator rather than sharing a single one between all
engines created from a configuration.

* Make `Engine::new` return `Result`.

This is a breaking API change for anyone using `Engine::new`.

As creating the pooling instance allocator may fail (likely cause is not enough
memory for the provided limits), instead of panicking when creating an
`Engine`, `Engine::new` now returns a `Result`.

* Remove `Config::new_async`.

This commit removes `Config::new_async` in favor of treating "async support" as
any other setting on `Config`.

The setting is `Config::async_support`.

* Remove order dependency when defining async host functions in `Config`.

This commit removes the order dependency where async support must be enabled on
the `Config` prior to defining async host functions.

The check is now delayed to when an `Engine` is created from the config.

* Update WASI example to use shared `Wasi::add_to_config`.

This commit updates the WASI example to use `Wasi::add_to_config`.

As only a single store and instance are used in the example, it has no semantic
difference from the previous example, but the intention is to steer users
towards defining WASI on the config and only using `Wasi::add_to_linker` when
more explicit scoping of the WASI context is required.
This commit is contained in:
Peter Huene
2021-03-11 08:14:03 -08:00
committed by GitHub
parent cc84c693a3
commit 54c07d8f16
45 changed files with 2123 additions and 809 deletions

View File

@@ -203,7 +203,7 @@ impl BenchState {
config.wasm_simd(true);
// NB: do not configure a code cache.
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let mut linker = Linker::new(&store);

View File

@@ -30,6 +30,6 @@ pub extern "C" fn wasm_engine_new() -> Box<wasm_engine_t> {
pub extern "C" fn wasm_engine_new_with_config(c: Box<wasm_config_t>) -> Box<wasm_engine_t> {
let config = c.config;
Box::new(wasm_engine_t {
engine: Engine::new(&config),
engine: Engine::new(&config).unwrap(),
})
}

View File

@@ -90,7 +90,7 @@ pub fn instantiate_with_config(
Timeout::Fuel(_) => true,
_ => false,
});
let engine = Engine::new(&config);
let engine = Engine::new(&config).unwrap();
let store = Store::new(&engine);
let mut timeout_state = SignalOnDrop::default();
@@ -143,7 +143,7 @@ pub fn instantiate_with_config(
pub fn compile(wasm: &[u8], strategy: Strategy) {
crate::init_fuzzing();
let engine = Engine::new(&crate::fuzz_default_config(strategy).unwrap());
let engine = Engine::new(&crate::fuzz_default_config(strategy).unwrap()).unwrap();
log_wasm(wasm);
let _ = Module::new(&engine, wasm);
}
@@ -180,7 +180,7 @@ pub fn differential_execution(
log_wasm(&wasm);
for config in &configs {
let engine = Engine::new(config);
let engine = Engine::new(config).unwrap();
let store = Store::new(&engine);
let module = Module::new(&engine, &wasm).unwrap();
@@ -320,7 +320,7 @@ pub fn make_api_calls(api: crate::generators::api::ApiCalls) {
ApiCall::EngineNew => {
log::trace!("creating engine");
assert!(engine.is_none());
engine = Some(Engine::new(config.as_ref().unwrap()));
engine = Some(Engine::new(config.as_ref().unwrap()).unwrap());
}
ApiCall::StoreNew => {
@@ -416,7 +416,7 @@ pub fn spectest(fuzz_config: crate::generators::Config, test: crate::generators:
let mut config = fuzz_config.to_wasmtime();
config.wasm_reference_types(false);
config.wasm_bulk_memory(false);
let store = Store::new(&Engine::new(&config));
let store = Store::new(&Engine::new(&config).unwrap());
if fuzz_config.consume_fuel {
store.add_fuel(u64::max_value()).unwrap();
}
@@ -439,7 +439,7 @@ pub fn table_ops(
{
let mut config = fuzz_config.to_wasmtime();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config).unwrap();
let store = Store::new(&engine);
if fuzz_config.consume_fuel {
store.add_fuel(u64::max_value()).unwrap();
@@ -554,7 +554,7 @@ pub fn differential_wasmi_execution(wasm: &[u8], config: &crate::generators::Con
// Instantiate wasmtime module and instance.
let mut wasmtime_config = config.to_wasmtime();
wasmtime_config.cranelift_nan_canonicalization(true);
let wasmtime_engine = Engine::new(&wasmtime_config);
let wasmtime_engine = Engine::new(&wasmtime_config).unwrap();
let wasmtime_store = Store::new(&wasmtime_engine);
if config.consume_fuel {
wasmtime_store.add_fuel(u64::max_value()).unwrap();

View File

@@ -359,7 +359,7 @@ mod tests {
let mut config = Config::default();
config.wasm_module_linking(true);
config.wasm_multi_memory(true);
let engine = wasmtime::Engine::new(&config);
let engine = wasmtime::Engine::new(&config).unwrap();
Store::new(&engine)
}

View File

@@ -581,9 +581,6 @@ unsafe impl InstanceAllocator for OnDemandInstanceAllocator {
&self,
mut req: InstanceAllocationRequest,
) -> Result<InstanceHandle, InstantiationError> {
debug_assert!(!req.externref_activations_table.is_null());
debug_assert!(!req.stack_map_registry.is_null());
let memories = self.create_memories(&req.module)?;
let tables = Self::create_tables(&req.module);

View File

@@ -1,7 +1,7 @@
//! WebAssembly trap handling, which is built on top of the lower-level
//! signalhandling mechanisms.
use crate::VMContext;
use crate::VMInterrupts;
use backtrace::Backtrace;
use std::any::Any;
use std::cell::Cell;
@@ -445,11 +445,7 @@ impl Trap {
/// returning them as a `Result`.
///
/// Highly unsafe since `closure` won't have any dtors run.
pub unsafe fn catch_traps<F>(
vmctx: *mut VMContext,
trap_info: &impl TrapInfo,
mut closure: F,
) -> Result<(), Trap>
pub unsafe fn catch_traps<F>(trap_info: &impl TrapInfo, mut closure: F) -> Result<(), Trap>
where
F: FnMut(),
{
@@ -457,7 +453,7 @@ where
#[cfg(unix)]
setup_unix_sigaltstack()?;
return CallThreadState::new(vmctx, trap_info).with(|cx| {
return CallThreadState::new(trap_info).with(|cx| {
RegisterSetjmp(
cx.jmp_buf.as_ptr(),
call_closure::<F>,
@@ -493,7 +489,6 @@ pub fn out_of_gas() {
pub struct CallThreadState<'a> {
unwind: Cell<UnwindReason>,
jmp_buf: Cell<*const u8>,
vmctx: *mut VMContext,
handling_trap: Cell<bool>,
trap_info: &'a (dyn TrapInfo + 'a),
}
@@ -526,6 +521,9 @@ pub unsafe trait TrapInfo {
///
/// This function may return, and it may also `raise_lib_trap`.
fn out_of_gas(&self);
/// Returns the VM interrupts to use for interrupting Wasm code.
fn interrupts(&self) -> &VMInterrupts;
}
enum UnwindReason {
@@ -537,10 +535,9 @@ enum UnwindReason {
}
impl<'a> CallThreadState<'a> {
fn new(vmctx: *mut VMContext, trap_info: &'a (dyn TrapInfo + 'a)) -> CallThreadState<'a> {
fn new(trap_info: &'a (dyn TrapInfo + 'a)) -> CallThreadState<'a> {
CallThreadState {
unwind: Cell::new(UnwindReason::None),
vmctx,
jmp_buf: Cell::new(ptr::null()),
handling_trap: Cell::new(false),
trap_info,
@@ -562,10 +559,9 @@ impl<'a> CallThreadState<'a> {
UnwindReason::LibTrap(trap) => Err(trap),
UnwindReason::JitTrap { backtrace, pc } => {
debug_assert_eq!(ret, 0);
let maybe_interrupted = unsafe {
let interrupts = (*self.vmctx).instance().interrupts();
(**interrupts).stack_limit.load(SeqCst) == wasmtime_environ::INTERRUPTED
};
let interrupts = self.trap_info.interrupts();
let maybe_interrupted =
interrupts.stack_limit.load(SeqCst) == wasmtime_environ::INTERRUPTED;
Err(Trap::Jit {
pc,
backtrace,
@@ -622,7 +618,7 @@ impl<'a> CallThreadState<'a> {
// (a million bytes) the slop shouldn't matter too much.
let wasm_stack_limit = psm::stack_pointer() as usize - self.trap_info.max_wasm_stack();
let interrupts = unsafe { &**(&*self.vmctx).instance().interrupts() };
let interrupts = self.trap_info.interrupts();
let reset_stack_limit = match interrupts.stack_limit.compare_exchange(
usize::max_value(),
wasm_stack_limit,

View File

@@ -10,7 +10,7 @@
use std::cell::RefCell;
use std::rc::Rc;
pub use wasi_common::{Error, WasiCtx, WasiCtxBuilder, WasiDir, WasiFile};
use wasmtime::{Linker, Store};
use wasmtime::{Config, Linker, Store};
/// An instantiated instance of all available wasi exports. Presently includes
/// both the "preview1" snapshot and the "unstable" (preview0) snapshot.
@@ -34,6 +34,15 @@ impl Wasi {
self.preview_0.add_to_linker(linker)?;
Ok(())
}
pub fn add_to_config(config: &mut Config) {
snapshots::preview_1::Wasi::add_to_config(config);
snapshots::preview_0::Wasi::add_to_config(config);
}
pub fn set_context(store: &Store, context: WasiCtx) -> Result<(), WasiCtx> {
// It doesn't matter which underlying `Wasi` type this gets called on as the
// implementations are identical
snapshots::preview_1::Wasi::set_context(store, context)
}
}
pub mod snapshots {

View File

@@ -1,11 +1,15 @@
use crate::memory::MemoryCreator;
use crate::trampoline::MemoryCreatorProxy;
use crate::{func::HostFunc, Caller, FuncType, IntoFunc, Trap, Val, WasmRet, WasmTy};
use anyhow::{bail, Result};
use std::cmp;
use std::collections::HashMap;
use std::convert::TryFrom;
use std::fmt;
use std::future::Future;
#[cfg(feature = "cache")]
use std::path::Path;
use std::pin::Pin;
use std::sync::Arc;
use wasmparser::WasmFeatures;
#[cfg(feature = "cache")]
@@ -14,7 +18,9 @@ use wasmtime_environ::settings::{self, Configurable, SetError};
use wasmtime_environ::{isa, isa::TargetIsa, Tunables};
use wasmtime_jit::{native, CompilationStrategy, Compiler};
use wasmtime_profiling::{JitDumpAgent, NullProfilerAgent, ProfilingAgent, VTuneAgent};
use wasmtime_runtime::{InstanceAllocator, OnDemandInstanceAllocator, PoolingInstanceAllocator};
use wasmtime_runtime::{
InstanceAllocator, OnDemandInstanceAllocator, PoolingInstanceAllocator, RuntimeMemoryCreator,
};
/// Represents the limits placed on a module for compiling with the pooling instance allocation strategy.
#[derive(Debug, Copy, Clone)]
@@ -266,6 +272,93 @@ impl Default for InstanceAllocationStrategy {
}
}
/// This type is used for storing host functions in a `Config`.
///
/// The module and function names are interned for more compact storage.
#[derive(Clone)]
struct HostFuncMap {
index_map: HashMap<Arc<str>, usize>,
strings: Vec<Arc<str>>,
funcs: HashMap<(usize, usize), (Arc<HostFunc>, bool)>,
}
impl HostFuncMap {
fn new() -> Self {
Self {
index_map: HashMap::new(),
strings: Vec::new(),
funcs: HashMap::new(),
}
}
fn insert(&mut self, module: &str, name: &str, async_required: bool, func: HostFunc) {
let key = (self.intern_str(module), self.intern_str(name));
self.funcs.insert(key, (Arc::new(func), async_required));
}
fn get(&self, module: &str, name: &str) -> Option<&HostFunc> {
let key = (
self.index_map.get(module).cloned()?,
self.index_map.get(name).cloned()?,
);
self.funcs.get(&key).map(|f| f.0.as_ref())
}
fn intern_str(&mut self, string: &str) -> usize {
if let Some(idx) = self.index_map.get(string) {
return *idx;
}
let string: Arc<str> = string.into();
let idx = self.strings.len();
self.strings.push(string.clone());
self.index_map.insert(string, idx);
idx
}
fn async_required(&self) -> bool {
self.funcs.values().any(|f| f.1)
}
}
macro_rules! generate_wrap_async_host_func {
($num:tt $($args:ident)*) => (paste::paste!{
/// Same as [`Config::wrap_host_func`], except the closure asynchronously produces
/// its result. For more information see the [`Func`](crate::Func) documentation.
///
/// Note: creating an engine will fail if an async host function is defined and
/// [async support](Config::async_support) is not enabled.
#[allow(non_snake_case)]
#[cfg(feature = "async")]
#[cfg_attr(nightlydoc, doc(cfg(feature = "async")))]
pub fn [<wrap $num _host_func_async>]<$($args,)* R>(
&mut self,
module: &str,
name: &str,
func: impl for <'a> Fn(Caller<'a>, $($args),*) -> Box<dyn Future<Output = R> + 'a> + Send + Sync + 'static,
)
where
$($args: WasmTy,)*
R: WasmRet,
{
// Defer the check for async support until engine creation time to not introduce an order dependency
self.host_funcs.insert(
module,
name,
true,
HostFunc::wrap(move |caller: Caller<'_>, $($args: $args),*| {
let store = caller.store().clone();
debug_assert!(store.async_support());
let mut future = Pin::from(func(caller, $($args),*));
match store.block_on(future.as_mut()) {
Ok(ret) => ret.into_result(),
Err(e) => Err(e),
}
})
);
}
})
}
/// Global configuration options used to create an [`Engine`](crate::Engine)
/// and customize its behavior.
///
@@ -280,10 +373,8 @@ pub struct Config {
#[cfg(feature = "cache")]
pub(crate) cache_config: CacheConfig,
pub(crate) profiler: Arc<dyn ProfilingAgent>,
pub(crate) instance_allocator: Option<Arc<dyn InstanceAllocator>>,
// The default instance allocator is used for instantiating host objects
// and for module instantiation when `instance_allocator` is None
pub(crate) default_instance_allocator: OnDemandInstanceAllocator,
pub(crate) mem_creator: Option<Arc<dyn RuntimeMemoryCreator>>,
pub(crate) allocation_strategy: InstanceAllocationStrategy,
pub(crate) max_wasm_stack: usize,
pub(crate) features: WasmFeatures,
pub(crate) wasm_backtrace_details_env_used: bool,
@@ -292,12 +383,14 @@ pub struct Config {
pub(crate) max_memories: usize,
#[cfg(feature = "async")]
pub(crate) async_stack_size: usize,
host_funcs: HostFuncMap,
pub(crate) async_support: bool,
}
impl Config {
/// Creates a new configuration object with the default configuration
/// specified.
pub fn new() -> Config {
pub fn new() -> Self {
let mut flags = settings::builder();
// There are two possible traps for division, and this way
@@ -321,7 +414,7 @@ impl Config {
.set("enable_probestack", "false")
.expect("should be valid flag");
let mut ret = Config {
let mut ret = Self {
tunables: Tunables::default(),
flags,
isa_flags: native::builder(),
@@ -329,8 +422,8 @@ impl Config {
#[cfg(feature = "cache")]
cache_config: CacheConfig::new_cache_disabled(),
profiler: Arc::new(NullProfilerAgent),
instance_allocator: None,
default_instance_allocator: OnDemandInstanceAllocator::new(None),
mem_creator: None,
allocation_strategy: InstanceAllocationStrategy::OnDemand,
max_wasm_stack: 1 << 20,
wasm_backtrace_details_env_used: false,
features: WasmFeatures {
@@ -344,9 +437,95 @@ impl Config {
max_memories: 10_000,
#[cfg(feature = "async")]
async_stack_size: 2 << 20,
host_funcs: HostFuncMap::new(),
async_support: false,
};
ret.wasm_backtrace_details(WasmBacktraceDetails::Environment);
return ret;
ret
}
/// Whether or not to enable support for asynchronous functions in Wasmtime.
///
/// When enabled, the config can optionally define host functions with `async`.
/// Instances created and functions called with this `Config` *must* be called
/// through their asynchronous APIs, however. For example using
/// [`Func::call`](crate::Func::call) will panic when used with this config.
///
/// # Asynchronous Wasm
///
/// WebAssembly does not currently have a way to specify at the bytecode
/// level what is and isn't async. Host-defined functions, however, may be
/// defined as `async`. WebAssembly imports always appear synchronous, which
/// gives rise to a bit of an impedance mismatch here. To solve this
/// Wasmtime supports "asynchronous configs" which enables calling these
/// asynchronous functions in a way that looks synchronous to the executing
/// WebAssembly code.
///
/// An asynchronous config must always invoke wasm code asynchronously,
/// meaning we'll always represent its computation as a
/// [`Future`](std::future::Future). The `poll` method of the futures
/// returned by Wasmtime will perform the actual work of calling the
/// WebAssembly. Wasmtime won't manage its own thread pools or similar,
/// that's left up to the embedder.
///
/// To implement futures in a way that WebAssembly sees asynchronous host
/// functions as synchronous, all async Wasmtime futures will execute on a
/// separately allocated native stack from the thread otherwise executing
/// Wasmtime. This separate native stack can then be switched to and from.
/// Using this whenever an `async` host function returns a future that
/// resolves to `Pending` we switch away from the temporary stack back to
/// the main stack and propagate the `Pending` status.
///
/// In general it's encouraged that the integration with `async` and
/// wasmtime is designed early on in your embedding of Wasmtime to ensure
/// that it's planned that WebAssembly executes in the right context of your
/// application.
///
/// # Execution in `poll`
///
/// The [`Future::poll`](std::future::Future::poll) method is the main
/// driving force behind Rust's futures. That method's own documentation
/// states "an implementation of `poll` should strive to return quickly, and
/// should not block". This, however, can be at odds with executing
/// WebAssembly code as part of the `poll` method itself. If your
/// WebAssembly is untrusted then this could allow the `poll` method to take
/// arbitrarily long in the worst case, likely blocking all other
/// asynchronous tasks.
///
/// To remedy this situation you have a two possible ways to solve this:
///
/// * First you can spawn futures into a thread pool. Care must be taken for
/// this because Wasmtime futures are not `Send` or `Sync`. If you ensure
/// that the entire state of a `Store` is wrapped up in a single future,
/// though, you can send the whole future at once to a separate thread. By
/// doing this in a thread pool you are relaxing the requirement that
/// `Future::poll` must be fast because your future is executing on a
/// separate thread. This strategy, however, would likely still require
/// some form of cancellation via [`crate::Store::interrupt_handle`] to ensure
/// wasm doesn't take *too* long to execute.
///
/// * Alternatively you can enable the
/// [`Config::consume_fuel`](crate::Config::consume_fuel) method as well
/// as [`crate::Store::out_of_fuel_async_yield`] When doing so this will
/// configure Wasmtime futures to yield periodically while they're
/// executing WebAssembly code. After consuming the specified amount of
/// fuel wasm futures will return `Poll::Pending` from their `poll`
/// method, and will get automatically re-polled later. This enables the
/// `Future::poll` method to take roughly a fixed amount of time since
/// fuel is guaranteed to get consumed while wasm is executing. Note that
/// to prevent infinite execution of wasm you'll still need to use
/// [`crate::Store::interrupt_handle`].
///
/// In either case special care needs to be taken when integrating
/// asynchronous wasm into your application. You should carefully plan where
/// WebAssembly will execute and what compute resources will be allotted to
/// it. If Wasmtime doesn't support exactly what you'd like just yet, please
/// feel free to open an issue!
#[cfg(feature = "async")]
#[cfg_attr(nightlydoc, doc(cfg(feature = "async")))]
pub fn async_support(&mut self, enable: bool) -> &mut Self {
self.async_support = enable;
self
}
/// Configures whether DWARF debug information will be emitted during
@@ -358,7 +537,7 @@ impl Config {
self
}
/// Configures backtraces in `Trap` will parse debuginfo in the wasm file to
/// Configures whether backtraces in `Trap` will parse debug info in the wasm file to
/// have filename/line number information.
///
/// When enabled this will causes modules to retain debugging information
@@ -437,9 +616,6 @@ impl Config {
/// on stack overflow, a host function that overflows the stack will
/// abort the process.
///
/// `max_wasm_stack` must be set prior to setting an instance allocation
/// strategy.
///
/// By default this option is 1 MiB.
pub fn max_wasm_stack(&mut self, size: usize) -> Result<&mut Self> {
#[cfg(feature = "async")]
@@ -451,12 +627,6 @@ impl Config {
bail!("wasm stack size cannot be zero");
}
if self.instance_allocator.is_some() {
bail!(
"wasm stack size cannot be modified after setting an instance allocation strategy"
);
}
self.max_wasm_stack = size;
Ok(self)
}
@@ -471,20 +641,12 @@ impl Config {
/// close to one another; doing so may cause host functions to overflow the
/// stack and abort the process.
///
/// `async_stack_size` must be set prior to setting an instance allocation
/// strategy.
///
/// By default this option is 2 MiB.
#[cfg(feature = "async")]
pub fn async_stack_size(&mut self, size: usize) -> Result<&mut Self> {
if size < self.max_wasm_stack {
bail!("async stack size cannot be less than the maximum wasm stack size");
}
if self.instance_allocator.is_some() {
bail!(
"async stack size cannot be modified after setting an instance allocation strategy"
);
}
self.async_stack_size = size;
Ok(self)
}
@@ -658,7 +820,7 @@ impl Config {
Ok(self)
}
/// Creates a default profiler based on the profiling strategy choosen
/// Creates a default profiler based on the profiling strategy chosen.
///
/// Profiler creation calls the type's default initializer where the purpose is
/// really just to put in place the type used for profiling.
@@ -820,8 +982,7 @@ impl Config {
/// Custom memory creators are used when creating host `Memory` objects or when
/// creating instance linear memories for the on-demand instance allocation strategy.
pub fn with_host_memory(&mut self, mem_creator: Arc<dyn MemoryCreator>) -> &mut Self {
self.default_instance_allocator =
OnDemandInstanceAllocator::new(Some(Arc::new(MemoryCreatorProxy(mem_creator))));
self.mem_creator = Some(Arc::new(MemoryCreatorProxy(mem_creator)));
self
}
@@ -832,32 +993,9 @@ impl Config {
/// This means the [`Config::static_memory_maximum_size`] and [`Config::static_memory_guard_size`] options
/// will be ignored in favor of [`InstanceLimits::memory_reservation_size`] when the pooling instance
/// allocation strategy is used.
pub fn with_allocation_strategy(
&mut self,
strategy: InstanceAllocationStrategy,
) -> Result<&mut Self> {
self.instance_allocator = match strategy {
InstanceAllocationStrategy::OnDemand => None,
InstanceAllocationStrategy::Pooling {
strategy,
module_limits,
instance_limits,
} => {
#[cfg(feature = "async")]
let stack_size = self.async_stack_size;
#[cfg(not(feature = "async"))]
let stack_size = 0;
Some(Arc::new(PoolingInstanceAllocator::new(
strategy.into(),
module_limits.into(),
instance_limits.into(),
stack_size,
)?))
}
};
Ok(self)
pub fn allocation_strategy(&mut self, strategy: InstanceAllocationStrategy) -> &mut Self {
self.allocation_strategy = strategy;
self
}
/// Configures the maximum size, in bytes, where a linear memory is
@@ -1062,6 +1200,103 @@ impl Config {
self
}
/// Defines a host function for the [`Config`] for the given callback.
///
/// Use [`Store::get_host_func`](crate::Store::get_host_func) to get a [`Func`](crate::Func) representing the function.
///
/// Note that the implementation of `func` must adhere to the `ty`
/// signature given, error or traps may occur if it does not respect the
/// `ty` signature.
///
/// Additionally note that this is quite a dynamic function since signatures
/// are not statically known. For performance reasons, it's recommended
/// to use [`Config::wrap_host_func`] if you can because with statically known
/// signatures the engine can optimize the implementation much more.
///
/// The callback must be `Send` and `Sync` as it is shared between all engines created
/// from the `Config`. For more relaxed bounds, use [`Func::new`](crate::Func::new) to define the function.
pub fn define_host_func(
&mut self,
module: &str,
name: &str,
ty: FuncType,
func: impl Fn(Caller<'_>, &[Val], &mut [Val]) -> Result<(), Trap> + Send + Sync + 'static,
) {
self.host_funcs
.insert(module, name, false, HostFunc::new(self, ty, func));
}
/// Defines an async host function for the [`Config`] for the given callback.
///
/// Use [`Store::get_host_func`](crate::Store::get_host_func) to get a [`Func`](crate::Func) representing the function.
///
/// This function is the asynchronous analogue of [`Config::define_host_func`] and much of
/// that documentation applies to this as well.
///
/// Additionally note that this is quite a dynamic function since signatures
/// are not statically known. For performance reasons, it's recommended
/// to use `Config::wrap$N_host_func_async` if you can because with statically known
/// signatures the engine can optimize the implementation much more.
///
/// The callback must be `Send` and `Sync` as it is shared between all engines created
/// from the `Config`. For more relaxed bounds, use [`Func::new_async`](crate::Func::new_async) to define the function.
///
/// Note: creating an engine will fail if an async host function is defined and [async support](Config::async_support)
/// is not enabled.
#[cfg(feature = "async")]
#[cfg_attr(nightlydoc, doc(cfg(feature = "async")))]
pub fn define_host_func_async<F>(&mut self, module: &str, name: &str, ty: FuncType, func: F)
where
F: for<'a> Fn(
Caller<'a>,
&'a [Val],
&'a mut [Val],
) -> Box<dyn Future<Output = Result<(), Trap>> + 'a>
+ Send
+ Sync
+ 'static,
{
// Defer the check for async support until engine creation time to not introduce an order dependency
self.host_funcs.insert(
module,
name,
true,
HostFunc::new(self, ty, move |caller, params, results| {
let store = caller.store().clone();
debug_assert!(store.async_support());
let mut future = Pin::from(func(caller, params, results));
match store.block_on(future.as_mut()) {
Ok(Ok(())) => Ok(()),
Ok(Err(trap)) | Err(trap) => Err(trap),
}
}),
);
}
/// Defines a host function for the [`Config`] from the given Rust closure.
///
/// Use [`Store::get_host_func`](crate::Store::get_host_func) to get a [`Func`](crate::Func) representing the function.
///
/// See [`Func::wrap`](crate::Func::wrap) for information about accepted parameter and result types for the closure.
///
/// The closure must be `Send` and `Sync` as it is shared between all engines created
/// from the `Config`. For more relaxed bounds, use [`Func::wrap`](crate::Func::wrap) to wrap the closure.
pub fn wrap_host_func<Params, Results>(
&mut self,
module: &str,
name: &str,
func: impl IntoFunc<Params, Results> + Send + Sync,
) {
self.host_funcs
.insert(module, name, false, HostFunc::wrap(func));
}
for_each_function_signature!(generate_wrap_async_host_func);
pub(crate) fn get_host_func(&self, module: &str, name: &str) -> Option<&HostFunc> {
self.host_funcs.get(module, name)
}
pub(crate) fn target_isa(&self) -> Box<dyn TargetIsa> {
self.isa_flags
.clone()
@@ -1074,17 +1309,49 @@ impl Config {
self.isa_flags.clone().finish(settings::Flags::new(flags))
}
pub(crate) fn build_compiler(&self) -> Compiler {
pub(crate) fn validate(&self) -> Result<()> {
// This is used to validate that the config is internally consistent prior to
// creating an engine using this config.
// Check that there isn't a host function defined that requires async support enabled
if self.host_funcs.async_required() && !self.async_support {
bail!("an async host function cannot be defined without async support enabled in the config");
}
Ok(())
}
pub(crate) fn build_compiler(&self, allocator: &dyn InstanceAllocator) -> Compiler {
let isa = self.target_isa();
let mut tunables = self.tunables.clone();
self.instance_allocator().adjust_tunables(&mut tunables);
allocator.adjust_tunables(&mut tunables);
Compiler::new(isa, self.strategy, tunables, self.features)
}
pub(crate) fn instance_allocator(&self) -> &dyn InstanceAllocator {
self.instance_allocator
.as_deref()
.unwrap_or(&self.default_instance_allocator)
pub(crate) fn build_allocator(&self) -> Result<Box<dyn InstanceAllocator>> {
match self.allocation_strategy {
InstanceAllocationStrategy::OnDemand => Ok(Box::new(OnDemandInstanceAllocator::new(
self.mem_creator.clone(),
))),
InstanceAllocationStrategy::Pooling {
strategy,
module_limits,
instance_limits,
} => {
#[cfg(feature = "async")]
let stack_size = self.async_stack_size;
#[cfg(not(feature = "async"))]
let stack_size = 0;
Ok(Box::new(PoolingInstanceAllocator::new(
strategy.into(),
module_limits.into(),
instance_limits.into(),
stack_size,
)?))
}
}
}
}

View File

@@ -1,9 +1,10 @@
use crate::Config;
use anyhow::Result;
use std::sync::Arc;
#[cfg(feature = "cache")]
use wasmtime_cache::CacheConfig;
use wasmtime_jit::Compiler;
use wasmtime_runtime::debug_builtins;
use wasmtime_runtime::{debug_builtins, InstanceAllocator};
/// An `Engine` which is a global context for compilation and management of wasm
/// modules.
@@ -35,19 +36,23 @@ pub struct Engine {
struct EngineInner {
config: Config,
compiler: Compiler,
allocator: Box<dyn InstanceAllocator>,
}
impl Engine {
/// Creates a new [`Engine`] with the specified compilation and
/// configuration settings.
pub fn new(config: &Config) -> Engine {
pub fn new(config: &Config) -> Result<Engine> {
debug_builtins::ensure_exported();
Engine {
config.validate()?;
let allocator = config.build_allocator()?;
Ok(Engine {
inner: Arc::new(EngineInner {
config: config.clone(),
compiler: config.build_compiler(),
compiler: config.build_compiler(allocator.as_ref()),
allocator,
}),
}
})
}
/// Returns the configuration settings that this engine is using.
@@ -59,6 +64,10 @@ impl Engine {
&self.inner.compiler
}
pub(crate) fn allocator(&self) -> &dyn InstanceAllocator {
self.inner.allocator.as_ref()
}
#[cfg(feature = "cache")]
pub(crate) fn cache_config(&self) -> &CacheConfig {
&self.config().cache_config
@@ -72,7 +81,7 @@ impl Engine {
impl Default for Engine {
fn default() -> Engine {
Engine::new(&Config::default())
Engine::new(&Config::default()).unwrap()
}
}
@@ -100,7 +109,7 @@ mod tests {
let mut cfg = Config::new();
cfg.cranelift_opt_level(OptLevel::None)
.cache_config_load(&config_path)?;
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
Module::new(&engine, "(module (func))")?;
assert_eq!(engine.config().cache_config.cache_hits(), 0);
assert_eq!(engine.config().cache_config.cache_misses(), 1);
@@ -111,7 +120,7 @@ mod tests {
let mut cfg = Config::new();
cfg.cranelift_opt_level(OptLevel::Speed)
.cache_config_load(&config_path)?;
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
Module::new(&engine, "(module (func))")?;
assert_eq!(engine.config().cache_config.cache_hits(), 0);
assert_eq!(engine.config().cache_config.cache_misses(), 1);
@@ -122,7 +131,7 @@ mod tests {
let mut cfg = Config::new();
cfg.cranelift_opt_level(OptLevel::SpeedAndSize)
.cache_config_load(&config_path)?;
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
Module::new(&engine, "(module (func))")?;
assert_eq!(engine.config().cache_config.cache_hits(), 0);
assert_eq!(engine.config().cache_config.cache_misses(), 1);
@@ -134,7 +143,7 @@ mod tests {
if !cfg!(target_arch = "aarch64") {
let mut cfg = Config::new();
cfg.debug_info(true).cache_config_load(&config_path)?;
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
Module::new(&engine, "(module (func))")?;
assert_eq!(engine.config().cache_config.cache_hits(), 0);
assert_eq!(engine.config().cache_config.cache_misses(), 1);

File diff suppressed because it is too large Load Diff

View File

@@ -92,16 +92,16 @@ impl Instance {
///
/// # Panics
///
/// This function will panic if called within an asynchronous store
/// (created with [`Store::new_async`]).
/// This function will panic if called with a store associated with a
/// [`asynchronous config`](crate::Config::async_support).
///
/// [inst]: https://webassembly.github.io/spec/core/exec/modules.html#exec-instantiation
/// [issue]: https://github.com/bytecodealliance/wasmtime/issues/727
/// [`ExternType`]: crate::ExternType
pub fn new(store: &Store, module: &Module, imports: &[Extern]) -> Result<Instance, Error> {
assert!(
!store.is_async(),
"cannot instantiate synchronously within an asynchronous store"
!store.async_support(),
"cannot use `new` when async support is enabled on the config"
);
// NB: this is the same code as `Instance::new_async`. It's intentionally
@@ -127,11 +127,9 @@ impl Instance {
///
/// # Panics
///
/// This function will panic if called within a non-asynchronous store
/// (created with [`Store::new`]). This is only compatible with asynchronous
/// stores created with [`Store::new_async`].
///
/// [asynchronous stores]: Store::new_async
/// This function will panic if called with a store associated with a [`synchronous
/// config`](crate::Config::new). This is only compatible with stores associated with
/// an [`asynchronous config`](crate::Config::async_support).
#[cfg(feature = "async")]
#[cfg_attr(nightlydoc, doc(cfg(feature = "async")))]
pub async fn new_async(
@@ -140,8 +138,8 @@ impl Instance {
imports: &[Extern],
) -> Result<Instance, Error> {
assert!(
store.is_async(),
"cannot instantiate asynchronously within a synchronous store"
store.async_support(),
"cannot use `new_async` without enabling async support on the config"
);
// NB: this is the same code as `Instance::new`. It's intentionally
@@ -493,9 +491,8 @@ impl<'a> Instantiator<'a> {
self.store.register_module(&self.cur.module);
unsafe {
let config = self.store.engine().config();
let allocator = config.instance_allocator();
let engine = self.store.engine();
let allocator = engine.allocator();
let instance = allocator.allocate(InstanceAllocationRequest {
module: compiled_module.module().clone(),
@@ -522,7 +519,7 @@ impl<'a> Instantiator<'a> {
// initialization is successful, we need to keep the instance alive.
let instance = self.store.add_instance(instance, false);
allocator
.initialize(&instance.handle, config.features.bulk_memory)
.initialize(&instance.handle, engine.config().features.bulk_memory)
.map_err(|e| -> Error {
match e {
InstantiationError::Trap(trap) => {
@@ -551,7 +548,7 @@ impl<'a> Instantiator<'a> {
};
let vmctx_ptr = instance.handle.vmctx_ptr();
unsafe {
super::func::invoke_wasm_and_catch_traps(vmctx_ptr, &instance.store, || {
super::func::invoke_wasm_and_catch_traps(&instance.store, || {
mem::transmute::<
*const VMFunctionBody,
unsafe extern "C" fn(*mut VMContext, *mut VMContext),

View File

@@ -275,11 +275,13 @@
#![cfg_attr(nightlydoc, feature(doc_cfg))]
#![cfg_attr(not(feature = "default"), allow(dead_code, unused_imports))]
#[macro_use]
mod func;
mod config;
mod engine;
mod externals;
mod frame_info;
mod func;
mod instance;
mod linker;
mod memory;

View File

@@ -518,6 +518,19 @@ impl Linker {
o.insert(item);
}
Entry::Vacant(v) => {
// If shadowing is not allowed, check for an existing host function
if !self.allow_shadowing {
if let Extern::Func(_) = &item {
if self.store.get_host_func(module, name).is_some() {
bail!(
"import of `{}::{}` with kind {:?} defined twice",
module,
name,
v.key().kind,
)
}
}
}
v.insert(item);
}
}
@@ -676,45 +689,45 @@ impl Linker {
///
/// Returns `None` if no match was found.
pub fn get(&self, import: &ImportType) -> Option<Extern> {
let key = ImportKey {
module: *self.string2idx.get(import.module())?,
name: match import.name() {
Some(name) => *self.string2idx.get(name)?,
None => usize::max_value(),
},
kind: self.import_kind(import.ty()),
};
if let Some(result) = self.map.get(&key).cloned() {
return Some(result);
if let Some(ext) = self.get_extern(import) {
return Some(ext);
}
// This is a key location where the module linking proposal is
// implemented. This logic allows single-level imports of an instance to
// get satisfied by multiple definitions of items within this `Linker`.
//
// The instance being import is iterated over to load the names from
// this `Linker` (recursively calling `get`). If anything isn't defined
// we return `None` since the entire value isn't defined. Otherwise when
// all values are loaded it's assembled into an `Instance` and
// returned`.
//
// Note that this isn't exactly the speediest implementation in the
// world. Ideally we would pre-create the `Instance` instead of creating
// it each time a module is instantiated. For now though while the
// module linking proposal is under development this should hopefully
// suffice.
if let ExternType::Instance(t) = import.ty() {
if import.name().is_none() {
let mut builder = InstanceBuilder::new();
for export in t.exports() {
let item = self.get(&export.as_import(import.module()))?;
builder.insert(export.name(), item);
match import.ty() {
// For function imports, check with the store for a host func
ExternType::Func(_) => self
.store
.get_host_func(import.module(), import.name()?)
.map(Into::into),
ExternType::Instance(t) => {
// This is a key location where the module linking proposal is
// implemented. This logic allows single-level imports of an instance to
// get satisfied by multiple definitions of items within this `Linker`.
//
// The instance being import is iterated over to load the names from
// this `Linker` (recursively calling `get`). If anything isn't defined
// we return `None` since the entire value isn't defined. Otherwise when
// all values are loaded it's assembled into an `Instance` and
// returned`.
//
// Note that this isn't exactly the speediest implementation in the
// world. Ideally we would pre-create the `Instance` instead of creating
// it each time a module is instantiated. For now though while the
// module linking proposal is under development this should hopefully
// suffice.
if import.name().is_none() {
let mut builder = InstanceBuilder::new();
for export in t.exports() {
let item = self.get(&export.as_import(import.module()))?;
builder.insert(export.name(), item);
}
Some(builder.finish(&self.store).into())
} else {
None
}
return Some(builder.finish(&self.store).into());
}
_ => None,
}
None
}
/// Returns all items defined for the `module` and `name` pair.
@@ -791,6 +804,18 @@ impl Linker {
move |_, _, _| Ok(()),
))
}
fn get_extern(&self, import: &ImportType) -> Option<Extern> {
let key = ImportKey {
module: *self.string2idx.get(import.module())?,
name: match import.name() {
Some(name) => *self.string2idx.get(name)?,
None => usize::max_value(),
},
kind: self.import_kind(import.ty()),
};
self.map.get(&key).cloned()
}
}
/// Modules can be interpreted either as Commands or Reactors.

View File

@@ -566,7 +566,7 @@ mod tests {
let mut cfg = Config::new();
cfg.static_memory_maximum_size(0)
.dynamic_memory_guard_size(0);
let store = Store::new(&Engine::new(&cfg));
let store = Store::new(&Engine::new(&cfg).unwrap());
let ty = MemoryType::new(Limits::new(1, None));
let mem = Memory::new(&store, ty);
assert_eq!(mem.wasmtime_export.memory.offset_guard_size, 0);

View File

@@ -332,10 +332,7 @@ impl Module {
let module = modules.remove(main_module);
// Validate the module can be used with the current allocator
engine
.config()
.instance_allocator()
.validate(module.module())?;
engine.allocator().validate(module.module())?;
Ok(Module {
inner: Arc::new(ModuleInner {

View File

@@ -1,34 +1,36 @@
use crate::frame_info::StoreFrameInfo;
use crate::sig_registry::SignatureRegistry;
use crate::trampoline::StoreInstanceHandle;
use crate::{Engine, Module, Trap};
use crate::{Engine, Func, FuncType, Module, Trap};
use anyhow::{bail, Result};
use std::any::Any;
use std::any::{Any, TypeId};
use std::cell::{Cell, RefCell};
use std::collections::HashSet;
use std::collections::{hash_map::Entry, HashMap, HashSet};
use std::convert::TryFrom;
use std::fmt;
use std::future::Future;
use std::hash::{Hash, Hasher};
use std::pin::Pin;
use std::ptr;
use std::rc::{Rc, Weak};
use std::rc::Rc;
use std::sync::Arc;
use std::task::{Context, Poll};
use wasmtime_environ::wasm;
use wasmtime_jit::{CompiledModule, ModuleCode, TypeTables};
use wasmtime_runtime::{
InstanceAllocator, InstanceHandle, SignalHandler, StackMapRegistry, TrapInfo, VMContext,
VMExternRef, VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex,
Export, InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler,
StackMapRegistry, TrapInfo, VMCallerCheckedAnyfunc, VMContext, VMExternRef,
VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex, VMTrampoline,
};
/// Used to associate instances with the store.
///
/// This is needed to track if the instance was allocated expliclty with the default
/// This is needed to track if the instance was allocated explicitly with the on-demand
/// instance allocator.
struct StoreInstance {
handle: InstanceHandle,
use_default_allocator: bool,
// Stores whether or not to use the on-demand allocator to deallocate the instance
ondemand: bool,
}
/// A `Store` is a collection of WebAssembly instances and host-defined items.
@@ -68,8 +70,9 @@ pub struct Store {
}
pub(crate) struct StoreInner {
is_async: bool,
engine: Engine,
/// The map of all host functions registered with this store's signature registry
host_funcs: RefCell<HashMap<InstanceHandle, Box<VMCallerCheckedAnyfunc>>>,
interrupts: Arc<VMInterrupts>,
signatures: RefCell<SignatureRegistry>,
instances: RefCell<Vec<StoreInstance>>,
@@ -82,21 +85,19 @@ pub(crate) struct StoreInner {
/// Set of all compiled modules that we're holding a strong reference to
/// the module's code for. This includes JIT functions, trampolines, etc.
modules: RefCell<HashSet<ArcModuleCode>>,
// Numbers of resources instantiated in this store.
instance_count: Cell<usize>,
memory_count: Cell<usize>,
table_count: Cell<usize>,
/// An adjustment to add to the fuel consumed value in `interrupts` above
/// to get the true amount of fuel consumed.
fuel_adj: Cell<i64>,
#[cfg(feature = "async")]
current_suspend: Cell<*const wasmtime_fiber::Suspend<Result<(), Trap>, (), Result<(), Trap>>>,
#[cfg(feature = "async")]
current_poll_cx: Cell<*mut Context<'static>>,
out_of_gas_behavior: Cell<OutOfGas>,
context_values: RefCell<HashMap<TypeId, Box<dyn Any>>>,
}
#[derive(Copy, Clone)]
@@ -129,99 +130,7 @@ impl Hash for HostInfoKey {
impl Store {
/// Creates a new store to be associated with the given [`Engine`].
///
/// Note that this `Store` cannot be used with asynchronous host calls nor
/// can it be used to call functions asynchronously. For that you'll want to
/// use [`Store::new_async`].
pub fn new(engine: &Engine) -> Store {
Store::_new(engine, false)
}
/// Creates a new async store to be associated with the given [`Engine`].
///
/// The returned store can optionally define host functions with `async`.
/// Instances created and functions called within the returned `Store`
/// *must* be called through their asynchronous APIs, however. For example
/// using [`Func::call`](crate::Func::call) will panic in the returned
/// store.
///
/// # Asynchronous Wasm
///
/// WebAssembly does not currently have a way to specify at the bytecode
/// level what is and isn't async. Host-defined functions, however, may be
/// defined as `async`. WebAssembly imports always appear synchronous, which
/// gives rise to a bit of an impedence mismatch here. To solve this
/// Wasmtime supports "asynchronous stores" which enables calling these
/// asynchronous functions in a way that looks synchronous to the executing
/// WebAssembly code.
///
/// An asynchronous store must always invoke wasm code asynchronously,
/// meaning we'll always represent its computation as a
/// [`Future`](std::future::Future). The `poll` method of the futures
/// returned by Wasmtime will perform the actual work of calling the
/// WebAssembly. Wasmtime won't manage its own thread pools or similar,
/// that's left up to the embedder.
///
/// To implement futures in a way that WebAssembly sees asynchronous host
/// functions as synchronous, all async Wasmtime futures will execute on a
/// separately allocated native stack from the thread otherwise executing
/// Wasmtime. This separate native stack can then be switched to and from.
/// Using this whenever an `async` host function returns a future that
/// resolves to `Pending` we switch away from the temporary stack back to
/// the main stack and propagate the `Pending` status.
///
/// In general it's encouraged that the integration with `async` and
/// wasmtime is designed early on in your embedding of Wasmtime to ensure
/// that it's planned that WebAssembly executes in the right context of your
/// application.
///
/// # Execution in `poll`
///
/// The [`Future::poll`](std::future::Future::poll) method is the main
/// driving force behind Rust's futures. That method's own documentation
/// states "an implementation of `poll` should strive to return quickly, and
/// should not block". This, however, can be at odds with executing
/// WebAssembly code as part of the `poll` method itself. If your
/// WebAssembly is untrusted then this could allow the `poll` method to take
/// arbitrarily long in the worst case, likely blocking all other
/// asynchronous tasks.
///
/// To remedy this situation you have a two possible ways to solve this:
///
/// * First you can spawn futures into a thread pool. Care must be taken for
/// this because Wasmtime futures are not `Send` or `Sync`. If you ensure
/// that the entire state of a `Store` is wrapped up in a single future,
/// though, you can send the whole future at once to a separate thread. By
/// doing this in a thread pool you are relaxing the requirement that
/// `Future::poll` must be fast because your future is executing on a
/// separate thread. This strategy, however, would likely still require
/// some form of cancellation via [`Store::interrupt_handle`] to ensure
/// wasm doesn't take *too* long to execute.
///
/// * Alternatively you can enable the
/// [`Config::consume_fuel`](crate::Config::consume_fuel) method as well
/// as [`Store::out_of_fuel_async_yield`] When doing so this will
/// configure Wasmtime futures to yield periodically while they're
/// executing WebAssembly code. After consuming the specified amount of
/// fuel wasm futures will return `Poll::Pending` from their `poll`
/// method, and will get automatically re-polled later. This enables the
/// `Future::poll` method to take roughly a fixed amount of time since
/// fuel is guaranteed to get consumed while wasm is executing. Note that
/// to prevent infinite execution of wasm you'll still need to use
/// [`Store::interrupt_handle`].
///
/// In either case special care needs to be taken when integrating
/// asynchronous wasm into your application. You should carefully plan where
/// WebAssembly will execute and what compute resources will be allotted to
/// it. If Wasmtime doesn't support exactly what you'd like just yet, please
/// feel free to open an issue!
#[cfg(feature = "async")]
#[cfg_attr(nightlydoc, doc(cfg(feature = "async")))]
pub fn new_async(engine: &Engine) -> Store {
Store::_new(engine, true)
}
fn _new(engine: &Engine, is_async: bool) -> Store {
// Ensure that wasmtime_runtime's signal handlers are configured. Note
// that at the `Store` level it means we should perform this
// once-per-thread. Platforms like Unix, however, only require this
@@ -231,8 +140,8 @@ impl Store {
Store {
inner: Rc::new(StoreInner {
is_async,
engine: engine.clone(),
host_funcs: RefCell::new(HashMap::new()),
interrupts: Arc::new(Default::default()),
signatures: RefCell::new(Default::default()),
instances: RefCell::new(Vec::new()),
@@ -250,12 +159,53 @@ impl Store {
#[cfg(feature = "async")]
current_poll_cx: Cell::new(ptr::null_mut()),
out_of_gas_behavior: Cell::new(OutOfGas::Trap),
context_values: RefCell::new(HashMap::new()),
}),
}
}
pub(crate) fn from_inner(inner: Rc<StoreInner>) -> Store {
Store { inner }
/// Gets a host function from the [`Config`](crate::Config) associated with this [`Store`].
///
/// Returns `None` if the given host function is not defined.
pub fn get_host_func(&self, module: &str, name: &str) -> Option<Func> {
self.inner
.engine
.config()
.get_host_func(module, name)
.map(|f| {
// This call is safe because we know the function is coming from the
// config associated with this store
unsafe { f.to_func(self) }
})
}
pub(crate) fn get_host_anyfunc(
&self,
instance: &InstanceHandle,
ty: &FuncType,
trampoline: VMTrampoline,
) -> *mut VMCallerCheckedAnyfunc {
let mut funcs = self.inner.host_funcs.borrow_mut();
let anyfunc = funcs.entry(unsafe { instance.clone() }).or_insert_with(|| {
let mut anyfunc = match instance
.lookup_by_declaration(&wasm::EntityIndex::Function(wasm::FuncIndex::from_u32(0)))
{
Export::Function(f) => unsafe { f.anyfunc.as_ref() }.clone(),
_ => unreachable!(),
};
// Register the function with this store's signature registry
anyfunc.type_index = self
.inner
.signatures
.borrow_mut()
.register(ty.as_wasm_func_type(), trampoline);
Box::new(anyfunc)
});
&mut **anyfunc
}
/// Returns the [`Engine`] that this store is associated with.
@@ -263,6 +213,34 @@ impl Store {
&self.inner.engine
}
/// Gets a context value from the store.
///
/// Returns a reference to the context value if present.
pub fn get<T: Any>(&self) -> Option<&T> {
let values = self.inner.context_values.borrow();
// Safety: a context value cannot be removed once added and therefore the addres is
// stable for the life of the store
values
.get(&TypeId::of::<T>())
.map(|v| unsafe { &*(v.downcast_ref::<T>().unwrap() as *const T) })
}
/// Sets a context value into the store.
///
/// Returns the given value as an error if an existing value is already set.
pub fn set<T: Any>(&self, value: T) -> Result<(), T> {
let mut values = self.inner.context_values.borrow_mut();
match values.entry(value.type_id()) {
Entry::Occupied(_) => Err(value),
Entry::Vacant(v) => {
v.insert(Box::new(value));
Ok(())
}
}
}
pub(crate) fn signatures(&self) -> &RefCell<SignatureRegistry> {
&self.inner.signatures
}
@@ -386,11 +364,11 @@ impl Store {
pub(crate) unsafe fn add_instance(
&self,
handle: InstanceHandle,
use_default_allocator: bool,
ondemand: bool,
) -> StoreInstanceHandle {
self.inner.instances.borrow_mut().push(StoreInstance {
handle: handle.clone(),
use_default_allocator,
ondemand,
});
StoreInstanceHandle {
store: self.clone(),
@@ -399,12 +377,14 @@ impl Store {
}
pub(crate) fn existing_instance_handle(&self, handle: InstanceHandle) -> StoreInstanceHandle {
debug_assert!(self
.inner
.instances
.borrow()
.iter()
.any(|i| i.handle.vmctx_ptr() == handle.vmctx_ptr()));
debug_assert!(
self.inner
.instances
.borrow()
.iter()
.any(|i| i.handle.vmctx_ptr() == handle.vmctx_ptr())
|| self.inner.host_funcs.borrow().get(&handle).is_some()
);
StoreInstanceHandle {
store: self.clone(),
handle,
@@ -415,15 +395,6 @@ impl Store {
self.existing_instance_handle(InstanceHandle::from_vmctx(cx))
}
pub(crate) fn weak(&self) -> Weak<StoreInner> {
Rc::downgrade(&self.inner)
}
pub(crate) fn upgrade(weak: &Weak<StoreInner>) -> Option<Self> {
let inner = weak.upgrade()?;
Some(Self { inner })
}
pub(crate) fn set_signal_handler(&self, handler: Option<Box<SignalHandler<'static>>>) {
*self.inner.signal_handler.borrow_mut() = handler;
}
@@ -499,7 +470,7 @@ impl Store {
/// # fn main() -> Result<()> {
/// // Enable interruptable code via `Config` and then create an interrupt
/// // handle which we'll use later to interrupt running code.
/// let engine = Engine::new(Config::new().interruptable(true));
/// let engine = Engine::new(Config::new().interruptable(true))?;
/// let store = Store::new(&engine);
/// let interrupt_handle = store.interrupt_handle()?;
///
@@ -648,8 +619,8 @@ impl Store {
/// [`Config::consume_fuel`](crate::Config::consume_fuel) this method will
/// configure what happens when fuel runs out. Specifically executing
/// WebAssembly will be suspended and control will be yielded back to the
/// caller. This is only suitable with use of [async
/// stores](Store::new_async) because only then are futures used and yields
/// caller. This is only suitable with use of a store associated with an [async
/// config](crate::Config::async_support) because only then are futures used and yields
/// are possible.
///
/// The purpose of this behavior is to ensure that futures which represent
@@ -673,18 +644,21 @@ impl Store {
///
/// # Panics
///
/// This method will panic if it is not called on an [async
/// store](Store::new_async).
/// This method will panic if it is not called on a store associated with an [async
/// config](crate::Config::async_support).
pub fn out_of_fuel_async_yield(&self, injection_count: u32, fuel_to_inject: u64) {
assert!(self.is_async());
assert!(
self.async_support(),
"cannot use `out_of_fuel_async_yield` without enabling async support in the config"
);
self.inner.out_of_gas_behavior.set(OutOfGas::InjectFuel {
injection_count,
fuel_to_inject,
});
}
pub(crate) fn is_async(&self) -> bool {
self.inner.is_async
pub(crate) fn async_support(&self) -> bool {
self.inner.engine.config().async_support
}
/// Blocks on the asynchronous computation represented by `future` and
@@ -712,7 +686,7 @@ impl Store {
&self,
mut future: Pin<&mut dyn Future<Output = T>>,
) -> Result<T, Trap> {
debug_assert!(self.is_async());
debug_assert!(self.async_support());
// Take our current `Suspend` context which was configured as soon as
// our fiber started. Note that we must load it at the front here and
@@ -761,7 +735,7 @@ impl Store {
pub(crate) async fn on_fiber<R>(&self, func: impl FnOnce() -> R) -> Result<R, Trap> {
let config = self.inner.engine.config();
debug_assert!(self.is_async());
debug_assert!(self.async_support());
debug_assert!(config.async_stack_size > 0);
type SuspendType = wasmtime_fiber::Suspend<Result<(), Trap>, (), Result<(), Trap>>;
@@ -786,7 +760,7 @@ impl Store {
Ok(())
};
let (fiber, stack) = match config.instance_allocator().allocate_fiber_stack() {
let (fiber, stack) = match self.inner.engine.allocator().allocate_fiber_stack() {
Ok(stack) => {
// Use the returned stack and deallocate it when finished
(
@@ -892,8 +866,7 @@ impl Store {
unsafe {
self.store
.engine()
.config()
.instance_allocator()
.allocator()
.deallocate_fiber_stack(self.stack)
};
}
@@ -1002,6 +975,10 @@ unsafe impl TrapInfo for Store {
OutOfGas::InjectFuel { .. } => unreachable!(),
}
}
fn interrupts(&self) -> &VMInterrupts {
&self.inner.interrupts
}
}
impl Default for Store {
@@ -1019,14 +996,12 @@ impl fmt::Debug for Store {
impl Drop for StoreInner {
fn drop(&mut self) {
let allocator = self.engine.config().instance_allocator();
let allocator = self.engine.allocator();
let ondemand = OnDemandInstanceAllocator::new(self.engine.config().mem_creator.clone());
for instance in self.instances.borrow().iter() {
unsafe {
if instance.use_default_allocator {
self.engine
.config()
.default_instance_allocator
.deallocate(&instance.handle);
if instance.ondemand {
ondemand.deallocate(&instance.handle);
} else {
allocator.deallocate(&instance.handle);
}

View File

@@ -1,6 +1,5 @@
//! Utility module to create trampolines in/out WebAssembly module.
mod create_handle;
mod func;
mod global;
mod memory;
@@ -8,16 +7,21 @@ mod table;
pub(crate) use memory::MemoryCreatorProxy;
use self::func::create_handle_with_function;
pub use self::func::{create_function, create_raw_function};
use self::global::create_global;
use self::memory::create_handle_with_memory;
use self::table::create_handle_with_table;
use crate::{FuncType, GlobalType, MemoryType, Store, TableType, Trap, Val};
use self::memory::create_memory;
use self::table::create_table;
use crate::{GlobalType, MemoryType, Store, TableType, Val};
use anyhow::Result;
use std::any::Any;
use std::ops::Deref;
use wasmtime_environ::wasm;
use wasmtime_runtime::{InstanceHandle, VMContext, VMFunctionBody, VMTrampoline};
use std::sync::Arc;
use wasmtime_environ::{entity::PrimaryMap, wasm, Module};
use wasmtime_runtime::{
Imports, InstanceAllocationRequest, InstanceAllocator, InstanceHandle,
OnDemandInstanceAllocator, StackMapRegistry, VMExternRefActivationsTable, VMFunctionBody,
VMFunctionImport, VMSharedSignatureIndex,
};
/// A wrapper around `wasmtime_runtime::InstanceHandle` which pairs it with the
/// `Store` that it's rooted within. The instance is deallocated when `Store` is
@@ -46,38 +50,36 @@ impl Deref for StoreInstanceHandle {
}
}
pub fn generate_func_export(
ft: &FuncType,
func: Box<dyn Fn(*mut VMContext, *mut u128) -> Result<(), Trap>>,
fn create_handle(
module: Module,
store: &Store,
) -> Result<(
StoreInstanceHandle,
wasmtime_runtime::ExportFunction,
VMTrampoline,
)> {
let (instance, trampoline) = create_handle_with_function(ft, func, store)?;
let idx = wasm::EntityIndex::Function(wasm::FuncIndex::from_u32(0));
match instance.lookup_by_declaration(&idx) {
wasmtime_runtime::Export::Function(f) => Ok((instance, f, trampoline)),
_ => unreachable!(),
}
}
finished_functions: PrimaryMap<wasm::DefinedFuncIndex, *mut [VMFunctionBody]>,
host_state: Box<dyn Any>,
func_imports: &[VMFunctionImport],
shared_signature_id: Option<VMSharedSignatureIndex>,
) -> Result<StoreInstanceHandle> {
let mut imports = Imports::default();
imports.functions = func_imports;
/// Note that this is `unsafe` since `func` must be a valid function pointer and
/// have a signature which matches `ft`, otherwise the returned
/// instance/export/etc may exhibit undefined behavior.
pub unsafe fn generate_raw_func_export(
ft: &FuncType,
func: *mut [VMFunctionBody],
trampoline: VMTrampoline,
store: &Store,
state: Box<dyn Any>,
) -> Result<(StoreInstanceHandle, wasmtime_runtime::ExportFunction)> {
let instance = func::create_handle_with_raw_function(ft, func, trampoline, store, state)?;
let idx = wasm::EntityIndex::Function(wasm::FuncIndex::from_u32(0));
match instance.lookup_by_declaration(&idx) {
wasmtime_runtime::Export::Function(f) => Ok((instance, f)),
_ => unreachable!(),
unsafe {
// Use the on-demand allocator when creating handles associated with host objects
// The configured instance allocator should only be used when creating module instances
// as we don't want host objects to count towards instance limits.
let handle = OnDemandInstanceAllocator::new(store.engine().config().mem_creator.clone())
.allocate(InstanceAllocationRequest {
module: Arc::new(module),
finished_functions: &finished_functions,
imports,
lookup_shared_signature: &|_| shared_signature_id.unwrap(),
host_state,
interrupts: store.interrupts(),
externref_activations_table: store.externref_activations_table()
as *const VMExternRefActivationsTable
as *mut _,
stack_map_registry: store.stack_map_registry() as *const StackMapRegistry as *mut _,
})?;
Ok(store.add_instance(handle, true))
}
}
@@ -98,7 +100,7 @@ pub fn generate_memory_export(
store: &Store,
m: &MemoryType,
) -> Result<(StoreInstanceHandle, wasmtime_runtime::ExportMemory)> {
let instance = create_handle_with_memory(store, m)?;
let instance = create_memory(store, m)?;
let idx = wasm::EntityIndex::Memory(wasm::MemoryIndex::from_u32(0));
match instance.lookup_by_declaration(&idx) {
wasmtime_runtime::Export::Memory(m) => Ok((instance, m)),
@@ -110,7 +112,7 @@ pub fn generate_table_export(
store: &Store,
t: &TableType,
) -> Result<(StoreInstanceHandle, wasmtime_runtime::ExportTable)> {
let instance = create_handle_with_table(store, t)?;
let instance = create_table(store, t)?;
let idx = wasm::EntityIndex::Table(wasm::TableIndex::from_u32(0));
match instance.lookup_by_declaration(&idx) {
wasmtime_runtime::Export::Table(t) => Ok((instance, t)),

View File

@@ -1,16 +1,15 @@
//! Support for a calling of an imported function.
use super::create_handle::create_handle;
use crate::trampoline::StoreInstanceHandle;
use crate::{FuncType, Store, Trap};
use crate::{sig_registry::SignatureRegistry, Config, FuncType, Trap};
use anyhow::Result;
use std::any::Any;
use std::cmp;
use std::mem;
use std::panic::{self, AssertUnwindSafe};
use std::sync::Arc;
use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::isa::TargetIsa;
use wasmtime_environ::wasm::SignatureIndex;
use wasmtime_environ::wasm::{DefinedFuncIndex, SignatureIndex};
use wasmtime_environ::{ir, wasm, CompiledFunction, Module, ModuleType};
use wasmtime_jit::trampoline::ir::{
ExternalName, Function, InstBuilder, MemFlags, StackSlotData, StackSlotKind,
@@ -19,7 +18,10 @@ use wasmtime_jit::trampoline::{
self, binemit, pretty_error, Context, FunctionBuilder, FunctionBuilderContext,
};
use wasmtime_jit::CodeMemory;
use wasmtime_runtime::{InstanceHandle, VMContext, VMFunctionBody, VMTrampoline};
use wasmtime_runtime::{
Imports, InstanceAllocationRequest, InstanceAllocator, InstanceHandle,
OnDemandInstanceAllocator, VMContext, VMFunctionBody, VMSharedSignatureIndex, VMTrampoline,
};
struct TrampolineState {
func: Box<dyn Fn(*mut VMContext, *mut u128) -> Result<(), Trap>>,
@@ -203,19 +205,23 @@ fn make_trampoline(
.expect("allocate_for_function")
}
pub fn create_handle_with_function(
fn create_function_trampoline(
config: &Config,
ft: &FuncType,
func: Box<dyn Fn(*mut VMContext, *mut u128) -> Result<(), Trap>>,
store: &Store,
) -> Result<(StoreInstanceHandle, VMTrampoline)> {
) -> Result<(
Module,
PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
VMTrampoline,
TrampolineState,
)> {
// Note that we specifically enable reference types here in our ISA because
// `Func::new` is intended to be infallible, but our signature may use
// reference types which requires safepoints.
let isa = store.engine().config().target_isa_with_reference_types();
let isa = config.target_isa_with_reference_types();
let pointer_type = isa.pointer_type();
let sig = ft.get_wasmtime_signature(pointer_type);
let wft = ft.as_wasm_func_type();
let mut fn_builder_ctx = FunctionBuilderContext::new();
let mut module = Module::new();
@@ -226,10 +232,12 @@ pub fn create_handle_with_function(
// and calls into `stub_fn`...
let sig_id = SignatureIndex::from_u32(u32::max_value() - 1);
module.types.push(ModuleType::Function(sig_id));
let func_id = module.functions.push(sig_id);
module
.exports
.insert(String::new(), wasm::EntityIndex::Function(func_id));
let trampoline = make_trampoline(isa.as_ref(), &mut code_memory, &mut fn_builder_ctx, &sig);
finished_functions.push(trampoline);
@@ -243,33 +251,51 @@ pub fn create_handle_with_function(
&sig,
mem::size_of::<u128>(),
)?;
let shared_signature_id = store.signatures().borrow_mut().register(wft, trampoline);
// Next up we wrap everything up into an `InstanceHandle` by publishing our
// code memory (makes it executable) and ensuring all our various bits of
// state make it into the instance constructors.
code_memory.publish(isa.as_ref());
let trampoline_state = TrampolineState { func, code_memory };
create_handle(
module,
store,
finished_functions,
Box::new(trampoline_state),
&[],
Some(shared_signature_id),
)
.map(|instance| (instance, trampoline))
Ok((module, finished_functions, trampoline, trampoline_state))
}
pub unsafe fn create_handle_with_raw_function(
pub fn create_function(
ft: &FuncType,
func: *mut [VMFunctionBody],
trampoline: VMTrampoline,
store: &Store,
state: Box<dyn Any>,
) -> Result<StoreInstanceHandle> {
let wft = ft.as_wasm_func_type();
func: Box<dyn Fn(*mut VMContext, *mut u128) -> Result<(), Trap>>,
config: &Config,
registry: Option<&mut SignatureRegistry>,
) -> Result<(InstanceHandle, VMTrampoline)> {
let (module, finished_functions, trampoline, trampoline_state) =
create_function_trampoline(config, ft, func)?;
// If there is no signature registry, use the default signature index which is
// guaranteed to trap if there is ever an indirect call on the function (should not happen)
let shared_signature_id = registry
.map(|r| r.register(ft.as_wasm_func_type(), trampoline))
.unwrap_or(VMSharedSignatureIndex::default());
unsafe {
Ok((
OnDemandInstanceAllocator::new(None).allocate(InstanceAllocationRequest {
module: Arc::new(module),
finished_functions: &finished_functions,
imports: Imports::default(),
lookup_shared_signature: &|_| shared_signature_id,
host_state: Box::new(trampoline_state),
interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(),
stack_map_registry: std::ptr::null_mut(),
})?,
trampoline,
))
}
}
pub unsafe fn create_raw_function(
func: *mut [VMFunctionBody],
host_state: Box<dyn Any>,
shared_signature_id: VMSharedSignatureIndex,
) -> Result<InstanceHandle> {
let mut module = Module::new();
let mut finished_functions = PrimaryMap::new();
@@ -280,14 +306,17 @@ pub unsafe fn create_handle_with_raw_function(
.exports
.insert(String::new(), wasm::EntityIndex::Function(func_id));
finished_functions.push(func);
let shared_signature_id = store.signatures().borrow_mut().register(wft, trampoline);
create_handle(
module,
store,
finished_functions,
state,
&[],
Some(shared_signature_id),
Ok(
OnDemandInstanceAllocator::new(None).allocate(InstanceAllocationRequest {
module: Arc::new(module),
finished_functions: &finished_functions,
imports: Imports::default(),
lookup_shared_signature: &|_| shared_signature_id,
host_state,
interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(),
stack_map_registry: std::ptr::null_mut(),
})?,
)
}

View File

@@ -1,5 +1,4 @@
use super::create_handle::create_handle;
use crate::trampoline::StoreInstanceHandle;
use crate::trampoline::{create_handle, StoreInstanceHandle};
use crate::{GlobalType, Mutability, Store, Val};
use anyhow::Result;
use wasmtime_environ::entity::PrimaryMap;

View File

@@ -1,6 +1,5 @@
use super::create_handle::create_handle;
use crate::memory::{LinearMemory, MemoryCreator};
use crate::trampoline::StoreInstanceHandle;
use crate::trampoline::{create_handle, StoreInstanceHandle};
use crate::Store;
use crate::{Limits, MemoryType};
use anyhow::{anyhow, Result};
@@ -10,10 +9,7 @@ use wasmtime_runtime::{RuntimeLinearMemory, RuntimeMemoryCreator, VMMemoryDefini
use std::sync::Arc;
pub fn create_handle_with_memory(
store: &Store,
memory: &MemoryType,
) -> Result<StoreInstanceHandle> {
pub fn create_memory(store: &Store, memory: &MemoryType) -> Result<StoreInstanceHandle> {
let mut module = Module::new();
let memory = wasm::Memory {

View File

@@ -1,12 +1,11 @@
use super::create_handle::create_handle;
use crate::trampoline::StoreInstanceHandle;
use crate::trampoline::{create_handle, StoreInstanceHandle};
use crate::Store;
use crate::{TableType, ValType};
use anyhow::{bail, Result};
use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::{wasm, Module};
pub fn create_handle_with_table(store: &Store, table: &TableType) -> Result<StoreInstanceHandle> {
pub fn create_table(store: &Store, table: &TableType) -> Result<StoreInstanceHandle> {
let mut module = Module::new();
let table = wasm::Table {

View File

@@ -1,6 +1,6 @@
use proc_macro::TokenStream;
use proc_macro2::{Ident, Span, TokenStream as TokenStream2};
use quote::quote;
use quote::{format_ident, quote};
use syn::parse_macro_input;
use wiggle_generate::Names;
@@ -102,14 +102,23 @@ fn generate_module(
let module_id = names.module(&module.name);
let target_module = quote! { #target_path::#module_id };
let ctor_externs = module.funcs().map(|f| {
let mut fns = Vec::new();
let mut ctor_externs = Vec::new();
let mut host_funcs = Vec::new();
for f in module.funcs() {
generate_func(
&module_id,
&f,
names,
&target_module,
ctx_type,
async_conf.is_async(module.name.as_str(), f.name.as_str()),
)
});
&mut fns,
&mut ctor_externs,
&mut host_funcs,
);
}
let type_name = module_conf.name.clone();
let type_docs = module_conf
@@ -121,7 +130,7 @@ fn generate_module(
"Creates a new [`{}`] instance.
External values are allocated into the `store` provided and
configuration of the wasi instance itself should be all
configuration of the instance itself should be all
contained in the `cx` parameter.",
module_conf.name.to_string()
);
@@ -134,7 +143,7 @@ contained in the `cx` parameter.",
impl #type_name {
#[doc = #constructor_docs]
pub fn new(store: &wasmtime::Store, cx: std::rc::Rc<std::cell::RefCell<#ctx_type>>) -> Self {
pub fn new(store: &wasmtime::Store, ctx: std::rc::Rc<std::cell::RefCell<#ctx_type>>) -> Self {
#(#ctor_externs)*
Self {
@@ -159,16 +168,47 @@ contained in the `cx` parameter.",
#(#linker_add)*
Ok(())
}
/// Adds the host functions to the given [`wasmtime::Config`].
///
/// Host functions added to the config expect [`set_context`] to be called.
///
/// Host functions will trap if the context is not set in the calling [`wasmtime::Store`].
pub fn add_to_config(config: &mut wasmtime::Config) {
#(#host_funcs)*
}
/// Sets the context in the given store.
///
/// Context must be set in the store when using [`add_to_config`] and prior to any
/// host function being called.
///
/// If the context is already set in the store, the given context is returned as an error.
pub fn set_context(store: &wasmtime::Store, ctx: #ctx_type) -> Result<(), #ctx_type> {
store.set(std::rc::Rc::new(std::cell::RefCell::new(ctx))).map_err(|ctx| {
match std::rc::Rc::try_unwrap(ctx) {
Ok(ctx) => ctx.into_inner(),
Err(_) => unreachable!(),
}
})
}
#(#fns)*
}
}
}
fn generate_func(
module_ident: &Ident,
func: &witx::InterfaceFunc,
names: &Names,
target_module: &TokenStream2,
ctx_type: &syn::Type,
is_async: bool,
) -> TokenStream2 {
fns: &mut Vec<TokenStream2>,
ctors: &mut Vec<TokenStream2>,
host_funcs: &mut Vec<TokenStream2>,
) {
let name_ident = names.func(&func.name);
let (params, results) = func.wasm_signature();
@@ -176,11 +216,15 @@ fn generate_func(
let arg_names = (0..params.len())
.map(|i| Ident::new(&format!("arg{}", i), Span::call_site()))
.collect::<Vec<_>>();
let arg_decls = params.iter().enumerate().map(|(i, ty)| {
let name = &arg_names[i];
let wasm = names.wasm_type(*ty);
quote! { #name: #wasm }
});
let arg_decls = params
.iter()
.enumerate()
.map(|(i, ty)| {
let name = &arg_names[i];
let wasm = names.wasm_type(*ty);
quote! { #name: #wasm }
})
.collect::<Vec<_>>();
let ret_ty = match results.len() {
0 => quote!(()),
@@ -188,54 +232,87 @@ fn generate_func(
_ => unimplemented!(),
};
let runtime = names.runtime_mod();
let async_ = if is_async { quote!(async) } else { quote!() };
let await_ = if is_async { quote!(.await) } else { quote!() };
let closure_body = quote! {
unsafe {
let mem = match caller.get_export("memory") {
Some(wasmtime::Extern::Memory(m)) => m,
_ => {
return Err(wasmtime::Trap::new("missing required memory export"));
}
};
let mem = #runtime::WasmtimeGuestMemory::new(mem);
let result = #target_module::#name_ident(
&mut *my_cx.borrow_mut(),
&mem,
#(#arg_names),*
) #await_;
match result {
Ok(r) => Ok(r.into()),
Err(wasmtime_wiggle::Trap::String(err)) => Err(wasmtime::Trap::new(err)),
Err(wasmtime_wiggle::Trap::I32Exit(err)) => Err(wasmtime::Trap::i32_exit(err)),
}
}
let runtime = names.runtime_mod();
let fn_ident = format_ident!("{}_{}", module_ident, name_ident);
};
if is_async {
let wrapper = quote::format_ident!("wrap{}_async", params.len());
quote! {
let #name_ident = wasmtime::Func::#wrapper(
store,
cx.clone(),
move |caller: wasmtime::Caller<'_>, my_cx: &Rc<RefCell<_>> #(,#arg_decls)*|
-> Box<dyn std::future::Future<Output = Result<#ret_ty, wasmtime::Trap>>>
{
Box::new(async move { #closure_body })
fns.push(quote! {
#async_ fn #fn_ident(caller: &wasmtime::Caller<'_>, ctx: &mut #ctx_type #(, #arg_decls)*) -> Result<#ret_ty, wasmtime::Trap> {
unsafe {
let mem = match caller.get_export("memory") {
Some(wasmtime::Extern::Memory(m)) => m,
_ => {
return Err(wasmtime::Trap::new("missing required memory export"));
}
};
let mem = #runtime::WasmtimeGuestMemory::new(mem);
match #target_module::#name_ident(ctx, &mem #(, #arg_names)*) #await_ {
Ok(r) => Ok(r.into()),
Err(wasmtime_wiggle::Trap::String(err)) => Err(wasmtime::Trap::new(err)),
Err(wasmtime_wiggle::Trap::I32Exit(err)) => Err(wasmtime::Trap::i32_exit(err)),
}
}
);
}
} else {
quote! {
let my_cx = cx.clone();
let #name_ident = wasmtime::Func::wrap(
});
if is_async {
let wrapper = format_ident!("wrap{}_async", params.len());
ctors.push(quote! {
let #name_ident = wasmtime::Func::#wrapper(
store,
move |caller: wasmtime::Caller<'_> #(,#arg_decls)*| -> Result<#ret_ty, wasmtime::Trap> {
#closure_body
ctx.clone(),
move |caller: wasmtime::Caller<'_>, my_ctx: &Rc<RefCell<_>> #(,#arg_decls)*|
-> Box<dyn std::future::Future<Output = Result<#ret_ty, wasmtime::Trap>>> {
Box::new(async move { Self::#fn_ident(&caller, &mut my_ctx.borrow_mut() #(, #arg_names)*).await })
}
);
}
});
} else {
ctors.push(quote! {
let my_ctx = ctx.clone();
let #name_ident = wasmtime::Func::wrap(
store,
move |caller: wasmtime::Caller #(, #arg_decls)*| -> Result<#ret_ty, wasmtime::Trap> {
Self::#fn_ident(&caller, &mut my_ctx.borrow_mut() #(, #arg_names)*)
}
);
});
}
if is_async {
let wrapper = format_ident!("wrap{}_host_func_async", params.len());
host_funcs.push(quote! {
config.#wrapper(
stringify!(#module_ident),
stringify!(#name_ident),
move |caller #(,#arg_decls)*|
-> Box<dyn std::future::Future<Output = Result<#ret_ty, wasmtime::Trap>>> {
Box::new(async move {
let ctx = caller.store()
.get::<std::rc::Rc<std::cell::RefCell<#ctx_type>>>()
.ok_or_else(|| wasmtime::Trap::new("context is missing in the store"))?;
let result = Self::#fn_ident(&caller, &mut ctx.borrow_mut() #(, #arg_names)*).await;
result
})
}
);
});
} else {
host_funcs.push(quote! {
config.wrap_host_func(
stringify!(#module_ident),
stringify!(#name_ident),
move |caller: wasmtime::Caller #(, #arg_decls)*| -> Result<#ret_ty, wasmtime::Trap> {
let ctx = caller
.store()
.get::<std::rc::Rc<std::cell::RefCell<#ctx_type>>>()
.ok_or_else(|| wasmtime::Trap::new("context is missing in the store"))?;
let result = Self::#fn_ident(&caller, &mut ctx.borrow_mut() #(, #arg_names)*);
result
},
);
});
}
}

View File

@@ -42,16 +42,8 @@ impl atoms::Atoms for Ctx {
}
}
#[test]
fn test_sync_host_func() {
let store = async_store();
let ctx = Rc::new(RefCell::new(Ctx));
let atoms = Atoms::new(&store, ctx.clone());
let shim_mod = shim_module(&store);
let mut linker = wasmtime::Linker::new(&store);
atoms.add_to_linker(&mut linker).unwrap();
fn run_sync_func(linker: &wasmtime::Linker) {
let shim_mod = shim_module(linker.store());
let shim_inst = run(linker.instantiate_async(&shim_mod)).unwrap();
let results = run(shim_inst
@@ -68,16 +60,8 @@ fn test_sync_host_func() {
);
}
#[test]
fn test_async_host_func() {
let store = async_store();
let ctx = Rc::new(RefCell::new(Ctx));
let atoms = Atoms::new(&store, ctx.clone());
let shim_mod = shim_module(&store);
let mut linker = wasmtime::Linker::new(&store);
atoms.add_to_linker(&mut linker).unwrap();
fn run_async_func(linker: &wasmtime::Linker) {
let shim_mod = shim_module(linker.store());
let shim_inst = run(linker.instantiate_async(&shim_mod)).unwrap();
let input: i32 = 123;
@@ -105,6 +89,62 @@ fn test_async_host_func() {
assert_eq!((input * 2) as f32, result);
}
#[test]
fn test_sync_host_func() {
let store = async_store();
let ctx = Rc::new(RefCell::new(Ctx));
let atoms = Atoms::new(&store, ctx.clone());
let mut linker = wasmtime::Linker::new(&store);
atoms.add_to_linker(&mut linker).unwrap();
run_sync_func(&linker);
}
#[test]
fn test_async_host_func() {
let store = async_store();
let ctx = Rc::new(RefCell::new(Ctx));
let atoms = Atoms::new(&store, ctx.clone());
let mut linker = wasmtime::Linker::new(&store);
atoms.add_to_linker(&mut linker).unwrap();
run_async_func(&linker);
}
#[test]
fn test_sync_config_host_func() {
let mut config = wasmtime::Config::new();
config.async_support(true);
Atoms::add_to_config(&mut config);
let engine = wasmtime::Engine::new(&config).unwrap();
let store = wasmtime::Store::new(&engine);
assert!(Atoms::set_context(&store, Ctx).is_ok());
let linker = wasmtime::Linker::new(&store);
run_sync_func(&linker);
}
#[test]
fn test_async_config_host_func() {
let mut config = wasmtime::Config::new();
config.async_support(true);
Atoms::add_to_config(&mut config);
let engine = wasmtime::Engine::new(&config).unwrap();
let store = wasmtime::Store::new(&engine);
assert!(Atoms::set_context(&store, Ctx).is_ok());
let linker = wasmtime::Linker::new(&store);
run_async_func(&linker);
}
fn run<F: Future>(future: F) -> F::Output {
let mut f = Pin::from(Box::new(future));
let waker = dummy_waker();
@@ -138,9 +178,11 @@ fn dummy_waker() -> Waker {
assert_eq!(ptr as usize, 5);
}
}
fn async_store() -> wasmtime::Store {
let engine = wasmtime::Engine::default();
wasmtime::Store::new_async(&engine)
wasmtime::Store::new(
&wasmtime::Engine::new(wasmtime::Config::new().async_support(true)).unwrap(),
)
}
// Wiggle expects the caller to have an exported memory. Wasmtime can only

View File

@@ -9,7 +9,7 @@ fn main() -> Result<()> {
println!("Initializing...");
let mut config = Config::new();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
println!("Compiling module...");

View File

@@ -15,7 +15,7 @@ fn main() -> Result<()> {
// Load our previously compiled wasm file (built previously with Cargo) and
// also ensure that we generate debuginfo so this executable can be
// debugged in GDB.
let engine = Engine::new(Config::new().debug_info(true));
let engine = Engine::new(Config::new().debug_info(true))?;
let store = Store::new(&engine);
let module = Module::from_file(&engine, "target/wasm32-unknown-unknown/debug/fib.wasm")?;
let instance = Instance::new(&store, &module, &[])?;

View File

@@ -9,7 +9,7 @@ use wasmtime::*;
fn main() -> Result<()> {
// Enable interruptable code via `Config` and then create an interrupt
// handle which we'll use later to interrupt running code.
let engine = Engine::new(Config::new().interruptable(true));
let engine = Engine::new(Config::new().interruptable(true))?;
let store = Store::new(&engine);
let interrupt_handle = store.interrupt_handle()?;

View File

@@ -14,20 +14,25 @@ fn main() -> Result<()> {
.with_ansi(true)
.init();
let store = Store::default();
let mut linker = Linker::new(&store);
// Define the WASI functions globally on the `Config`.
let mut config = Config::default();
Wasi::add_to_config(&mut config);
// Create an instance of `Wasi` which contains a `WasiCtx`. Note that
// `WasiCtx` provides a number of ways to configure what the target program
let store = Store::new(&Engine::new(&config)?);
// Set the WASI context in the store; all instances in the store share this context.
// `WasiCtxBuilder` provides a number of ways to configure what the target program
// will have access to.
let wasi = Wasi::new(
assert!(Wasi::set_context(
&store,
WasiCtxBuilder::new()
.inherit_stdio()
.inherit_args()?
.build()?,
);
wasi.add_to_linker(&mut linker)?;
.build()?
)
.is_ok());
let mut linker = Linker::new(&store);
// Instantiate our module with the imports we've created, and run it.
let module = Module::from_file(store.engine(), "target/wasm32-wasi/debug/wasi.wasm")?;

View File

@@ -137,7 +137,7 @@ impl RunCommand {
if self.wasm_timeout.is_some() {
config.interruptable(true);
}
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
// Make wasi available by default.

View File

@@ -34,7 +34,7 @@ impl WastCommand {
}
let config = self.common.config()?;
let store = Store::new(&Engine::new(&config));
let store = Store::new(&Engine::new(&config)?);
let mut wast_context = WastContext::new(store);
wast_context

View File

@@ -6,8 +6,26 @@ use std::task::{Context, Poll, RawWaker, RawWakerVTable, Waker};
use wasmtime::*;
fn async_store() -> Store {
let engine = Engine::default();
Store::new_async(&engine)
Store::new(&Engine::new(Config::new().async_support(true)).unwrap())
}
fn run_smoke_test(func: &Func) {
run(func.call_async(&[])).unwrap();
run(func.call_async(&[])).unwrap();
let future1 = func.call_async(&[]);
let future2 = func.call_async(&[]);
run(future2).unwrap();
run(future1).unwrap();
}
fn run_smoke_get0_test(func: &Func) {
let func = func.get0_async::<()>().unwrap();
run(func()).unwrap();
run(func()).unwrap();
let future1 = func();
let future2 = func();
run(future2).unwrap();
run(future1).unwrap();
}
#[test]
@@ -19,12 +37,43 @@ fn smoke() {
(),
move |_caller, _state, _params, _results| Box::new(async { Ok(()) }),
);
run(func.call_async(&[])).unwrap();
run(func.call_async(&[])).unwrap();
let future1 = func.call_async(&[]);
let future2 = func.call_async(&[]);
run(future2).unwrap();
run(future1).unwrap();
run_smoke_test(&func);
run_smoke_get0_test(&func);
let func = Func::wrap0_async(&store, (), move |_caller: Caller<'_>, _state| {
Box::new(async { Ok(()) })
});
run_smoke_test(&func);
run_smoke_get0_test(&func);
}
#[test]
fn smoke_host_func() {
let mut config = Config::new();
config.async_support(true);
config.define_host_func_async(
"",
"first",
FuncType::new(None, None),
move |_caller, _params, _results| Box::new(async { Ok(()) }),
);
config.wrap0_host_func_async("", "second", move |_caller: Caller<'_>| {
Box::new(async { Ok(()) })
});
let store = Store::new(&Engine::new(&config).unwrap());
let func = store
.get_host_func("", "first")
.expect("expected host function");
run_smoke_test(&func);
run_smoke_get0_test(&func);
let func = store
.get_host_func("", "second")
.expect("expected host function");
run_smoke_test(&func);
run_smoke_get0_test(&func);
}
#[test]
@@ -41,35 +90,54 @@ fn smoke_with_suspension() {
})
},
);
run(func.call_async(&[])).unwrap();
run(func.call_async(&[])).unwrap();
let future1 = func.call_async(&[]);
let future2 = func.call_async(&[]);
run(future2).unwrap();
run(future1).unwrap();
run_smoke_test(&func);
run_smoke_get0_test(&func);
let func = Func::wrap0_async(&store, (), move |_caller: Caller<'_>, _state| {
Box::new(async {
PendingOnce::default().await;
Ok(())
})
});
run_smoke_test(&func);
run_smoke_get0_test(&func);
}
#[test]
fn smoke_get_with_suspension() {
let store = async_store();
let func = Func::new_async(
&store,
fn smoke_host_func_with_suspension() {
let mut config = Config::new();
config.async_support(true);
config.define_host_func_async(
"",
"first",
FuncType::new(None, None),
(),
move |_caller, _state, _params, _results| {
move |_caller, _params, _results| {
Box::new(async {
PendingOnce::default().await;
Ok(())
})
},
);
let func = func.get0_async::<()>().unwrap();
run(func()).unwrap();
run(func()).unwrap();
let future1 = func();
let future2 = func();
run(future2).unwrap();
run(future1).unwrap();
config.wrap0_host_func_async("", "second", move |_caller: Caller<'_>| {
Box::new(async {
PendingOnce::default().await;
Ok(())
})
});
let store = Store::new(&Engine::new(&config).unwrap());
let func = store
.get_host_func("", "first")
.expect("expected host function");
run_smoke_test(&func);
run_smoke_get0_test(&func);
let func = store
.get_host_func("", "second")
.expect("expected host function");
run_smoke_test(&func);
run_smoke_get0_test(&func);
}
#[test]
@@ -304,8 +372,8 @@ fn dummy_waker() -> Waker {
#[test]
fn iloop_with_fuel() {
let engine = Engine::new(Config::new().consume_fuel(true));
let store = Store::new_async(&engine);
let engine = Engine::new(Config::new().async_support(true).consume_fuel(true)).unwrap();
let store = Store::new(&engine);
store.out_of_fuel_async_yield(1_000, 10);
let module = Module::new(
&engine,
@@ -338,8 +406,8 @@ fn iloop_with_fuel() {
#[test]
fn fuel_eventually_finishes() {
let engine = Engine::new(Config::new().consume_fuel(true));
let store = Store::new_async(&engine);
let engine = Engine::new(Config::new().async_support(true).consume_fuel(true)).unwrap();
let store = Store::new(&engine);
store.out_of_fuel_async_yield(u32::max_value(), 10);
let module = Module::new(
&engine,
@@ -368,33 +436,60 @@ fn fuel_eventually_finishes() {
#[test]
fn async_with_pooling_stacks() {
let mut config = Config::new();
config
.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
table_elements: 0,
..Default::default()
},
instance_limits: InstanceLimits {
count: 1,
memory_reservation_size: 1,
},
})
.expect("pooling allocator created");
config.async_support(true);
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
table_elements: 0,
..Default::default()
},
instance_limits: InstanceLimits {
count: 1,
memory_reservation_size: 1,
},
});
let engine = Engine::new(&config);
let store = Store::new_async(&engine);
let engine = Engine::new(&config).unwrap();
let store = Store::new(&engine);
let func = Func::new_async(
&store,
FuncType::new(None, None),
(),
move |_caller, _state, _params, _results| Box::new(async { Ok(()) }),
);
run(func.call_async(&[])).unwrap();
run(func.call_async(&[])).unwrap();
let future1 = func.call_async(&[]);
let future2 = func.call_async(&[]);
run(future2).unwrap();
run(future1).unwrap();
run_smoke_test(&func);
run_smoke_get0_test(&func);
}
#[test]
fn async_host_func_with_pooling_stacks() {
let mut config = Config::new();
config.async_support(true);
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
table_elements: 0,
..Default::default()
},
instance_limits: InstanceLimits {
count: 1,
memory_reservation_size: 1,
},
});
config.define_host_func_async(
"",
"",
FuncType::new(None, None),
move |_caller, _params, _results| Box::new(async { Ok(()) }),
);
let store = Store::new(&Engine::new(&config).unwrap());
let func = store.get_host_func("", "").expect("expected host function");
run_smoke_test(&func);
run_smoke_get0_test(&func);
}

View File

@@ -112,7 +112,7 @@ mod tests {
// hostcall can be handled.
#[test]
fn test_custom_signal_handler_single_instance_hostcall() -> Result<()> {
let engine = Engine::new(&Config::default());
let engine = Engine::new(&Config::default())?;
let store = Store::new(&engine);
let module = Module::new(&engine, WAT1)?;
@@ -132,7 +132,7 @@ mod tests {
#[test]
fn test_custom_signal_handler_single_instance() -> Result<()> {
let engine = Engine::new(&Config::default());
let engine = Engine::new(&Config::default())?;
let store = Store::new(&engine);
let module = Module::new(&engine, WAT1)?;
@@ -193,7 +193,7 @@ mod tests {
#[test]
fn test_custom_signal_handler_multiple_instances() -> Result<()> {
let engine = Engine::new(&Config::default());
let engine = Engine::new(&Config::default())?;
let store = Store::new(&engine);
let module = Module::new(&engine, WAT1)?;
@@ -286,7 +286,7 @@ mod tests {
#[test]
fn test_custom_signal_handler_instance_calling_another_instance() -> Result<()> {
let engine = Engine::new(&Config::default());
let engine = Engine::new(&Config::default())?;
let store = Store::new(&engine);
// instance1 which defines 'read'

View File

@@ -57,7 +57,7 @@ fn bad_tables() {
fn cross_store() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store1 = Store::new(&engine);
let store2 = Store::new(&engine);
@@ -127,7 +127,7 @@ fn cross_store() -> anyhow::Result<()> {
fn get_set_externref_globals_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
// Initialize with a null externref.
@@ -162,7 +162,7 @@ fn get_set_externref_globals_via_api() -> anyhow::Result<()> {
fn get_set_funcref_globals_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let f = Func::wrap(&store, || {});
@@ -197,7 +197,7 @@ fn get_set_funcref_globals_via_api() -> anyhow::Result<()> {
fn create_get_set_funcref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::FuncRef, Limits::at_least(10));
@@ -218,7 +218,7 @@ fn create_get_set_funcref_tables_via_api() -> anyhow::Result<()> {
fn fill_funcref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::FuncRef, Limits::at_least(10));
@@ -244,7 +244,7 @@ fn fill_funcref_tables_via_api() -> anyhow::Result<()> {
fn grow_funcref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::FuncRef, Limits::at_least(10));
@@ -261,7 +261,7 @@ fn grow_funcref_tables_via_api() -> anyhow::Result<()> {
fn create_get_set_externref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::ExternRef, Limits::at_least(10));
@@ -292,7 +292,7 @@ fn create_get_set_externref_tables_via_api() -> anyhow::Result<()> {
fn fill_externref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::ExternRef, Limits::at_least(10));
@@ -328,7 +328,7 @@ fn fill_externref_tables_via_api() -> anyhow::Result<()> {
fn grow_externref_tables_via_api() -> anyhow::Result<()> {
let mut cfg = Config::new();
cfg.wasm_reference_types(true);
let engine = Engine::new(&cfg);
let engine = Engine::new(&cfg)?;
let store = Store::new(&engine);
let table_ty = TableType::new(ValType::ExternRef, Limits::at_least(10));
@@ -344,7 +344,7 @@ fn grow_externref_tables_via_api() -> anyhow::Result<()> {
#[test]
fn read_write_memory_via_api() {
let cfg = Config::new();
let store = Store::new(&Engine::new(&cfg));
let store = Store::new(&Engine::new(&cfg).unwrap());
let ty = MemoryType::new(Limits::new(1, None));
let mem = Memory::new(&store, ty);
mem.grow(1).unwrap();

View File

@@ -48,7 +48,7 @@ fn run() -> Result<()> {
fn fuel_consumed(wasm: &[u8]) -> u64 {
let mut config = Config::new();
config.consume_fuel(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config).unwrap();
let module = Module::new(&engine, wasm).unwrap();
let store = Store::new(&engine);
store.add_fuel(u64::max_value()).unwrap();
@@ -110,7 +110,7 @@ fn iloop() {
fn iloop_aborts(wat: &str) {
let mut config = Config::new();
config.consume_fuel(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config).unwrap();
let module = Module::new(&engine, wat).unwrap();
let store = Store::new(&engine);
store.add_fuel(10_000).unwrap();

View File

@@ -156,7 +156,7 @@ fn import_works() -> Result<()> {
)?;
let mut config = Config::new();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let module = Module::new(&engine, &wasm)?;
let instance = Instance::new(
@@ -459,7 +459,7 @@ fn return_cross_store_value() -> anyhow::Result<()> {
)?;
let mut config = Config::new();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, &wasm)?;
let store1 = Store::new(&engine);
@@ -485,7 +485,7 @@ fn return_cross_store_value() -> anyhow::Result<()> {
fn pass_cross_store_arg() -> anyhow::Result<()> {
let mut config = Config::new();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store1 = Store::new(&engine);
let store2 = Store::new(&engine);

638
tests/all/host_funcs.rs Normal file
View File

@@ -0,0 +1,638 @@
use anyhow::Result;
use std::sync::atomic::{AtomicUsize, Ordering::SeqCst};
use wasi_cap_std_sync::WasiCtxBuilder;
use wasmtime::*;
use wasmtime_wasi::Wasi;
#[test]
fn async_required() {
let mut config = Config::default();
config.define_host_func_async(
"",
"",
FuncType::new(None, None),
move |_caller, _params, _results| Box::new(async { Ok(()) }),
);
assert_eq!(
Engine::new(&config)
.map_err(|e| e.to_string())
.err()
.unwrap(),
"an async host function cannot be defined without async support enabled in the config"
);
}
#[test]
fn wrap_func() {
let mut config = Config::default();
config.wrap_host_func("", "", || {});
config.wrap_host_func("m", "f", |_: i32| {});
config.wrap_host_func("m", "f2", |_: i32, _: i64| {});
config.wrap_host_func("m2", "f", |_: f32, _: f64| {});
config.wrap_host_func("m2", "f2", || -> i32 { 0 });
config.wrap_host_func("", "", || -> i64 { 0 });
config.wrap_host_func("m", "f", || -> f32 { 0.0 });
config.wrap_host_func("m2", "f", || -> f64 { 0.0 });
config.wrap_host_func("m3", "", || -> Option<ExternRef> { None });
config.wrap_host_func("m3", "f", || -> Option<Func> { None });
config.wrap_host_func("", "f1", || -> Result<(), Trap> { loop {} });
config.wrap_host_func("", "f2", || -> Result<i32, Trap> { loop {} });
config.wrap_host_func("", "f3", || -> Result<i64, Trap> { loop {} });
config.wrap_host_func("", "f4", || -> Result<f32, Trap> { loop {} });
config.wrap_host_func("", "f5", || -> Result<f64, Trap> { loop {} });
config.wrap_host_func("", "f6", || -> Result<Option<ExternRef>, Trap> { loop {} });
config.wrap_host_func("", "f7", || -> Result<Option<Func>, Trap> { loop {} });
}
#[test]
fn drop_func() -> Result<()> {
static HITS: AtomicUsize = AtomicUsize::new(0);
struct A;
impl Drop for A {
fn drop(&mut self) {
HITS.fetch_add(1, SeqCst);
}
}
let mut config = Config::default();
let a = A;
config.wrap_host_func("", "", move || {
drop(&a);
});
assert_eq!(HITS.load(SeqCst), 0);
// Define the function again to ensure redefined functions are dropped
let a = A;
config.wrap_host_func("", "", move || {
drop(&a);
});
assert_eq!(HITS.load(SeqCst), 1);
drop(config);
assert_eq!(HITS.load(SeqCst), 2);
Ok(())
}
#[test]
fn drop_delayed() -> Result<()> {
static HITS: AtomicUsize = AtomicUsize::new(0);
struct A;
impl Drop for A {
fn drop(&mut self) {
HITS.fetch_add(1, SeqCst);
}
}
let mut config = Config::default();
let a = A;
config.wrap_host_func("", "", move || drop(&a));
assert_eq!(HITS.load(SeqCst), 0);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, &wat::parse_str(r#"(import "" "" (func))"#)?)?;
let store = Store::new(&engine);
let instance = Instance::new(
&store,
&module,
&[store
.get_host_func("", "")
.expect("function should be defined")
.into()],
)?;
drop((instance, store));
assert_eq!(HITS.load(SeqCst), 0);
let store = Store::new(&engine);
let instance = Instance::new(
&store,
&module,
&[store
.get_host_func("", "")
.expect("function should be defined")
.into()],
)?;
drop((instance, store, engine, module));
assert_eq!(HITS.load(SeqCst), 0);
drop(config);
assert_eq!(HITS.load(SeqCst), 1);
Ok(())
}
#[test]
fn signatures_match() -> Result<()> {
let mut config = Config::default();
config.wrap_host_func("", "f1", || {});
config.wrap_host_func("", "f2", || -> i32 { loop {} });
config.wrap_host_func("", "f3", || -> i64 { loop {} });
config.wrap_host_func("", "f4", || -> f32 { loop {} });
config.wrap_host_func("", "f5", || -> f64 { loop {} });
config.wrap_host_func(
"",
"f6",
|_: f32, _: f64, _: i32, _: i64, _: i32, _: Option<ExternRef>, _: Option<Func>| -> f64 {
loop {}
},
);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let f = store
.get_host_func("", "f1")
.expect("func should be defined");
assert_eq!(f.ty().params().collect::<Vec<_>>(), &[]);
assert_eq!(f.param_arity(), 0);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[]);
assert_eq!(f.result_arity(), 0);
let f = store
.get_host_func("", "f2")
.expect("func should be defined");
assert_eq!(f.ty().params().collect::<Vec<_>>(), &[]);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[ValType::I32]);
let f = store
.get_host_func("", "f3")
.expect("func should be defined");
assert_eq!(f.ty().params().collect::<Vec<_>>(), &[]);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[ValType::I64]);
let f = store
.get_host_func("", "f4")
.expect("func should be defined");
assert_eq!(f.ty().params().collect::<Vec<_>>(), &[]);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[ValType::F32]);
let f = store
.get_host_func("", "f5")
.expect("func should be defined");
assert_eq!(f.ty().params().collect::<Vec<_>>(), &[]);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[ValType::F64]);
let f = store
.get_host_func("", "f6")
.expect("func should be defined");
assert_eq!(
f.ty().params().collect::<Vec<_>>(),
&[
ValType::F32,
ValType::F64,
ValType::I32,
ValType::I64,
ValType::I32,
ValType::ExternRef,
ValType::FuncRef,
]
);
assert_eq!(f.ty().results().collect::<Vec<_>>(), &[ValType::F64]);
Ok(())
}
#[test]
// Note: Cranelift only supports refrerence types (used in the wasm in this
// test) on x64.
#[cfg(target_arch = "x86_64")]
fn import_works() -> Result<()> {
static HITS: AtomicUsize = AtomicUsize::new(0);
let wasm = wat::parse_str(
r#"
(import "" "f1" (func))
(import "" "f2" (func (param i32) (result i32)))
(import "" "f3" (func (param i32) (param i64)))
(import "" "f4" (func (param i32 i64 i32 f32 f64 externref funcref)))
(func (export "run") (param externref funcref)
call 0
i32.const 0
call 1
i32.const 1
i32.add
i64.const 3
call 2
i32.const 100
i64.const 200
i32.const 300
f32.const 400
f64.const 500
local.get 0
local.get 1
call 3
)
"#,
)?;
let mut config = Config::new();
config.wasm_reference_types(true);
config.wrap_host_func("", "f1", || {
assert_eq!(HITS.fetch_add(1, SeqCst), 0);
});
config.wrap_host_func("", "f2", |x: i32| -> i32 {
assert_eq!(x, 0);
assert_eq!(HITS.fetch_add(1, SeqCst), 1);
1
});
config.wrap_host_func("", "f3", |x: i32, y: i64| {
assert_eq!(x, 2);
assert_eq!(y, 3);
assert_eq!(HITS.fetch_add(1, SeqCst), 2);
});
config.wrap_host_func(
"",
"f4",
|a: i32, b: i64, c: i32, d: f32, e: f64, f: Option<ExternRef>, g: Option<Func>| {
assert_eq!(a, 100);
assert_eq!(b, 200);
assert_eq!(c, 300);
assert_eq!(d, 400.0);
assert_eq!(e, 500.0);
assert_eq!(
f.as_ref().unwrap().data().downcast_ref::<String>().unwrap(),
"hello"
);
assert_eq!(g.as_ref().unwrap().call(&[]).unwrap()[0].unwrap_i32(), 42);
assert_eq!(HITS.fetch_add(1, SeqCst), 3);
},
);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, &wasm)?;
let store = Store::new(&engine);
let instance = Instance::new(
&store,
&module,
&[
store
.get_host_func("", "f1")
.expect("should be defined")
.into(),
store
.get_host_func("", "f2")
.expect("should be defined")
.into(),
store
.get_host_func("", "f3")
.expect("should be defined")
.into(),
store
.get_host_func("", "f4")
.expect("should be defined")
.into(),
],
)?;
let run = instance.get_func("run").unwrap();
run.call(&[
Val::ExternRef(Some(ExternRef::new("hello".to_string()))),
Val::FuncRef(Some(Func::wrap(&store, || -> i32 { 42 }))),
])?;
assert_eq!(HITS.load(SeqCst), 4);
Ok(())
}
#[test]
fn trap_smoke() -> Result<()> {
let mut config = Config::default();
config.wrap_host_func("", "", || -> Result<(), Trap> { Err(Trap::new("test")) });
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let f = store.get_host_func("", "").expect("should be defined");
let err = f.call(&[]).unwrap_err().downcast::<Trap>()?;
assert!(err.to_string().contains("test"));
assert!(err.i32_exit_status().is_none());
Ok(())
}
#[test]
fn trap_import() -> Result<()> {
let wasm = wat::parse_str(
r#"
(import "" "" (func))
(start 0)
"#,
)?;
let mut config = Config::default();
config.wrap_host_func("", "", || -> Result<(), Trap> { Err(Trap::new("foo")) });
let engine = Engine::new(&config)?;
let module = Module::new(&engine, &wasm)?;
let store = Store::new(&engine);
let trap = Instance::new(
&store,
&module,
&[store.get_host_func("", "").expect("defined").into()],
)
.err()
.unwrap()
.downcast::<Trap>()?;
assert!(trap.to_string().contains("foo"));
Ok(())
}
#[test]
fn new_from_signature() -> Result<()> {
let mut config = Config::default();
let ty = FuncType::new(None, None);
config.define_host_func("", "f1", ty, |_, _, _| panic!());
let ty = FuncType::new(Some(ValType::I32), Some(ValType::F64));
config.define_host_func("", "f2", ty, |_, _, _| panic!());
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let f = store.get_host_func("", "f1").expect("func defined");
assert!(f.get0::<()>().is_ok());
assert!(f.get0::<i32>().is_err());
assert!(f.get1::<i32, ()>().is_err());
let f = store.get_host_func("", "f2").expect("func defined");
assert!(f.get0::<()>().is_err());
assert!(f.get0::<i32>().is_err());
assert!(f.get1::<i32, ()>().is_err());
assert!(f.get1::<i32, f64>().is_ok());
Ok(())
}
#[test]
fn call_wrapped_func() -> Result<()> {
let mut config = Config::default();
config.wrap_host_func("", "f1", |a: i32, b: i64, c: f32, d: f64| {
assert_eq!(a, 1);
assert_eq!(b, 2);
assert_eq!(c, 3.0);
assert_eq!(d, 4.0);
});
config.wrap_host_func("", "f2", || 1i32);
config.wrap_host_func("", "f3", || 2i64);
config.wrap_host_func("", "f4", || 3.0f32);
config.wrap_host_func("", "f5", || 4.0f64);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let f = store.get_host_func("", "f1").expect("func defined");
f.call(&[Val::I32(1), Val::I64(2), 3.0f32.into(), 4.0f64.into()])?;
f.get4::<i32, i64, f32, f64, ()>()?(1, 2, 3.0, 4.0)?;
let f = store.get_host_func("", "f2").expect("func defined");
let results = f.call(&[])?;
assert_eq!(results.len(), 1);
assert_eq!(results[0].unwrap_i32(), 1);
assert_eq!(f.get0::<i32>()?()?, 1);
let f = store.get_host_func("", "f3").expect("func defined");
let results = f.call(&[])?;
assert_eq!(results.len(), 1);
assert_eq!(results[0].unwrap_i64(), 2);
assert_eq!(f.get0::<i64>()?()?, 2);
let f = store.get_host_func("", "f4").expect("func defined");
let results = f.call(&[])?;
assert_eq!(results.len(), 1);
assert_eq!(results[0].unwrap_f32(), 3.0);
assert_eq!(f.get0::<f32>()?()?, 3.0);
let f = store.get_host_func("", "f5").expect("func defined");
let results = f.call(&[])?;
assert_eq!(results.len(), 1);
assert_eq!(results[0].unwrap_f64(), 4.0);
assert_eq!(f.get0::<f64>()?()?, 4.0);
Ok(())
}
#[test]
fn func_return_nothing() -> Result<()> {
let mut config = Config::default();
let ty = FuncType::new(None, Some(ValType::I32));
config.define_host_func("", "", ty, |_, _, _| Ok(()));
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let f = store.get_host_func("", "").expect("func defined");
let err = f.call(&[]).unwrap_err().downcast::<Trap>()?;
assert!(err
.to_string()
.contains("function attempted to return an incompatible value"));
Ok(())
}
#[test]
fn call_via_funcref() -> Result<()> {
static HITS: AtomicUsize = AtomicUsize::new(0);
struct A;
impl Drop for A {
fn drop(&mut self) {
HITS.fetch_add(1, SeqCst);
}
}
let wasm = wat::parse_str(
r#"
(table $t 1 funcref)
(type $add (func (param i32 i32) (result i32)))
(func (export "call") (param funcref) (result i32 funcref)
(table.set $t (i32.const 0) (local.get 0))
(call_indirect (type $add) (i32.const 3) (i32.const 4) (i32.const 0))
(local.get 0)
)
"#,
)?;
let mut config = Config::default();
let a = A;
config.wrap_host_func("", "", move |x: i32, y: i32| {
drop(&a);
x + y
});
let engine = Engine::new(&config)?;
let module = Module::new(&engine, &wasm)?;
let store = Store::new(&engine);
let instance = Instance::new(&store, &module, &[])?;
let results = instance
.get_func("call")
.unwrap()
.call(&[store.get_host_func("", "").expect("func defined").into()])?;
assert_eq!(results.len(), 2);
assert_eq!(results[0].unwrap_i32(), 7);
{
let f = results[1].unwrap_funcref().unwrap();
let results = f.call(&[1.into(), 2.into()])?;
assert_eq!(results.len(), 1);
assert_eq!(results[0].unwrap_i32(), 3);
}
assert_eq!(HITS.load(SeqCst), 0);
drop((results, instance, store, module, engine));
assert_eq!(HITS.load(SeqCst), 0);
drop(config);
assert_eq!(HITS.load(SeqCst), 1);
Ok(())
}
#[test]
fn store_with_context() -> Result<()> {
struct Ctx {
called: std::cell::Cell<bool>,
}
let mut config = Config::default();
config.wrap_host_func("", "", |caller: Caller| {
let ctx = caller
.store()
.get::<Ctx>()
.expect("store should have context");
ctx.called.set(true);
});
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
assert!(store.get::<Ctx>().is_none());
assert!(store
.set(Ctx {
called: std::cell::Cell::new(false)
})
.is_ok());
assert!(store
.set(Ctx {
called: std::cell::Cell::new(false)
})
.is_err());
assert!(!store.get::<Ctx>().unwrap().called.get());
let f = store.get_host_func("", "").expect("func defined");
f.call(&[])?;
assert!(store.get::<Ctx>().unwrap().called.get());
Ok(())
}
#[test]
fn wasi_imports_missing_context() -> Result<()> {
let mut config = Config::default();
Wasi::add_to_config(&mut config);
let wasm = wat::parse_str(
r#"
(import "wasi_snapshot_preview1" "proc_exit" (func $__wasi_proc_exit (param i32)))
(memory (export "memory") 0)
(func (export "_start")
(call $__wasi_proc_exit (i32.const 123))
)
"#,
)?;
let engine = Engine::new(&config)?;
let module = Module::new(&engine, wasm)?;
let store = Store::new(&engine);
let linker = Linker::new(&store);
let instance = linker.instantiate(&module)?;
let start = instance.get_func("_start").unwrap().get0::<()>()?;
let trap = start().unwrap_err();
assert!(trap.to_string().contains("context is missing in the store"));
assert!(trap.i32_exit_status().is_none());
Ok(())
}
#[test]
fn wasi_imports() -> Result<()> {
let mut config = Config::default();
Wasi::add_to_config(&mut config);
let wasm = wat::parse_str(
r#"
(import "wasi_snapshot_preview1" "proc_exit" (func $__wasi_proc_exit (param i32)))
(memory (export "memory") 0)
(func (export "_start")
(call $__wasi_proc_exit (i32.const 123))
)
"#,
)?;
let engine = Engine::new(&config)?;
let module = Module::new(&engine, wasm)?;
let store = Store::new(&engine);
assert!(Wasi::set_context(&store, WasiCtxBuilder::new().build()?).is_ok());
let linker = Linker::new(&store);
let instance = linker.instantiate(&module)?;
let start = instance.get_func("_start").unwrap().get0::<()>()?;
let trap = start().unwrap_err();
assert_eq!(trap.i32_exit_status(), Some(123));
Ok(())
}

View File

@@ -2,7 +2,7 @@ use std::sync::atomic::{AtomicUsize, Ordering::SeqCst};
use wasmtime::*;
fn interruptable_store() -> Store {
let engine = Engine::new(Config::new().interruptable(true));
let engine = Engine::new(Config::new().interruptable(true)).unwrap();
Store::new(&engine)
}

View File

@@ -223,3 +223,40 @@ fn no_leak_with_imports() -> Result<()> {
assert!(flag.get(), "store was leaked");
Ok(())
}
#[test]
fn get_host_function() -> Result<()> {
let mut config = Config::default();
config.wrap_host_func("mod", "f1", || {});
let engine = Engine::new(&config)?;
let module = Module::new(&engine, r#"(module (import "mod" "f1" (func)))"#)?;
let store = Store::new(&engine);
let linker = Linker::new(&store);
assert!(linker.get(&module.imports().nth(0).unwrap()).is_some());
Ok(())
}
#[test]
fn shadowing_host_function() -> Result<()> {
let mut config = Config::default();
config.wrap_host_func("mod", "f1", || {});
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let mut linker = Linker::new(&store);
assert!(linker
.define("mod", "f1", Func::wrap(&store, || {}))
.is_err());
linker.define("mod", "f2", Func::wrap(&store, || {}))?;
let mut linker = Linker::new(&store);
linker.allow_shadowing(true);
linker.define("mod", "f1", Func::wrap(&store, || {}))?;
linker.define("mod", "f2", Func::wrap(&store, || {}))?;
Ok(())
}

View File

@@ -7,6 +7,7 @@ mod fuel;
mod func;
mod fuzzing;
mod globals;
mod host_funcs;
mod iloop;
mod import_calling_export;
mod import_indexes;
@@ -43,7 +44,7 @@ pub(crate) fn ref_types_module(
let mut config = Config::new();
config.wasm_reference_types(true);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let module = Module::new(&engine, source)?;

View File

@@ -136,7 +136,7 @@ mod not_for_windows {
.with_host_memory(mem_creator.clone())
.static_memory_maximum_size(0)
.dynamic_memory_guard_size(0);
(Store::new(&Engine::new(&config)), mem_creator)
(Store::new(&Engine::new(&config).unwrap()), mem_creator)
}
#[test]

View File

@@ -5,13 +5,13 @@ fn caches_across_engines() {
let mut c = Config::new();
c.cranelift_clear_cpu_flags();
let bytes = Module::new(&Engine::new(&c), "(module)")
let bytes = Module::new(&Engine::new(&c).unwrap(), "(module)")
.unwrap()
.serialize()
.unwrap();
let res = Module::deserialize(
&Engine::new(&Config::new().cranelift_clear_cpu_flags()),
&Engine::new(&Config::new().cranelift_clear_cpu_flags()).unwrap(),
&bytes,
);
assert!(res.is_ok());
@@ -22,7 +22,8 @@ fn caches_across_engines() {
&Config::new()
.cranelift_clear_cpu_flags()
.cranelift_nan_canonicalization(true),
),
)
.unwrap(),
&bytes,
);
assert!(res.is_err());
@@ -33,7 +34,8 @@ fn caches_across_engines() {
&Config::new()
.cranelift_clear_cpu_flags()
.cranelift_opt_level(OptLevel::None),
),
)
.unwrap(),
&bytes,
);
assert!(res.is_err());
@@ -46,7 +48,8 @@ fn caches_across_engines() {
.cranelift_clear_cpu_flags()
.cranelift_other_flag("has_sse3", "true")
.unwrap()
}),
})
.unwrap(),
&bytes,
);
assert!(res.is_err());

View File

@@ -4,7 +4,7 @@ use wasmtime::*;
fn engine() -> Engine {
let mut config = Config::new();
config.wasm_module_linking(true);
Engine::new(&config)
Engine::new(&config).unwrap()
}
#[test]
@@ -191,7 +191,7 @@ fn limit_instances() -> Result<()> {
let mut config = Config::new();
config.wasm_module_linking(true);
config.max_instances(10);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
r#"
@@ -232,7 +232,7 @@ fn limit_memories() -> Result<()> {
config.wasm_module_linking(true);
config.wasm_multi_memory(true);
config.max_memories(10);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
r#"
@@ -267,7 +267,7 @@ fn limit_tables() -> Result<()> {
let mut config = Config::new();
config.wasm_module_linking(true);
config.max_tables(10);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
r#"

View File

@@ -39,7 +39,7 @@ fn test_module_serialize_fail() -> Result<()> {
let mut config = Config::new();
config.cranelift_opt_level(OptLevel::None);
let store = Store::new(&Engine::new(&config));
let store = Store::new(&Engine::new(&config)?);
match deserialize_and_instantiate(&store, &buffer) {
Ok(_) => bail!("expected failure at deserialization"),
Err(_) => (),

View File

@@ -4,7 +4,7 @@ use wasmtime::*;
#[test]
fn successful_instantiation() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
@@ -15,9 +15,9 @@ fn successful_instantiation() -> Result<()> {
count: 1,
memory_reservation_size: 1,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, r#"(module (memory 1) (table 10 funcref))"#)?;
// Module should instantiate
@@ -30,7 +30,7 @@ fn successful_instantiation() -> Result<()> {
#[test]
fn memory_limit() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 3,
@@ -41,9 +41,9 @@ fn memory_limit() -> Result<()> {
count: 1,
memory_reservation_size: 196608,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
// Module should fail to validate because the minimum is greater than the configured limit
match Module::new(&engine, r#"(module (memory 4))"#) {
@@ -92,7 +92,7 @@ fn memory_limit() -> Result<()> {
#[test]
fn memory_init() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 2,
@@ -103,9 +103,9 @@ fn memory_init() -> Result<()> {
count: 1,
..Default::default()
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
@@ -130,7 +130,7 @@ fn memory_init() -> Result<()> {
#[test]
fn memory_guard_page_trap() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 2,
@@ -141,9 +141,9 @@ fn memory_guard_page_trap() -> Result<()> {
count: 1,
..Default::default()
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
@@ -185,7 +185,7 @@ fn memory_guard_page_trap() -> Result<()> {
#[cfg_attr(target_arch = "aarch64", ignore)] // https://github.com/bytecodealliance/wasmtime/pull/2518#issuecomment-747280133
fn memory_zeroed() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
@@ -196,9 +196,9 @@ fn memory_zeroed() -> Result<()> {
count: 1,
memory_reservation_size: 1,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, r#"(module (memory (export "m") 1))"#)?;
@@ -228,7 +228,7 @@ fn memory_zeroed() -> Result<()> {
fn table_limit() -> Result<()> {
const TABLE_ELEMENTS: u32 = 10;
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
@@ -239,9 +239,9 @@ fn table_limit() -> Result<()> {
count: 1,
memory_reservation_size: 1,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
// Module should fail to validate because the minimum is greater than the configured limit
match Module::new(&engine, r#"(module (table 31 funcref))"#) {
@@ -296,7 +296,7 @@ fn table_limit() -> Result<()> {
#[test]
fn table_init() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 0,
@@ -307,9 +307,9 @@ fn table_init() -> Result<()> {
count: 1,
..Default::default()
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(
&engine,
@@ -346,7 +346,7 @@ fn table_init() -> Result<()> {
#[cfg_attr(target_arch = "aarch64", ignore)] // https://github.com/bytecodealliance/wasmtime/pull/2518#issuecomment-747280133
fn table_zeroed() -> Result<()> {
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
@@ -357,9 +357,9 @@ fn table_zeroed() -> Result<()> {
count: 1,
memory_reservation_size: 1,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, r#"(module (table (export "t") 10 funcref))"#)?;
@@ -388,7 +388,7 @@ fn table_zeroed() -> Result<()> {
fn instantiation_limit() -> Result<()> {
const INSTANCE_LIMIT: u32 = 10;
let mut config = Config::new();
config.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
config.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
memory_pages: 1,
@@ -399,9 +399,9 @@ fn instantiation_limit() -> Result<()> {
count: INSTANCE_LIMIT,
memory_reservation_size: 1,
},
})?;
});
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let module = Module::new(&engine, r#"(module)"#)?;
// Instantiate to the limit

View File

@@ -526,7 +526,7 @@ fn parse_dwarf_info() -> Result<()> {
);
let mut config = Config::new();
config.wasm_backtrace_details(WasmBacktraceDetails::Enable);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let module = Module::new(&engine, &wasm)?;
let mut linker = Linker::new(&store);
@@ -561,7 +561,7 @@ fn parse_dwarf_info() -> Result<()> {
fn no_hint_even_with_dwarf_info() -> Result<()> {
let mut config = Config::new();
config.wasm_backtrace_details(WasmBacktraceDetails::Disable);
let engine = Engine::new(&config);
let engine = Engine::new(&config)?;
let store = Store::new(&engine);
let module = Module::new(
&engine,

View File

@@ -52,7 +52,7 @@ fn run_wast(wast: &str, strategy: Strategy, pooling: bool) -> anyhow::Result<()>
// However, these limits may become insufficient in the future as the wast tests change.
// If a wast test fails because of a limit being "exceeded" or if memory/table
// fails to grow, the values here will need to be adjusted.
cfg.with_allocation_strategy(InstanceAllocationStrategy::Pooling {
cfg.allocation_strategy(InstanceAllocationStrategy::Pooling {
strategy: PoolingAllocationStrategy::NextAvailable,
module_limits: ModuleLimits {
imported_memories: 2,
@@ -68,10 +68,10 @@ fn run_wast(wast: &str, strategy: Strategy, pooling: bool) -> anyhow::Result<()>
count: 450,
..Default::default()
},
})?;
});
}
let store = Store::new(&Engine::new(&cfg));
let store = Store::new(&Engine::new(&cfg)?);
let mut wast_context = WastContext::new(store);
wast_context.register_spectest()?;
wast_context.run_file(wast)?;