Write ELF image and instantiate code_memory from it (#1931)

- Create the ELF image from Compilation
- Create CodeMemory from the ELF image
- Link using ELF image
- Remove creation of GDB JIT images from crates/debug
- Move make_trampoline from compiler.rs
This commit is contained in:
Yury Delendik
2020-07-07 12:51:24 -05:00
committed by GitHub
parent 79f054f77f
commit bef1b87be0
13 changed files with 1093 additions and 580 deletions

View File

@@ -30,6 +30,7 @@ anyhow = "1.0"
cfg-if = "0.1.9"
log = "0.4"
gimli = { version = "0.21.0", default-features = false, features = ["write"] }
object = { version = "0.20", default-features = false, features = ["write"] }
[target.'cfg(target_os = "windows")'.dependencies]
winapi = { version = "0.3.8", features = ["winnt", "impl-default"] }

View File

@@ -1,21 +1,26 @@
//! Memory management for executable code.
use crate::object::{
utils::{try_parse_func_name, try_parse_trampoline_name},
ObjectUnwindInfo,
};
use crate::unwind::UnwindRegistry;
use object::read::{File as ObjectFile, Object, ObjectSection};
use region;
use std::collections::BTreeMap;
use std::mem::ManuallyDrop;
use std::{cmp, mem};
use wasmtime_environ::{
isa::{unwind::UnwindInfo, TargetIsa},
Compilation, CompiledFunction, Relocation, Relocations,
wasm::{FuncIndex, SignatureIndex},
Compilation, CompiledFunction,
};
use wasmtime_runtime::{Mmap, VMFunctionBody};
type CodeMemoryRelocations = Vec<(u32, Vec<Relocation>)>;
struct CodeMemoryEntry {
mmap: ManuallyDrop<Mmap>,
registry: ManuallyDrop<UnwindRegistry>,
relocs: CodeMemoryRelocations,
len: usize,
}
impl CodeMemoryEntry {
@@ -25,13 +30,13 @@ impl CodeMemoryEntry {
Ok(Self {
mmap,
registry,
relocs: vec![],
len: 0,
})
}
fn range(&self) -> (usize, usize) {
let start = self.mmap.as_ptr() as usize;
let end = start + self.mmap.len();
let end = start + self.len;
(start, end)
}
}
@@ -46,11 +51,40 @@ impl Drop for CodeMemoryEntry {
}
}
pub(crate) struct CodeMemoryObjectAllocation<'a> {
buf: &'a mut [u8],
funcs: BTreeMap<FuncIndex, (usize, usize)>,
trampolines: BTreeMap<SignatureIndex, (usize, usize)>,
}
impl<'a> CodeMemoryObjectAllocation<'a> {
pub fn code_range(self) -> &'a mut [u8] {
self.buf
}
pub fn funcs(&'a self) -> impl Iterator<Item = (FuncIndex, &'a mut [VMFunctionBody])> + 'a {
let buf = self.buf as *const _ as *mut [u8];
self.funcs.iter().map(move |(i, (start, len))| {
(*i, unsafe {
CodeMemory::view_as_mut_vmfunc_slice(&mut (*buf)[*start..*start + *len])
})
})
}
pub fn trampolines(
&'a self,
) -> impl Iterator<Item = (SignatureIndex, &'a mut [VMFunctionBody])> + 'a {
let buf = self.buf as *const _ as *mut [u8];
self.trampolines.iter().map(move |(i, (start, len))| {
(*i, unsafe {
CodeMemory::view_as_mut_vmfunc_slice(&mut (*buf)[*start..*start + *len])
})
})
}
}
/// Memory manager for executable code.
pub struct CodeMemory {
current: Option<CodeMemoryEntry>,
entries: Vec<CodeMemoryEntry>,
position: usize,
published: usize,
}
@@ -65,7 +99,6 @@ impl CodeMemory {
Self {
current: None,
entries: Vec::new(),
position: 0,
published: 0,
}
}
@@ -76,16 +109,13 @@ impl CodeMemory {
pub fn allocate_for_function<'a>(
&mut self,
func: &'a CompiledFunction,
relocs: impl Iterator<Item = &'a Relocation>,
) -> Result<&mut [VMFunctionBody], String> {
let size = Self::function_allocation_size(func);
let (buf, registry, start, m_relocs) = self.allocate(size)?;
let (buf, registry, start) = self.allocate(size)?;
let (_, _, vmfunc) = Self::copy_function(func, start as u32, buf, registry);
Self::copy_relocs(m_relocs, start as u32, relocs);
Ok(vmfunc)
}
@@ -93,23 +123,20 @@ impl CodeMemory {
pub fn allocate_for_compilation(
&mut self,
compilation: &Compilation,
relocations: &Relocations,
) -> Result<Box<[&mut [VMFunctionBody]]>, String> {
let total_len = compilation
.into_iter()
.fold(0, |acc, func| acc + Self::function_allocation_size(func));
let (mut buf, registry, start, m_relocs) = self.allocate(total_len)?;
let (mut buf, registry, start) = self.allocate(total_len)?;
let mut result = Vec::with_capacity(compilation.len());
let mut start = start as u32;
for (func, relocs) in compilation.into_iter().zip(relocations.values()) {
for func in compilation.into_iter() {
let (next_start, next_buf, vmfunc) = Self::copy_function(func, start, buf, registry);
result.push(vmfunc);
Self::copy_relocs(m_relocs, start, relocs.iter());
start = next_start;
buf = next_buf;
}
@@ -125,7 +152,7 @@ impl CodeMemory {
for CodeMemoryEntry {
mmap: m,
registry: r,
relocs,
..
} in &mut self.entries[self.published..]
{
// Remove write access to the pages due to the relocation fixups.
@@ -138,10 +165,6 @@ impl CodeMemory {
}
.expect("unable to make memory readonly and executable");
}
// Relocs data in not needed anymore -- clearing.
// TODO use relocs to serialize the published code.
relocs.clear();
}
self.published = self.entries.len();
@@ -159,37 +182,24 @@ impl CodeMemory {
/// * The offset within the current mmap that the slice starts at
///
/// TODO: Add an alignment flag.
fn allocate(
&mut self,
size: usize,
) -> Result<
(
&mut [u8],
&mut UnwindRegistry,
usize,
&mut CodeMemoryRelocations,
),
String,
> {
fn allocate(&mut self, size: usize) -> Result<(&mut [u8], &mut UnwindRegistry, usize), String> {
assert!(size > 0);
if match &self.current {
Some(e) => e.mmap.len() - self.position < size,
Some(e) => e.mmap.len() - e.len < size,
None => true,
} {
self.push_current(cmp::max(0x10000, size))?;
}
let old_position = self.position;
self.position += size;
let e = self.current.as_mut().unwrap();
let old_position = e.len;
e.len += size;
Ok((
&mut e.mmap.as_mut_slice()[old_position..self.position],
&mut e.mmap.as_mut_slice()[old_position..e.len],
&mut e.registry,
old_position,
&mut e.relocs,
))
}
@@ -206,14 +216,6 @@ impl CodeMemory {
}
}
fn copy_relocs<'a>(
entry_relocs: &'_ mut CodeMemoryRelocations,
start: u32,
relocs: impl Iterator<Item = &'a Relocation>,
) {
entry_relocs.push((start, relocs.cloned().collect()));
}
/// Copies the data of the compiled function to the given buffer.
///
/// This will also add the function to the current unwind registry.
@@ -276,8 +278,6 @@ impl CodeMemory {
self.entries.push(e);
}
self.position = 0;
Ok(())
}
@@ -288,18 +288,78 @@ impl CodeMemory {
.map(|entry| entry.range())
}
/// Returns all relocations for the unpublished memory.
pub fn unpublished_relocations<'a>(
&'a self,
) -> impl Iterator<Item = (*const u8, &'a Relocation)> + 'a {
self.entries[self.published..]
.iter()
.chain(self.current.iter())
.flat_map(|entry| {
entry.relocs.iter().flat_map(move |(start, relocs)| {
let base_ptr = unsafe { entry.mmap.as_ptr().add(*start as usize) };
relocs.iter().map(move |r| (base_ptr, r))
})
})
/// Allocates and copies the ELF image code section into CodeMemory.
/// Returns references to functions and trampolines defined there.
pub(crate) fn allocate_for_object<'a>(
&'a mut self,
obj: &ObjectFile,
unwind_info: &[ObjectUnwindInfo],
) -> Result<CodeMemoryObjectAllocation<'a>, String> {
let text_section = obj.section_by_name(".text").unwrap();
if text_section.size() == 0 {
// No code in the image.
return Ok(CodeMemoryObjectAllocation {
buf: &mut [],
funcs: BTreeMap::new(),
trampolines: BTreeMap::new(),
});
}
// Allocate chunk memory that spans entire code section.
let (buf, registry, start) = self.allocate(text_section.size() as usize)?;
buf.copy_from_slice(
text_section
.data()
.map_err(|_| "cannot read section data".to_string())?,
);
// Track locations of all defined functions and trampolines.
let mut funcs = BTreeMap::new();
let mut trampolines = BTreeMap::new();
for (_id, sym) in obj.symbols() {
match sym.name() {
Some(name) => {
if let Some(index) = try_parse_func_name(name) {
let is_import = sym.section_index().is_none();
if !is_import {
funcs.insert(
index,
(start + sym.address() as usize, sym.size() as usize),
);
}
} else if let Some(index) = try_parse_trampoline_name(name) {
trampolines
.insert(index, (start + sym.address() as usize, sym.size() as usize));
}
}
None => (),
}
}
// Register all unwind entiries for functions and trampolines.
// TODO will `u32` type for start/len be enough for large code base.
for i in unwind_info {
match i {
ObjectUnwindInfo::Func(func_index, info) => {
let (start, len) = funcs.get(&func_index).unwrap();
registry
.register(*start as u32, *len as u32, &info)
.expect("failed to register unwind information");
}
ObjectUnwindInfo::Trampoline(trampoline_index, info) => {
let (start, len) = trampolines.get(&trampoline_index).unwrap();
registry
.register(*start as u32, *len as u32, &info)
.expect("failed to register unwind information");
}
}
}
Ok(CodeMemoryObjectAllocation {
buf: &mut buf[..text_section.size() as usize],
funcs,
trampolines,
})
}
}

View File

@@ -1,23 +1,17 @@
//! JIT compilation.
use crate::code_memory::CodeMemory;
use crate::instantiate::SetupError;
use cranelift_codegen::ir::ExternalName;
use cranelift_codegen::ir::InstBuilder;
use cranelift_codegen::print_errors::pretty_error;
use cranelift_codegen::Context;
use cranelift_codegen::{binemit, ir};
use cranelift_frontend::{FunctionBuilder, FunctionBuilderContext};
use crate::object::{build_object, ObjectUnwindInfo};
use cranelift_codegen::ir;
use wasmtime_debug::{emit_dwarf, DebugInfoData, DwarfSection};
use wasmtime_environ::entity::{EntityRef, PrimaryMap};
use wasmtime_environ::isa::{TargetFrontendConfig, TargetIsa};
use wasmtime_environ::wasm::{DefinedFuncIndex, DefinedMemoryIndex, MemoryIndex, SignatureIndex};
use wasmtime_environ::wasm::{DefinedFuncIndex, DefinedMemoryIndex, MemoryIndex};
use wasmtime_environ::{
CacheConfig, CompileError, CompiledFunction, Compiler as _C, Module, ModuleAddressMap,
ModuleMemoryOffset, ModuleTranslation, ModuleVmctxInfo, Relocation, RelocationTarget,
Relocations, StackMaps, Traps, Tunables, VMOffsets, ValueLabelsRanges,
CacheConfig, Compiler as _C, Module, ModuleAddressMap, ModuleMemoryOffset, ModuleTranslation,
ModuleVmctxInfo, StackMaps, Traps, Tunables, VMOffsets, ValueLabelsRanges,
};
use wasmtime_runtime::{InstantiationError, VMFunctionBody, VMTrampoline};
use wasmtime_runtime::InstantiationError;
/// Select which kind of compilation to use.
#[derive(Copy, Clone, Debug)]
@@ -107,36 +101,10 @@ fn transform_dwarf_data(
.map_err(SetupError::DebugInfo)
}
fn get_code_range(
compilation: &wasmtime_environ::Compilation,
finished_functions: &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
) -> (*const u8, usize) {
if finished_functions.is_empty() {
return (::std::ptr::null(), 0);
}
// Assuming all functions in the same code block, looking min/max of its range.
let (start, end) = finished_functions.iter().fold::<(usize, usize), _>(
(!0, 0),
|(start, end), (i, body_ptr)| {
let body_ptr = (*body_ptr) as *const u8 as usize;
let body_len = compilation.get(i).body.len();
(
::std::cmp::min(start, body_ptr),
::std::cmp::max(end, body_ptr + body_len),
)
},
);
(start as *const u8, end - start)
}
#[allow(missing_docs)]
pub struct Compilation {
pub code_memory: CodeMemory,
pub finished_functions: PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
pub code_range: (*const u8, usize),
pub trampolines: PrimaryMap<SignatureIndex, VMTrampoline>,
pub jt_offsets: PrimaryMap<DefinedFuncIndex, ir::JumpTableOffsets>,
pub dwarf_sections: Vec<DwarfSection>,
pub obj: Vec<u8>,
pub unwind_info: Vec<ObjectUnwindInfo>,
pub traps: Traps,
pub stack_maps: StackMaps,
pub address_transform: ModuleAddressMap,
@@ -164,8 +132,6 @@ impl Compiler {
translation: &ModuleTranslation,
debug_data: Option<DebugInfoData>,
) -> Result<Compilation, SetupError> {
let mut code_memory = CodeMemory::new();
let (
compilation,
relocations,
@@ -209,257 +175,25 @@ impl Compiler {
vec![]
};
// Allocate all of the compiled functions into executable memory,
// copying over their contents.
let finished_functions = allocate_functions(&mut code_memory, &compilation, &relocations)
.map_err(|message| {
SetupError::Instantiate(InstantiationError::Resource(format!(
"failed to allocate memory for functions: {}",
message
)))
let (obj, unwind_info) = build_object(
&*self.isa,
&compilation,
&relocations,
&translation.module,
&dwarf_sections,
)?;
let obj = obj.write().map_err(|_| {
SetupError::Instantiate(InstantiationError::Resource(
"failed to create image memory".to_string(),
))
})?;
// Eagerly generate a entry trampoline for every type signature in the
// module. This should be "relatively lightweight" for most modules and
// guarantees that all functions (including indirect ones through
// tables) have a trampoline when invoked through the wasmtime API.
let mut cx = FunctionBuilderContext::new();
let mut trampolines = PrimaryMap::new();
for (_, (_, native_sig)) in translation.module.local.signatures.iter() {
let trampoline = make_trampoline(
&*self.isa,
&mut code_memory,
&mut cx,
native_sig,
std::mem::size_of::<u128>(),
)?;
trampolines.push(trampoline);
}
let jt_offsets = compilation.get_jt_offsets();
let code_range = get_code_range(&compilation, &finished_functions);
Ok(Compilation {
code_memory,
finished_functions,
code_range,
trampolines,
jt_offsets,
dwarf_sections,
obj,
unwind_info,
traps,
stack_maps,
address_transform,
})
}
}
/// Create a trampoline for invoking a function.
pub fn make_trampoline(
isa: &dyn TargetIsa,
code_memory: &mut CodeMemory,
fn_builder_ctx: &mut FunctionBuilderContext,
signature: &ir::Signature,
value_size: usize,
) -> Result<VMTrampoline, SetupError> {
let pointer_type = isa.pointer_type();
let mut wrapper_sig = ir::Signature::new(isa.frontend_config().default_call_conv);
// Add the callee `vmctx` parameter.
wrapper_sig.params.push(ir::AbiParam::special(
pointer_type,
ir::ArgumentPurpose::VMContext,
));
// Add the caller `vmctx` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
// Add the `callee_address` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
// Add the `values_vec` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
let mut context = Context::new();
context.func = ir::Function::with_name_signature(ir::ExternalName::user(0, 0), wrapper_sig);
{
let mut builder = FunctionBuilder::new(&mut context.func, fn_builder_ctx);
let block0 = builder.create_block();
builder.append_block_params_for_function_params(block0);
builder.switch_to_block(block0);
builder.seal_block(block0);
let (vmctx_ptr_val, caller_vmctx_ptr_val, callee_value, values_vec_ptr_val) = {
let params = builder.func.dfg.block_params(block0);
(params[0], params[1], params[2], params[3])
};
// Load the argument values out of `values_vec`.
let mflags = ir::MemFlags::trusted();
let callee_args = signature
.params
.iter()
.enumerate()
.map(|(i, r)| {
match i {
0 => vmctx_ptr_val,
1 => caller_vmctx_ptr_val,
_ =>
// i - 2 because vmctx and caller vmctx aren't passed through `values_vec`.
{
builder.ins().load(
r.value_type,
mflags,
values_vec_ptr_val,
((i - 2) * value_size) as i32,
)
}
}
})
.collect::<Vec<_>>();
let new_sig = builder.import_signature(signature.clone());
let call = builder
.ins()
.call_indirect(new_sig, callee_value, &callee_args);
let results = builder.func.dfg.inst_results(call).to_vec();
// Store the return values into `values_vec`.
let mflags = ir::MemFlags::trusted();
for (i, r) in results.iter().enumerate() {
builder
.ins()
.store(mflags, *r, values_vec_ptr_val, (i * value_size) as i32);
}
builder.ins().return_(&[]);
builder.finalize()
}
let mut code_buf = Vec::new();
let mut reloc_sink = RelocSink::default();
let mut trap_sink = binemit::NullTrapSink {};
let mut stackmap_sink = binemit::NullStackmapSink {};
context
.compile_and_emit(
isa,
&mut code_buf,
&mut reloc_sink,
&mut trap_sink,
&mut stackmap_sink,
)
.map_err(|error| {
SetupError::Compile(CompileError::Codegen(pretty_error(
&context.func,
Some(isa),
error,
)))
})?;
let unwind_info = context.create_unwind_info(isa).map_err(|error| {
SetupError::Compile(CompileError::Codegen(pretty_error(
&context.func,
Some(isa),
error,
)))
})?;
let ptr = code_memory
.allocate_for_function(
&CompiledFunction {
body: code_buf,
jt_offsets: context.func.jt_offsets,
unwind_info,
},
reloc_sink.relocs.iter(),
)
.map_err(|message| SetupError::Instantiate(InstantiationError::Resource(message)))?
.as_ptr();
Ok(unsafe { std::mem::transmute::<*const VMFunctionBody, VMTrampoline>(ptr) })
}
fn allocate_functions(
code_memory: &mut CodeMemory,
compilation: &wasmtime_environ::Compilation,
relocations: &Relocations,
) -> Result<PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>, String> {
if compilation.is_empty() {
return Ok(PrimaryMap::new());
}
let fat_ptrs = code_memory.allocate_for_compilation(compilation, relocations)?;
// Second, create a PrimaryMap from result vector of pointers.
let mut result = PrimaryMap::with_capacity(compilation.len());
for i in 0..fat_ptrs.len() {
let fat_ptr: *mut [VMFunctionBody] = fat_ptrs[i];
result.push(fat_ptr);
}
Ok(result)
}
/// We don't expect trampoline compilation to produce many relocations, so
/// this `RelocSink` just asserts that it doesn't recieve most of them, but
/// handles libcall ones.
#[derive(Default)]
pub struct RelocSink {
relocs: Vec<Relocation>,
}
impl RelocSink {
/// Returns collected relocations.
pub fn relocs(&self) -> &[Relocation] {
&self.relocs
}
}
impl binemit::RelocSink for RelocSink {
fn reloc_block(
&mut self,
_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_block_offset: binemit::CodeOffset,
) {
panic!("trampoline compilation should not produce block relocs");
}
fn reloc_external(
&mut self,
offset: binemit::CodeOffset,
_srcloc: ir::SourceLoc,
reloc: binemit::Reloc,
name: &ir::ExternalName,
addend: binemit::Addend,
) {
let reloc_target = if let ExternalName::LibCall(libcall) = *name {
RelocationTarget::LibCall(libcall)
} else {
panic!("unrecognized external name")
};
self.relocs.push(Relocation {
reloc,
reloc_target,
offset,
addend,
});
}
fn reloc_constant(
&mut self,
_code_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_constant_offset: ir::ConstantOffset,
) {
panic!("trampoline compilation should not produce constant relocs");
}
fn reloc_jt(
&mut self,
_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_jt: ir::JumpTable,
) {
panic!("trampoline compilation should not produce jump table relocs");
}
}

View File

@@ -7,12 +7,14 @@ use crate::code_memory::CodeMemory;
use crate::compiler::{Compilation, Compiler};
use crate::imports::resolve_imports;
use crate::link::link_module;
use crate::object::ObjectUnwindInfo;
use crate::resolver::Resolver;
use object::File as ObjectFile;
use std::any::Any;
use std::collections::HashMap;
use std::sync::Arc;
use thiserror::Error;
use wasmtime_debug::{read_debuginfo, write_debugsections_image, DwarfSection};
use wasmtime_debug::{create_gdbjit_image, read_debuginfo};
use wasmtime_environ::entity::{BoxedSlice, PrimaryMap};
use wasmtime_environ::isa::TargetIsa;
use wasmtime_environ::wasm::{DefinedFuncIndex, SignatureIndex};
@@ -49,6 +51,63 @@ pub enum SetupError {
DebugInfo(#[from] anyhow::Error),
}
// Contains all compilation artifacts.
struct CompilationArtifacts {
module: Module,
obj: Box<[u8]>,
unwind_info: Box<[ObjectUnwindInfo]>,
data_initializers: Box<[OwnedDataInitializer]>,
traps: Traps,
stack_maps: StackMaps,
address_transform: ModuleAddressMap,
}
impl CompilationArtifacts {
fn new(compiler: &Compiler, data: &[u8]) -> Result<Self, SetupError> {
let environ = ModuleEnvironment::new(compiler.frontend_config(), compiler.tunables());
let translation = environ
.translate(data)
.map_err(|error| SetupError::Compile(CompileError::Wasm(error)))?;
let mut debug_data = None;
if compiler.tunables().debug_info {
// TODO Do we want to ignore invalid DWARF data?
debug_data = Some(read_debuginfo(&data)?);
}
let Compilation {
obj,
unwind_info,
traps,
stack_maps,
address_transform,
} = compiler.compile(&translation, debug_data)?;
let ModuleTranslation {
module,
data_initializers,
..
} = translation;
let data_initializers = data_initializers
.into_iter()
.map(OwnedDataInitializer::new)
.collect::<Vec<_>>()
.into_boxed_slice();
Ok(Self {
module,
obj: obj.into_boxed_slice(),
unwind_info: unwind_info.into_boxed_slice(),
data_initializers,
traps,
stack_maps,
address_transform,
})
}
}
struct FinishedFunctions(BoxedSlice<DefinedFuncIndex, *mut [VMFunctionBody]>);
unsafe impl Send for FinishedFunctions {}
@@ -80,55 +139,31 @@ impl CompiledModule {
data: &'data [u8],
profiler: &dyn ProfilingAgent,
) -> Result<Self, SetupError> {
let environ = ModuleEnvironment::new(compiler.frontend_config(), compiler.tunables());
let artifacts = CompilationArtifacts::new(compiler, data)?;
let translation = environ
.translate(data)
.map_err(|error| SetupError::Compile(CompileError::Wasm(error)))?;
let mut debug_data = None;
if compiler.tunables().debug_info {
// TODO Do we want to ignore invalid DWARF data?
debug_data = Some(read_debuginfo(&data)?);
}
let Compilation {
mut code_memory,
finished_functions,
code_range,
trampolines,
jt_offsets,
dwarf_sections,
let CompilationArtifacts {
module,
obj,
unwind_info,
data_initializers,
traps,
stack_maps,
address_transform,
} = compiler.compile(&translation, debug_data)?;
} = artifacts;
let ModuleTranslation {
module,
data_initializers,
..
} = translation;
link_module(&mut code_memory, &module, &finished_functions, &jt_offsets);
// Make all code compiled thus far executable.
code_memory.publish(compiler.isa());
let data_initializers = data_initializers
.into_iter()
.map(OwnedDataInitializer::new)
.collect::<Vec<_>>()
.into_boxed_slice();
// Allocate all of the compiled functions into executable memory,
// copying over their contents.
let (code_memory, code_range, finished_functions, trampolines) =
build_code_memory(compiler.isa(), &obj, &module, unwind_info).map_err(|message| {
SetupError::Instantiate(InstantiationError::Resource(format!(
"failed to build code memory for functions: {}",
message
)))
})?;
// Register GDB JIT images; initialize profiler and load the wasm module.
let dbg_jit_registration = if !dwarf_sections.is_empty() {
let bytes = create_dbg_image(
dwarf_sections,
compiler.isa(),
code_range,
&finished_functions,
)?;
let dbg_jit_registration = if compiler.tunables().debug_info {
let bytes = create_dbg_image(obj.to_vec(), code_range, &module, &finished_functions)?;
profiler.module_load(&module, &finished_functions, Some(&bytes));
@@ -282,15 +317,64 @@ impl OwnedDataInitializer {
}
fn create_dbg_image(
dwarf_sections: Vec<DwarfSection>,
isa: &dyn TargetIsa,
obj: Vec<u8>,
code_range: (*const u8, usize),
module: &Module,
finished_functions: &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
) -> Result<Vec<u8>, SetupError> {
let funcs = finished_functions
.values()
.map(|allocated: &*mut [VMFunctionBody]| (*allocated) as *const u8)
.collect::<Vec<_>>();
write_debugsections_image(isa, dwarf_sections, code_range, &funcs)
create_gdbjit_image(obj, code_range, module.local.num_imported_funcs, &funcs)
.map_err(SetupError::DebugInfo)
}
fn build_code_memory(
isa: &dyn TargetIsa,
obj: &[u8],
module: &Module,
unwind_info: Box<[ObjectUnwindInfo]>,
) -> Result<
(
CodeMemory,
(*const u8, usize),
PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
PrimaryMap<SignatureIndex, VMTrampoline>,
),
String,
> {
let obj = ObjectFile::parse(obj).map_err(|_| "Unable to read obj".to_string())?;
let mut code_memory = CodeMemory::new();
let allocation = code_memory.allocate_for_object(&obj, &unwind_info)?;
// Second, create a PrimaryMap from result vector of pointers.
let mut finished_functions = PrimaryMap::new();
for (i, fat_ptr) in allocation.funcs() {
let fat_ptr: *mut [VMFunctionBody] = fat_ptr;
assert_eq!(
Some(finished_functions.push(fat_ptr)),
module.local.defined_func_index(i)
);
}
let mut trampolines = PrimaryMap::new();
for (i, fat_ptr) in allocation.trampolines() {
let fat_ptr =
unsafe { std::mem::transmute::<*const VMFunctionBody, VMTrampoline>(fat_ptr.as_ptr()) };
assert_eq!(trampolines.push(fat_ptr), i);
}
let code_range = allocation.code_range();
link_module(&obj, &module, code_range, &finished_functions);
let code_range = (code_range.as_ptr(), code_range.len());
// Make all code compiled thus far executable.
code_memory.publish(isa);
Ok((code_memory, code_range, finished_functions, trampolines))
}

View File

@@ -26,6 +26,7 @@ mod compiler;
mod imports;
mod instantiate;
mod link;
mod object;
mod resolver;
mod unwind;
@@ -33,7 +34,7 @@ pub mod native;
pub mod trampoline;
pub use crate::code_memory::CodeMemory;
pub use crate::compiler::{make_trampoline, Compilation, CompilationStrategy, Compiler};
pub use crate::compiler::{Compilation, CompilationStrategy, Compiler};
pub use crate::instantiate::{CompiledModule, SetupError};
pub use crate::link::link_module;
pub use crate::resolver::{NullResolver, Resolver};

View File

@@ -1,96 +1,87 @@
//! Linking for JIT-compiled code.
use crate::CodeMemory;
use cranelift_codegen::binemit::Reloc;
use cranelift_codegen::ir::JumpTableOffsets;
use crate::object::utils::try_parse_func_name;
use object::read::{Object, ObjectSection, Relocation, RelocationTarget};
use object::{elf, File, RelocationEncoding, RelocationKind};
use std::ptr::{read_unaligned, write_unaligned};
use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::wasm::DefinedFuncIndex;
use wasmtime_environ::{Module, Relocation, RelocationTarget};
use wasmtime_environ::Module;
use wasmtime_runtime::libcalls;
use wasmtime_runtime::VMFunctionBody;
/// Links a module that has been compiled with `compiled_module` in `wasmtime-environ`.
///
/// Performs all required relocations inside the function code, provided the necessary metadata.
/// The relocations data provided in the object file, see object.rs for details.
///
/// Currently, the produced ELF image can be trusted.
/// TODO refactor logic to remove panics and add defensive code the image data
/// becomes untrusted.
pub fn link_module(
code_memory: &mut CodeMemory,
obj: &File,
module: &Module,
code_range: &mut [u8],
finished_functions: &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
jt_offsets: &PrimaryMap<DefinedFuncIndex, JumpTableOffsets>,
) {
for (fatptr, r) in code_memory.unpublished_relocations() {
let body = fatptr as *const VMFunctionBody;
apply_reloc(module, finished_functions, jt_offsets, body, r);
// Read the ".text" section and process its relocations.
let text_section = obj.section_by_name(".text").unwrap();
let body = code_range.as_ptr() as *const VMFunctionBody;
for (offset, r) in text_section.relocations() {
apply_reloc(module, obj, finished_functions, body, offset, r);
}
}
fn apply_reloc(
module: &Module,
obj: &File,
finished_functions: &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>,
jt_offsets: &PrimaryMap<DefinedFuncIndex, JumpTableOffsets>,
body: *const VMFunctionBody,
r: &Relocation,
offset: u64,
r: Relocation,
) {
use self::libcalls::*;
let target_func_address: usize = match r.reloc_target {
RelocationTarget::UserFunc(index) => match module.local.defined_func_index(index) {
Some(f) => {
let fatptr: *const [VMFunctionBody] = finished_functions[f];
fatptr as *const VMFunctionBody as usize
}
None => panic!("direct call to import"),
},
RelocationTarget::LibCall(libcall) => {
use cranelift_codegen::ir::LibCall::*;
match libcall {
UdivI64 => wasmtime_i64_udiv as usize,
SdivI64 => wasmtime_i64_sdiv as usize,
UremI64 => wasmtime_i64_urem as usize,
SremI64 => wasmtime_i64_srem as usize,
IshlI64 => wasmtime_i64_ishl as usize,
UshrI64 => wasmtime_i64_ushr as usize,
SshrI64 => wasmtime_i64_sshr as usize,
CeilF32 => wasmtime_f32_ceil as usize,
FloorF32 => wasmtime_f32_floor as usize,
TruncF32 => wasmtime_f32_trunc as usize,
NearestF32 => wasmtime_f32_nearest as usize,
CeilF64 => wasmtime_f64_ceil as usize,
FloorF64 => wasmtime_f64_floor as usize,
TruncF64 => wasmtime_f64_trunc as usize,
NearestF64 => wasmtime_f64_nearest as usize,
other => panic!("unexpected libcall: {}", other),
}
}
RelocationTarget::JumpTable(func_index, jt) => {
match module.local.defined_func_index(func_index) {
Some(f) => {
let offset = *jt_offsets
.get(f)
.and_then(|ofs| ofs.get(jt))
.expect("func jump table");
let fatptr: *const [VMFunctionBody] = finished_functions[f];
fatptr as *const VMFunctionBody as usize + offset as usize
let target_func_address: usize = match r.target() {
RelocationTarget::Symbol(i) => {
// Processing relocation target is a named symbols that is compiled
// wasm function or runtime libcall.
let sym = obj.symbol_by_index(i).unwrap();
match sym.name() {
Some(name) => {
if let Some(index) = try_parse_func_name(name) {
match module.local.defined_func_index(index) {
Some(f) => {
let fatptr: *const [VMFunctionBody] = finished_functions[f];
fatptr as *const VMFunctionBody as usize
}
None => panic!("direct call to import"),
}
} else if let Some(addr) = to_libcall_address(name) {
addr
} else {
panic!("unknown function to link: {}", name);
}
}
None => panic!("func index of jump table"),
None => panic!("unexpected relocation target: not a symbol"),
}
}
_ => panic!("unexpected relocation target"),
};
match r.reloc {
match (r.kind(), r.encoding(), r.size()) {
#[cfg(target_pointer_width = "64")]
Reloc::Abs8 => unsafe {
let reloc_address = body.add(r.offset as usize) as usize;
let reloc_addend = r.addend as isize;
(RelocationKind::Absolute, RelocationEncoding::Generic, 64) => unsafe {
let reloc_address = body.add(offset as usize) as usize;
let reloc_addend = r.addend() as isize;
let reloc_abs = (target_func_address as u64)
.checked_add(reloc_addend as u64)
.unwrap();
write_unaligned(reloc_address as *mut u64, reloc_abs);
},
#[cfg(target_pointer_width = "32")]
Reloc::X86PCRel4 => unsafe {
let reloc_address = body.add(r.offset as usize) as usize;
let reloc_addend = r.addend as isize;
(RelocationKind::Relative, RelocationEncoding::Generic, 32) => unsafe {
let reloc_address = body.add(offset as usize) as usize;
let reloc_addend = r.addend() as isize;
let reloc_delta_u32 = (target_func_address as u32)
.wrapping_sub(reloc_address as u32)
.checked_add(reloc_addend as u32)
@@ -98,18 +89,18 @@ fn apply_reloc(
write_unaligned(reloc_address as *mut u32, reloc_delta_u32);
},
#[cfg(target_pointer_width = "32")]
Reloc::X86CallPCRel4 => unsafe {
let reloc_address = body.add(r.offset as usize) as usize;
let reloc_addend = r.addend as isize;
(RelocationKind::Relative, RelocationEncoding::X86Branch, 32) => unsafe {
let reloc_address = body.add(offset as usize) as usize;
let reloc_addend = r.addend() as isize;
let reloc_delta_u32 = (target_func_address as u32)
.wrapping_sub(reloc_address as u32)
.wrapping_add(reloc_addend as u32);
write_unaligned(reloc_address as *mut u32, reloc_delta_u32);
},
#[cfg(target_pointer_width = "64")]
Reloc::X86CallPCRel4 => unsafe {
let reloc_address = body.add(r.offset as usize) as usize;
let reloc_addend = r.addend as isize;
(RelocationKind::Relative, RelocationEncoding::X86Branch, 32) => unsafe {
let reloc_address = body.add(offset as usize) as usize;
let reloc_addend = r.addend() as isize;
let reloc_delta_u64 = (target_func_address as u64)
.wrapping_sub(reloc_address as u64)
.wrapping_add(reloc_addend as u64);
@@ -119,12 +110,9 @@ fn apply_reloc(
);
write_unaligned(reloc_address as *mut u32, reloc_delta_u64 as u32);
},
Reloc::X86PCRelRodata4 => {
// ignore
}
Reloc::Arm64Call => unsafe {
let reloc_address = body.add(r.offset as usize) as usize;
let reloc_addend = r.addend as isize;
(RelocationKind::Elf(elf::R_AARCH64_CALL26), RelocationEncoding::Generic, 32) => unsafe {
let reloc_address = body.add(offset as usize) as usize;
let reloc_addend = r.addend() as isize;
let reloc_delta = (target_func_address as u64).wrapping_sub(reloc_address as u64);
// TODO: come up with a PLT-like solution for longer calls. We can't extend the
// code segment at this point, but we could conservatively allocate space at the
@@ -139,6 +127,24 @@ fn apply_reloc(
let new_insn = (insn & 0xfc00_0000) | (delta_bits & 0x03ff_ffff);
write_unaligned(reloc_address as *mut u32, new_insn);
},
_ => panic!("unsupported reloc kind"),
other => panic!("unsupported reloc kind: {:?}", other),
}
}
fn to_libcall_address(name: &str) -> Option<usize> {
use self::libcalls::*;
use crate::for_each_libcall;
macro_rules! add_libcall_symbol {
[$(($libcall:ident, $export:ident)),*] => {
Some(match name {
$(
stringify!($export) => $export as usize,
)+
_ => {
return None;
}
})
};
}
for_each_libcall!(add_libcall_symbol)
}

386
crates/jit/src/object.rs Normal file
View File

@@ -0,0 +1,386 @@
//! Object file generation.
//!
//! Creates ELF image based on `Compilation` information. The ELF contains
//! functions and trampolines in the ".text" section. It also contains all
//! relocation records for linking stage. If DWARF sections exist, their
//! content will be written as well.
//!
//! The object file has symbols for each function and trampoline, as well as
//! symbols that refer libcalls.
//!
//! The function symbol names have format "_wasm_function_N", where N is
//! `FuncIndex`. The defined wasm function symbols refer to a JIT compiled
//! function body, the imported wasm function do not. The trampolines symbol
//! names have format "_trampoline_N", where N is `SignatureIndex`.
use super::trampoline::build_trampoline;
use cranelift_codegen::binemit::Reloc;
use cranelift_codegen::ir::{JumpTableOffsets, LibCall};
use cranelift_frontend::FunctionBuilderContext;
use object::write::{
Object, Relocation as ObjectRelocation, SectionId, StandardSegment, Symbol, SymbolId,
SymbolSection,
};
use object::{
elf, Architecture, BinaryFormat, Endianness, RelocationEncoding, RelocationKind, SectionKind,
SymbolFlags, SymbolKind, SymbolScope,
};
use std::collections::HashMap;
use wasmtime_debug::{DwarfSection, DwarfSectionRelocTarget};
use wasmtime_environ::entity::{EntityRef, PrimaryMap};
use wasmtime_environ::isa::{unwind::UnwindInfo, TargetIsa};
use wasmtime_environ::wasm::{DefinedFuncIndex, FuncIndex, SignatureIndex};
use wasmtime_environ::{Compilation, Module, Relocation, RelocationTarget, Relocations};
fn to_object_relocations<'a>(
it: impl Iterator<Item = &'a Relocation> + 'a,
off: u64,
module: &'a Module,
funcs: &'a PrimaryMap<FuncIndex, SymbolId>,
libcalls: &'a HashMap<LibCall, SymbolId>,
jt_offsets: &'a PrimaryMap<DefinedFuncIndex, JumpTableOffsets>,
) -> impl Iterator<Item = ObjectRelocation> + 'a {
it.filter_map(move |r| {
let (symbol, symbol_offset) = match r.reloc_target {
RelocationTarget::UserFunc(index) => (funcs[index], 0),
RelocationTarget::LibCall(call) => (libcalls[&call], 0),
RelocationTarget::JumpTable(f, jt) => {
let df = module.local.defined_func_index(f).unwrap();
let offset = *jt_offsets
.get(df)
.and_then(|ofs| ofs.get(jt))
.expect("func jump table");
(funcs[f], offset)
}
};
let (kind, encoding, size) = match r.reloc {
Reloc::Abs4 => (RelocationKind::Absolute, RelocationEncoding::Generic, 32),
Reloc::Abs8 => (RelocationKind::Absolute, RelocationEncoding::Generic, 64),
Reloc::X86PCRel4 => (RelocationKind::Relative, RelocationEncoding::Generic, 32),
Reloc::X86CallPCRel4 => (RelocationKind::Relative, RelocationEncoding::X86Branch, 32),
// TODO: Get Cranelift to tell us when we can use
// R_X86_64_GOTPCRELX/R_X86_64_REX_GOTPCRELX.
Reloc::X86CallPLTRel4 => (
RelocationKind::PltRelative,
RelocationEncoding::X86Branch,
32,
),
Reloc::X86GOTPCRel4 => (RelocationKind::GotRelative, RelocationEncoding::Generic, 32),
Reloc::ElfX86_64TlsGd => (
RelocationKind::Elf(elf::R_X86_64_TLSGD),
RelocationEncoding::Generic,
32,
),
Reloc::X86PCRelRodata4 => {
return None;
}
Reloc::Arm64Call => (
RelocationKind::Elf(elf::R_AARCH64_CALL26),
RelocationEncoding::Generic,
32,
),
other => unimplemented!("Unimplemented relocation {:?}", other),
};
Some(ObjectRelocation {
offset: off + r.offset as u64,
size,
kind,
encoding,
symbol,
addend: r.addend.wrapping_add(symbol_offset as i64),
})
})
}
fn to_object_architecture(
arch: target_lexicon::Architecture,
) -> Result<Architecture, anyhow::Error> {
use target_lexicon::Architecture::*;
Ok(match arch {
I386 | I586 | I686 => Architecture::I386,
X86_64 => Architecture::X86_64,
Arm(_) => Architecture::Arm,
Aarch64(_) => Architecture::Aarch64,
architecture => {
anyhow::bail!("target architecture {:?} is unsupported", architecture,);
}
})
}
const TEXT_SECTION_NAME: &[u8] = b".text";
/// Unwind information for object files functions (including trampolines).
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum ObjectUnwindInfo {
Func(FuncIndex, UnwindInfo),
Trampoline(SignatureIndex, UnwindInfo),
}
fn process_unwind_info(info: &UnwindInfo, obj: &mut Object, code_section: SectionId) {
if let UnwindInfo::WindowsX64(info) = &info {
// Windows prefers Unwind info after the code -- writing it here.
let unwind_size = info.emit_size();
let mut unwind_info = vec![0; unwind_size];
info.emit(&mut unwind_info);
let _off = obj.append_section_data(code_section, &unwind_info, 4);
}
}
// Builds ELF image from the module `Compilation`.
pub(crate) fn build_object(
isa: &dyn TargetIsa,
compilation: &Compilation,
relocations: &Relocations,
module: &Module,
dwarf_sections: &[DwarfSection],
) -> Result<(Object, Vec<ObjectUnwindInfo>), anyhow::Error> {
const CODE_SECTION_ALIGNMENT: u64 = 0x1000;
assert_eq!(
isa.triple().architecture.endianness(),
Ok(target_lexicon::Endianness::Little)
);
let mut obj = Object::new(
BinaryFormat::Elf,
to_object_architecture(isa.triple().architecture)?,
Endianness::Little,
);
// Entire code (functions and trampolines) will be placed
// in the ".text" section.
let section_id = obj.add_section(
obj.segment_name(StandardSegment::Text).to_vec(),
TEXT_SECTION_NAME.to_vec(),
SectionKind::Text,
);
let mut unwind_info = Vec::new();
// Create symbols for imports -- needed during linking.
let mut func_symbols = PrimaryMap::with_capacity(compilation.len());
for index in 0..module.local.num_imported_funcs {
let symbol_id = obj.add_symbol(Symbol {
name: utils::func_symbol_name(FuncIndex::new(index))
.as_bytes()
.to_vec(),
value: 0,
size: 0,
kind: SymbolKind::Text,
scope: SymbolScope::Linkage,
weak: false,
section: SymbolSection::Undefined,
flags: SymbolFlags::None,
});
func_symbols.push(symbol_id);
}
// Create symbols and section data for the compiled functions.
for (index, func) in compilation.into_iter().enumerate() {
let off = obj.append_section_data(section_id, &func.body, 1);
let symbol_id = obj.add_symbol(Symbol {
name: utils::func_symbol_name(module.local.func_index(DefinedFuncIndex::new(index)))
.as_bytes()
.to_vec(),
value: off,
size: func.body.len() as u64,
kind: SymbolKind::Text,
scope: SymbolScope::Compilation,
weak: false,
section: SymbolSection::Section(section_id),
flags: SymbolFlags::None,
});
func_symbols.push(symbol_id);
// Preserve function unwind info.
if let Some(info) = &func.unwind_info {
process_unwind_info(info, &mut obj, section_id);
unwind_info.push(ObjectUnwindInfo::Func(
FuncIndex::new(module.local.num_imported_funcs + index),
info.clone(),
))
}
}
let mut trampoline_relocs = HashMap::new();
let mut cx = FunctionBuilderContext::new();
// Build trampolines for every signature.
for (i, (_, native_sig)) in module.local.signatures.iter() {
let (func, relocs) =
build_trampoline(isa, &mut cx, native_sig, std::mem::size_of::<u128>())?;
let off = obj.append_section_data(section_id, &func.body, 1);
let symbol_id = obj.add_symbol(Symbol {
name: utils::trampoline_symbol_name(i).as_bytes().to_vec(),
value: off,
size: func.body.len() as u64,
kind: SymbolKind::Text,
scope: SymbolScope::Compilation,
weak: false,
section: SymbolSection::Section(section_id),
flags: SymbolFlags::None,
});
trampoline_relocs.insert(symbol_id, relocs);
// Preserve trampoline function unwind info.
if let Some(info) = &func.unwind_info {
process_unwind_info(info, &mut obj, section_id);
unwind_info.push(ObjectUnwindInfo::Trampoline(i, info.clone()))
}
}
obj.append_section_data(section_id, &[], CODE_SECTION_ALIGNMENT);
// If we have DWARF data, write it in the object file.
let (debug_bodies, debug_relocs) = dwarf_sections
.into_iter()
.map(|s| ((s.name, &s.body), (s.name, &s.relocs)))
.unzip::<_, _, Vec<_>, Vec<_>>();
let mut dwarf_sections_ids = HashMap::new();
for (name, body) in debug_bodies {
let segment = obj.segment_name(StandardSegment::Debug).to_vec();
let section_id = obj.add_section(segment, name.as_bytes().to_vec(), SectionKind::Debug);
dwarf_sections_ids.insert(name.to_string(), section_id);
obj.append_section_data(section_id, &body, 1);
}
let libcalls = write_libcall_symbols(&mut obj);
let jt_offsets = compilation.get_jt_offsets();
// Write all functions relocations.
for (index, relocs) in relocations.into_iter() {
let func_index = module.local.func_index(index);
let (_, off) = obj
.symbol_section_and_offset(func_symbols[func_index])
.unwrap();
for r in to_object_relocations(
relocs.iter(),
off,
module,
&func_symbols,
&libcalls,
&jt_offsets,
) {
obj.add_relocation(section_id, r)?;
}
}
for (symbol, relocs) in trampoline_relocs {
let (_, off) = obj.symbol_section_and_offset(symbol).unwrap();
for r in to_object_relocations(
relocs.iter(),
off,
module,
&func_symbols,
&libcalls,
&jt_offsets,
) {
obj.add_relocation(section_id, r)?;
}
}
// Write all debug data relocations.
for (name, relocs) in debug_relocs {
let section_id = *dwarf_sections_ids.get(name).unwrap();
for reloc in relocs {
let target_symbol = match reloc.target {
DwarfSectionRelocTarget::Func(index) => func_symbols[FuncIndex::new(index)],
DwarfSectionRelocTarget::Section(name) => {
obj.section_symbol(*dwarf_sections_ids.get(name).unwrap())
}
};
obj.add_relocation(
section_id,
ObjectRelocation {
offset: u64::from(reloc.offset),
size: reloc.size << 3,
kind: RelocationKind::Absolute,
encoding: RelocationEncoding::Generic,
symbol: target_symbol,
addend: i64::from(reloc.addend),
},
)?;
}
}
Ok((obj, unwind_info))
}
/// Iterates through all `LibCall` members and all runtime exported functions.
#[macro_export]
macro_rules! for_each_libcall {
($op:ident) => {
$op![
(UdivI64, wasmtime_i64_udiv),
(UdivI64, wasmtime_i64_udiv),
(SdivI64, wasmtime_i64_sdiv),
(UremI64, wasmtime_i64_urem),
(SremI64, wasmtime_i64_srem),
(IshlI64, wasmtime_i64_ishl),
(UshrI64, wasmtime_i64_ushr),
(SshrI64, wasmtime_i64_sshr),
(CeilF32, wasmtime_f32_ceil),
(FloorF32, wasmtime_f32_floor),
(TruncF32, wasmtime_f32_trunc),
(NearestF32, wasmtime_f32_nearest),
(CeilF64, wasmtime_f64_ceil),
(FloorF64, wasmtime_f64_floor),
(TruncF64, wasmtime_f64_trunc),
(NearestF64, wasmtime_f64_nearest)
];
};
}
fn write_libcall_symbols(obj: &mut Object) -> HashMap<LibCall, SymbolId> {
let mut libcalls = HashMap::new();
macro_rules! add_libcall_symbol {
[$(($libcall:ident, $export:ident)),*] => {{
$(
let symbol_id = obj.add_symbol(Symbol {
name: stringify!($export).as_bytes().to_vec(),
value: 0,
size: 0,
kind: SymbolKind::Text,
scope: SymbolScope::Linkage,
weak: true,
section: SymbolSection::Undefined,
flags: SymbolFlags::None,
});
libcalls.insert(LibCall::$libcall, symbol_id);
)+
}};
}
for_each_libcall!(add_libcall_symbol);
libcalls
}
pub(crate) mod utils {
use wasmtime_environ::entity::EntityRef;
use wasmtime_environ::wasm::{FuncIndex, SignatureIndex};
pub const FUNCTION_PREFIX: &str = "_wasm_function_";
pub const TRAMPOLINE_PREFIX: &str = "_trampoline_";
pub fn func_symbol_name(index: FuncIndex) -> String {
format!("_wasm_function_{}", index.index())
}
pub fn try_parse_func_name(name: &str) -> Option<FuncIndex> {
if !name.starts_with(FUNCTION_PREFIX) {
return None;
}
name[FUNCTION_PREFIX.len()..]
.parse()
.ok()
.map(FuncIndex::new)
}
pub fn trampoline_symbol_name(index: SignatureIndex) -> String {
format!("_trampoline_{}", index.index())
}
pub fn try_parse_trampoline_name(name: &str) -> Option<SignatureIndex> {
if !name.starts_with(TRAMPOLINE_PREFIX) {
return None;
}
name[TRAMPOLINE_PREFIX.len()..]
.parse()
.ok()
.map(SignatureIndex::new)
}
}

View File

@@ -1,6 +1,16 @@
#![allow(missing_docs)]
use crate::code_memory::CodeMemory;
use crate::instantiate::SetupError;
use cranelift_codegen::ir::InstBuilder;
use cranelift_codegen::isa::TargetIsa;
use wasmtime_environ::{CompileError, CompiledFunction, Relocation, RelocationTarget};
use wasmtime_runtime::{InstantiationError, VMFunctionBody, VMTrampoline};
pub mod ir {
pub(super) use cranelift_codegen::ir::{
AbiParam, ArgumentPurpose, ConstantOffset, JumpTable, Signature, SourceLoc,
};
pub use cranelift_codegen::ir::{
ExternalName, Function, InstBuilder, MemFlags, StackSlotData, StackSlotKind,
};
@@ -10,7 +20,209 @@ pub use cranelift_codegen::Context;
pub use cranelift_frontend::{FunctionBuilder, FunctionBuilderContext};
pub mod binemit {
pub use crate::compiler::RelocSink as TrampolineRelocSink;
pub use cranelift_codegen::binemit::NullTrapSink;
pub(super) use cranelift_codegen::binemit::{Addend, Reloc, RelocSink};
pub use cranelift_codegen::binemit::{CodeOffset, NullStackmapSink, TrapSink};
}
/// Create a trampoline for invoking a function.
pub fn make_trampoline(
isa: &dyn TargetIsa,
code_memory: &mut CodeMemory,
fn_builder_ctx: &mut FunctionBuilderContext,
signature: &ir::Signature,
value_size: usize,
) -> Result<VMTrampoline, SetupError> {
let (compiled_function, relocs) = build_trampoline(isa, fn_builder_ctx, signature, value_size)?;
assert!(relocs.is_empty());
let ptr = code_memory
.allocate_for_function(&compiled_function)
.map_err(|message| SetupError::Instantiate(InstantiationError::Resource(message)))?
.as_ptr();
Ok(unsafe { std::mem::transmute::<*const VMFunctionBody, VMTrampoline>(ptr) })
}
pub(crate) fn build_trampoline(
isa: &dyn TargetIsa,
fn_builder_ctx: &mut FunctionBuilderContext,
signature: &ir::Signature,
value_size: usize,
) -> Result<(CompiledFunction, Vec<Relocation>), SetupError> {
let pointer_type = isa.pointer_type();
let mut wrapper_sig = ir::Signature::new(isa.frontend_config().default_call_conv);
// Add the callee `vmctx` parameter.
wrapper_sig.params.push(ir::AbiParam::special(
pointer_type,
ir::ArgumentPurpose::VMContext,
));
// Add the caller `vmctx` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
// Add the `callee_address` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
// Add the `values_vec` parameter.
wrapper_sig.params.push(ir::AbiParam::new(pointer_type));
let mut context = Context::new();
context.func = ir::Function::with_name_signature(ir::ExternalName::user(0, 0), wrapper_sig);
{
let mut builder = FunctionBuilder::new(&mut context.func, fn_builder_ctx);
let block0 = builder.create_block();
builder.append_block_params_for_function_params(block0);
builder.switch_to_block(block0);
builder.seal_block(block0);
let (vmctx_ptr_val, caller_vmctx_ptr_val, callee_value, values_vec_ptr_val) = {
let params = builder.func.dfg.block_params(block0);
(params[0], params[1], params[2], params[3])
};
// Load the argument values out of `values_vec`.
let mflags = ir::MemFlags::trusted();
let callee_args = signature
.params
.iter()
.enumerate()
.map(|(i, r)| {
match i {
0 => vmctx_ptr_val,
1 => caller_vmctx_ptr_val,
_ =>
// i - 2 because vmctx and caller vmctx aren't passed through `values_vec`.
{
builder.ins().load(
r.value_type,
mflags,
values_vec_ptr_val,
((i - 2) * value_size) as i32,
)
}
}
})
.collect::<Vec<_>>();
let new_sig = builder.import_signature(signature.clone());
let call = builder
.ins()
.call_indirect(new_sig, callee_value, &callee_args);
let results = builder.func.dfg.inst_results(call).to_vec();
// Store the return values into `values_vec`.
let mflags = ir::MemFlags::trusted();
for (i, r) in results.iter().enumerate() {
builder
.ins()
.store(mflags, *r, values_vec_ptr_val, (i * value_size) as i32);
}
builder.ins().return_(&[]);
builder.finalize()
}
let mut code_buf = Vec::new();
let mut reloc_sink = TrampolineRelocSink::default();
let mut trap_sink = binemit::NullTrapSink {};
let mut stackmap_sink = binemit::NullStackmapSink {};
context
.compile_and_emit(
isa,
&mut code_buf,
&mut reloc_sink,
&mut trap_sink,
&mut stackmap_sink,
)
.map_err(|error| {
SetupError::Compile(CompileError::Codegen(pretty_error(
&context.func,
Some(isa),
error,
)))
})?;
let unwind_info = context.create_unwind_info(isa).map_err(|error| {
SetupError::Compile(CompileError::Codegen(pretty_error(
&context.func,
Some(isa),
error,
)))
})?;
Ok((
CompiledFunction {
body: code_buf,
jt_offsets: context.func.jt_offsets,
unwind_info,
},
reloc_sink.relocs,
))
}
/// We don't expect trampoline compilation to produce many relocations, so
/// this `RelocSink` just asserts that it doesn't recieve most of them, but
/// handles libcall ones.
#[derive(Default)]
pub struct TrampolineRelocSink {
relocs: Vec<Relocation>,
}
impl TrampolineRelocSink {
/// Returns collected relocations.
pub fn relocs(&self) -> &[Relocation] {
&self.relocs
}
}
impl binemit::RelocSink for TrampolineRelocSink {
fn reloc_block(
&mut self,
_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_block_offset: binemit::CodeOffset,
) {
panic!("trampoline compilation should not produce block relocs");
}
fn reloc_external(
&mut self,
offset: binemit::CodeOffset,
_srcloc: ir::SourceLoc,
reloc: binemit::Reloc,
name: &ir::ExternalName,
addend: binemit::Addend,
) {
let reloc_target = if let ir::ExternalName::LibCall(libcall) = *name {
RelocationTarget::LibCall(libcall)
} else {
panic!("unrecognized external name")
};
self.relocs.push(Relocation {
reloc,
reloc_target,
offset,
addend,
});
}
fn reloc_constant(
&mut self,
_code_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_constant_offset: ir::ConstantOffset,
) {
panic!("trampoline compilation should not produce constant relocs");
}
fn reloc_jt(
&mut self,
_offset: binemit::CodeOffset,
_reloc: binemit::Reloc,
_jt: ir::JumpTable,
) {
panic!("trampoline compilation should not produce jump table relocs");
}
}