Refactors JIT tests to not be concurrent. (#414)
This changes the way the JIT engine is tested by focusing on two areas: * Did NewModuleEngine set a finalizer for each compiled function correctly? * Did ModuleEngine.Close remove its compiled function correctly? A later change will do parallel tests for both engines and won't have to at the same time check how finalizers work. Signed-off-by: Adrian Cole <adrian@tetrate.io>
This commit is contained in:
@@ -27,8 +27,24 @@ import (
|
||||
publicwasm "github.com/tetratelabs/wazero/wasm"
|
||||
)
|
||||
|
||||
func RunTestModuleEngine_Call(t *testing.T, newEngine func() wasm.Engine) {
|
||||
e := newEngine()
|
||||
type EngineTester interface {
|
||||
NewEngine() wasm.Engine
|
||||
InitTable(me wasm.ModuleEngine, initTableLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{}
|
||||
}
|
||||
|
||||
func RunTestEngine_NewModuleEngine(t *testing.T, et EngineTester) {
|
||||
e := et.NewEngine()
|
||||
|
||||
t.Run("sets module name", func(t *testing.T) {
|
||||
me, err := e.NewModuleEngine(t.Name(), nil, nil, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer closeModuleEngine(t, me)
|
||||
require.Equal(t, t.Name(), me.Name())
|
||||
})
|
||||
}
|
||||
|
||||
func RunTestModuleEngine_Call(t *testing.T, et EngineTester) {
|
||||
e := et.NewEngine()
|
||||
|
||||
// Define a basic function which defines one parameter. This is used to test results when incorrect arity is used.
|
||||
i64 := wasm.ValueTypeI64
|
||||
@@ -46,7 +62,7 @@ func RunTestModuleEngine_Call(t *testing.T, newEngine func() wasm.Engine) {
|
||||
// Compile the module
|
||||
me, err := e.NewModuleEngine(module.Name, nil, module.Functions, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer me.Close()
|
||||
defer closeModuleEngine(t, me)
|
||||
|
||||
// Create a call context which links the module to the module-engine compiled from it.
|
||||
ctx := newModuleContext(module, me)
|
||||
@@ -67,8 +83,14 @@ func RunTestModuleEngine_Call(t *testing.T, newEngine func() wasm.Engine) {
|
||||
})
|
||||
}
|
||||
|
||||
func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, initTable func(me wasm.ModuleEngine, initTableLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{}, newEngine func() wasm.Engine) {
|
||||
e := newEngine()
|
||||
// closeModuleEngine allows unit tests to check `Close` didn't err.
|
||||
func closeModuleEngine(t *testing.T, me wasm.ModuleEngine) {
|
||||
err := me.Close()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, et EngineTester) {
|
||||
e := et.NewEngine()
|
||||
|
||||
t.Run("no table elements", func(t *testing.T) {
|
||||
table := &wasm.TableInstance{Min: 2, Table: make([]interface{}, 2)}
|
||||
@@ -79,7 +101,7 @@ func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, initTable func(me was
|
||||
// Instantiate the module, which has nothing but an empty table.
|
||||
me, err := e.NewModuleEngine(t.Name(), importedFunctions, moduleFunctions, table, tableInit)
|
||||
require.NoError(t, err)
|
||||
defer me.Close()
|
||||
defer closeModuleEngine(t, me)
|
||||
|
||||
// Since there are no elements to initialize, we expect the table to be nil.
|
||||
require.Equal(t, table.Table, make([]interface{}, 2))
|
||||
@@ -99,10 +121,10 @@ func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, initTable func(me was
|
||||
// Instantiate the module whose table points to its own functions.
|
||||
me, err := e.NewModuleEngine(t.Name(), importedFunctions, moduleFunctions, table, tableInit)
|
||||
require.NoError(t, err)
|
||||
defer me.Close()
|
||||
defer closeModuleEngine(t, me)
|
||||
|
||||
// The functions mapped to the table are defined in the same moduleEngine
|
||||
require.Equal(t, table.Table, initTable(me, table.Min, tableInit))
|
||||
require.Equal(t, table.Table, et.InitTable(me, table.Min, tableInit))
|
||||
})
|
||||
|
||||
t.Run("imported function", func(t *testing.T) {
|
||||
@@ -119,15 +141,15 @@ func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, initTable func(me was
|
||||
// Imported functions are compiled before the importing module is instantiated.
|
||||
imported, err := e.NewModuleEngine(t.Name(), nil, importedFunctions, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer imported.Close()
|
||||
defer closeModuleEngine(t, imported)
|
||||
|
||||
// Instantiate the importing module, which is whose table is initialized.
|
||||
importing, err := e.NewModuleEngine(t.Name(), importedFunctions, moduleFunctions, table, tableInit)
|
||||
require.NoError(t, err)
|
||||
defer imported.Close()
|
||||
defer closeModuleEngine(t, importing)
|
||||
|
||||
// A moduleEngine's compiled function slice includes its imports, so the offsets is absolute.
|
||||
require.Equal(t, table.Table, initTable(importing, table.Min, tableInit))
|
||||
require.Equal(t, table.Table, et.InitTable(importing, table.Min, tableInit))
|
||||
})
|
||||
|
||||
t.Run("mixed functions", func(t *testing.T) {
|
||||
@@ -149,19 +171,19 @@ func RunTestEngine_NewModuleEngine_InitTable(t *testing.T, initTable func(me was
|
||||
// Imported functions are compiled before the importing module is instantiated.
|
||||
imported, err := e.NewModuleEngine(t.Name(), nil, importedFunctions, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer imported.Close()
|
||||
defer closeModuleEngine(t, imported)
|
||||
|
||||
// Instantiate the importing module, which is whose table is initialized.
|
||||
importing, err := e.NewModuleEngine(t.Name(), importedFunctions, moduleFunctions, table, tableInit)
|
||||
require.NoError(t, err)
|
||||
defer importing.Close()
|
||||
defer closeModuleEngine(t, importing)
|
||||
|
||||
// A moduleEngine's compiled function slice includes its imports, so the offsets are absolute.
|
||||
require.Equal(t, table.Table, initTable(importing, table.Min, tableInit))
|
||||
require.Equal(t, table.Table, et.InitTable(importing, table.Min, tableInit))
|
||||
})
|
||||
}
|
||||
|
||||
func RunTestModuleEngine_Call_HostFn(t *testing.T, newEngine func() wasm.Engine) {
|
||||
func RunTestModuleEngine_Call_HostFn(t *testing.T, et EngineTester) {
|
||||
memory := &wasm.MemoryInstance{}
|
||||
var ctxMemory publicwasm.Memory
|
||||
hostFn := reflect.ValueOf(func(ctx publicwasm.Module, v uint64) uint64 {
|
||||
@@ -169,7 +191,7 @@ func RunTestModuleEngine_Call_HostFn(t *testing.T, newEngine func() wasm.Engine)
|
||||
return v
|
||||
})
|
||||
|
||||
e := newEngine()
|
||||
e := et.NewEngine()
|
||||
module := &wasm.ModuleInstance{Memory: memory}
|
||||
modCtx := wasm.NewModuleContext(context.Background(), wasm.NewStore(e, wasm.Features20191205), module, nil)
|
||||
|
||||
|
||||
@@ -20,12 +20,15 @@ type Engine interface {
|
||||
|
||||
// ModuleEngine implements function calls for a given module.
|
||||
type ModuleEngine interface {
|
||||
// Closer releases the resources allocated by functions in this ModuleEngine.
|
||||
io.Closer
|
||||
// ^^ io.Closer not due to I/O, but to allow future static analysis to catch leaks (unclosed Closers).
|
||||
// Name returns the name of the module this engine was compiled for.
|
||||
Name() string
|
||||
|
||||
// Call invokes a function instance f with given parameters.
|
||||
// Returns the results from the function.
|
||||
// The ctx's context.Context will be the outer-most ancestor of the argument to wasm.Function.
|
||||
Call(ctx *ModuleContext, f *FunctionInstance, params ...uint64) (results []uint64, err error)
|
||||
|
||||
// Closer releases the resources allocated by functions in this ModuleEngine.
|
||||
io.Closer
|
||||
// ^^ io.Closer not due to I/O, but to allow future static analysis to catch leaks (unclosed Closers).
|
||||
}
|
||||
|
||||
@@ -150,16 +150,7 @@ type interpreterOp struct {
|
||||
rs []*wazeroir.InclusiveRange
|
||||
}
|
||||
|
||||
// Close implements io.Closer
|
||||
func (me *moduleEngine) Close() (err error) {
|
||||
// Release all the function instances declared in this module.
|
||||
for _, cf := range me.compiledFunctions[me.importedFunctionCounts:] {
|
||||
me.parentEngine.deleteCompiledFunction(cf.funcInstance)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// NewModuleEngine implements internalwasm.Engine NewModuleEngine
|
||||
// NewModuleEngine implements the same method as documented on internalwasm.Engine.
|
||||
func (e *engine) NewModuleEngine(name string, importedFunctions, moduleFunctions []*wasm.FunctionInstance, table *wasm.TableInstance, tableInit map[wasm.Index]wasm.Index) (wasm.ModuleEngine, error) {
|
||||
me := &moduleEngine{
|
||||
name: name,
|
||||
@@ -181,14 +172,14 @@ func (e *engine) NewModuleEngine(name string, importedFunctions, moduleFunctions
|
||||
if f.Kind == wasm.FunctionKindWasm {
|
||||
ir, err := wazeroir.Compile(f)
|
||||
if err != nil {
|
||||
me.Close()
|
||||
me.doClose() // safe because the reference to me was never leaked.
|
||||
// TODO(Adrian): extract Module.funcDesc so that errors here have more context
|
||||
return nil, fmt.Errorf("function[%d/%d] failed to lower to wazeroir: %w", i, len(moduleFunctions)-1, err)
|
||||
}
|
||||
|
||||
compiled, err = e.lowerIROps(f, ir.Operations)
|
||||
if err != nil {
|
||||
me.Close()
|
||||
me.doClose() // safe because the reference to me was never leaked.
|
||||
return nil, fmt.Errorf("function[%d/%d] failed to convert wazeroir operations: %w", i, len(moduleFunctions)-1, err)
|
||||
}
|
||||
} else {
|
||||
@@ -491,7 +482,12 @@ func (e *engine) lowerIROps(f *wasm.FunctionInstance,
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
// Call implements internalwasm.ModuleEngine Call.
|
||||
// Name implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Name() string {
|
||||
return me.name
|
||||
}
|
||||
|
||||
// Call implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Call(ctx *wasm.ModuleContext, f *wasm.FunctionInstance, params ...uint64) (results []uint64, err error) {
|
||||
compiled := me.compiledFunctions[f.Index]
|
||||
if compiled == nil {
|
||||
@@ -1569,3 +1565,16 @@ func (ce *callEngine) callNativeFunc(ctx *wasm.ModuleContext, f *compiledFunctio
|
||||
}
|
||||
ce.popFrame()
|
||||
}
|
||||
|
||||
// Close implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Close() error {
|
||||
me.doClose()
|
||||
return nil
|
||||
}
|
||||
|
||||
// doClose releases all the function instances declared in this module.
|
||||
func (me *moduleEngine) doClose() {
|
||||
for _, cf := range me.compiledFunctions[me.importedFunctionCounts:] {
|
||||
me.parentEngine.deleteCompiledFunction(cf.funcInstance)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -45,35 +45,39 @@ func TestCallEngine_PushFrame_StackOverflow(t *testing.T) {
|
||||
require.Panics(t, func() { vm.pushFrame(f4) })
|
||||
}
|
||||
|
||||
func TestEngine_NewModuleEngine(t *testing.T) {
|
||||
e := NewEngine()
|
||||
// et is used for tests defined in the enginetest package.
|
||||
var et = &engineTester{}
|
||||
|
||||
t.Run("sets module name", func(t *testing.T) {
|
||||
me, err := e.NewModuleEngine(t.Name(), nil, nil, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer me.Close()
|
||||
require.Equal(t, t.Name(), me.(*moduleEngine).name)
|
||||
})
|
||||
type engineTester struct {
|
||||
}
|
||||
|
||||
func (e engineTester) NewEngine() wasm.Engine {
|
||||
return NewEngine()
|
||||
}
|
||||
|
||||
func (e engineTester) InitTable(me wasm.ModuleEngine, initTableLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{} {
|
||||
table := make([]interface{}, initTableLen)
|
||||
internal := me.(*moduleEngine)
|
||||
for idx, fnidx := range initTableIdxToFnIdx {
|
||||
table[idx] = internal.compiledFunctions[fnidx]
|
||||
}
|
||||
return table
|
||||
}
|
||||
|
||||
func TestEngine_NewModuleEngine(t *testing.T) {
|
||||
enginetest.RunTestEngine_NewModuleEngine(t, et)
|
||||
}
|
||||
|
||||
func TestEngine_NewModuleEngine_InitTable(t *testing.T) {
|
||||
initTable := func(me wasm.ModuleEngine, tableInitLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{} {
|
||||
table := make([]interface{}, tableInitLen)
|
||||
internal := me.(*moduleEngine)
|
||||
for idx, fnidx := range initTableIdxToFnIdx {
|
||||
table[idx] = internal.compiledFunctions[fnidx]
|
||||
}
|
||||
return table
|
||||
}
|
||||
enginetest.RunTestEngine_NewModuleEngine_InitTable(t, initTable, NewEngine)
|
||||
enginetest.RunTestEngine_NewModuleEngine_InitTable(t, et)
|
||||
}
|
||||
|
||||
func TestModuleEngine_Call(t *testing.T) {
|
||||
enginetest.RunTestModuleEngine_Call(t, NewEngine)
|
||||
enginetest.RunTestModuleEngine_Call(t, et)
|
||||
}
|
||||
|
||||
func TestTestModuleEngine_Call_HostFn(t *testing.T) {
|
||||
enginetest.RunTestModuleEngine_Call_HostFn(t, NewEngine)
|
||||
enginetest.RunTestModuleEngine_Call_HostFn(t, et)
|
||||
}
|
||||
|
||||
func TestCallEngine_callNativeFunc_signExtend(t *testing.T) {
|
||||
@@ -190,7 +194,7 @@ func TestCallEngine_callNativeFunc_signExtend(t *testing.T) {
|
||||
|
||||
func TestEngineCompile_Errors(t *testing.T) {
|
||||
t.Run("invalid import", func(t *testing.T) {
|
||||
e := NewEngine().(*engine)
|
||||
e := et.NewEngine().(*engine)
|
||||
_, err := e.NewModuleEngine(t.Name(),
|
||||
[]*wasm.FunctionInstance{{Module: &wasm.ModuleInstance{Name: "uncompiled"}, Name: "fn"}},
|
||||
nil, // moduleFunctions
|
||||
@@ -201,7 +205,7 @@ func TestEngineCompile_Errors(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("release on compilation error", func(t *testing.T) {
|
||||
e := NewEngine().(*engine)
|
||||
e := et.NewEngine().(*engine)
|
||||
|
||||
importedFunctions := []*wasm.FunctionInstance{
|
||||
{Name: "1", Type: &wasm.FunctionType{}, Body: []byte{wasm.OpcodeEnd}, Module: &wasm.ModuleInstance{}},
|
||||
@@ -261,7 +265,7 @@ func TestClose(t *testing.T) {
|
||||
} {
|
||||
tc := tc
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
e := NewEngine().(*engine)
|
||||
e := et.NewEngine().(*engine)
|
||||
if len(tc.importedFunctions) > 0 {
|
||||
// initialize the module-engine containing imported functions
|
||||
me, err := e.NewModuleEngine(t.Name(), nil, tc.importedFunctions, nil, nil)
|
||||
@@ -281,7 +285,8 @@ func TestClose(t *testing.T) {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
}
|
||||
|
||||
me.Close()
|
||||
err = me.Close()
|
||||
require.NoError(t, err)
|
||||
|
||||
require.Len(t, e.compiledFunctions, len(tc.importedFunctions))
|
||||
for _, f := range tc.importedFunctions {
|
||||
|
||||
@@ -21,6 +21,8 @@ type (
|
||||
engine struct {
|
||||
compiledFunctions map[*wasm.FunctionInstance]*compiledFunction // guarded by mutex.
|
||||
mux sync.RWMutex
|
||||
// setFinalizer defaults to runtime.SetFinalizer, but overridable for tests.
|
||||
setFinalizer func(obj interface{}, finalizer interface{})
|
||||
}
|
||||
|
||||
// moduleEngine implements internalwasm.ModuleEngine
|
||||
@@ -38,10 +40,13 @@ type (
|
||||
parentEngine *engine
|
||||
importedFunctionCounts int
|
||||
|
||||
// Closed indicates whether Close was called for this moduleEngine, and used to prevent
|
||||
// double-free of compiledFunctions.
|
||||
// Note: intentionally use uint32 instead of bool for atomic operation.
|
||||
closed uint32
|
||||
// closed is the pointer used both to guard moduleEngine.CloseWithExitCode and to store the exit code.
|
||||
//
|
||||
// The update value is 1 + exitCode << 32. This ensures an exit code of zero isn't mistaken for never closed.
|
||||
//
|
||||
// Note: Exclusively reading and updating this with atomics guarantees cross-goroutine observations.
|
||||
// See /RATIONALE.md
|
||||
closed uint64
|
||||
}
|
||||
|
||||
// callEngine holds context per moduleEngine.Call, and shared across all the
|
||||
@@ -171,7 +176,7 @@ type (
|
||||
|
||||
// Pre-calculated pointer pointing to the initial byte of .codeSegment slice.
|
||||
// That mean codeInitialAddress always equals uintptr(unsafe.Pointer(&.codeSegment[0]))
|
||||
// and we cache the value (uintptr(unsafe.Pointer(&.codeSegment[0]))) to this field
|
||||
// and we cache the value (uintptr(unsafe.Pointer(&.codeSegment[0]))) to this field,
|
||||
// so we don't need to repeat the calculation on each function call.
|
||||
codeInitialAddress uintptr
|
||||
// The max of the stack pointer this function can reach. Lazily applied via maybeGrowValueStack.
|
||||
@@ -332,7 +337,24 @@ func (c *callFrame) String() string {
|
||||
)
|
||||
}
|
||||
|
||||
// NewModuleEngine implements internalwasm.Engine NewModuleEngine
|
||||
// releaseCompiledFunction is a runtime.SetFinalizer function that munmaps the compiledFunction.codeSegment.
|
||||
func releaseCompiledFunction(compiledFn *compiledFunction) {
|
||||
codeSegment := compiledFn.codeSegment
|
||||
if codeSegment == nil {
|
||||
return // already released
|
||||
}
|
||||
|
||||
// Setting this to nil allows tests to know the correct finalizer function was called.
|
||||
compiledFn.codeSegment = nil
|
||||
if err := munmapCodeSegment(codeSegment); err != nil {
|
||||
// munmap failure cannot recover, and happen asynchronously on the finalizer thread. While finalizer
|
||||
// functions can return errors, they are ignored. To make these visible for troubleshooting, we panic
|
||||
// with additional context. module+funcidx should be enough, but if not, we can add more later.
|
||||
panic(fmt.Errorf("jit: failed to munmap code segment for %s.function[%d]: %w", compiledFn.source.Module.Name, compiledFn.source.Index, err))
|
||||
}
|
||||
}
|
||||
|
||||
// NewModuleEngine implements the same method as documented on internalwasm.Engine.
|
||||
func (e *engine) NewModuleEngine(name string, importedFunctions, moduleFunctions []*wasm.FunctionInstance, table *wasm.TableInstance, tableInit map[wasm.Index]wasm.Index) (wasm.ModuleEngine, error) {
|
||||
imported := len(importedFunctions)
|
||||
me := &moduleEngine{
|
||||
@@ -359,9 +381,14 @@ func (e *engine) NewModuleEngine(name string, importedFunctions, moduleFunctions
|
||||
compiled, err = compileHostFunction(f)
|
||||
}
|
||||
if err != nil {
|
||||
me.Close()
|
||||
me.doClose() // safe because the reference to me was never leaked.
|
||||
return nil, fmt.Errorf("function[%d/%d] %w", i, len(moduleFunctions)-1, err)
|
||||
}
|
||||
|
||||
// As this uses mmap, we need a finalizer in case moduleEngine.Close was never called. Regardless, we need a
|
||||
// finalizer due to how moduleEngine.doClose is implemented.
|
||||
e.setFinalizer(compiled, releaseCompiledFunction)
|
||||
|
||||
me.compiledFunctions = append(me.compiledFunctions, compiled)
|
||||
|
||||
// Add the compiled function to the store-wide engine as well so that
|
||||
@@ -375,20 +402,16 @@ func (e *engine) NewModuleEngine(name string, importedFunctions, moduleFunctions
|
||||
return me, nil
|
||||
}
|
||||
|
||||
// Close implements io.Closer
|
||||
func (me *moduleEngine) Close() (err error) {
|
||||
me.doClose(runtime.SetFinalizer) // extracted for testability
|
||||
return
|
||||
}
|
||||
|
||||
// doClose is currently guarded with CAS to only be called once. There is a race-condition inside the
|
||||
// critical section: functions are removed from the parent engine, but there's no guard to prevent this
|
||||
// doClose is guarded by the caller with CAS, which means it happens only once. However, there is a race-condition
|
||||
// inside the critical section: functions are removed from the parent engine, but there's no guard to prevent this
|
||||
// moduleInstance from making new calls. This means at inside the critical section there could be in-flight calls,
|
||||
// and even after it new calls can be made, given a reference to this moduleEngine.
|
||||
//
|
||||
// To ensure neither in-flight, nor new calls segfault due to missing code segment, memory isn't unmapped here.
|
||||
// Rather, it is added to the finalizer queue to be done at some point (perhaps never). This needs to eventually be
|
||||
// as close leaks resources until cleaned up by the finalizer which by docs may never be run.
|
||||
// To ensure neither in-flight, nor new calls segfault due to missing code segment, memory isn't unmapped here. So, this
|
||||
// relies on the fact that NewModuleEngine already added a finalizer for each compiledFunction,
|
||||
//
|
||||
// Note that the finalizer is a queue of work to be done at some point (perhaps never). In worst case, the finalizer
|
||||
// doesn't run and functions in already closed modules retain memory until exhaustion.
|
||||
//
|
||||
// Potential future design (possibly faulty, so expect impl to be more complete or better):
|
||||
// * Change this to implement io.Closer and document this is blocking
|
||||
@@ -402,26 +425,11 @@ func (me *moduleEngine) Close() (err error) {
|
||||
// * Knowing which in-flight calls from other modules, that can use this module may be tricky
|
||||
// * Pure wasm functions can be left to complete.
|
||||
// * Host functions are the only unknowns (ex can do I/O) so they may need to be tracked.
|
||||
func (me *moduleEngine) doClose(setFinalizer func(obj interface{}, finalizer interface{})) {
|
||||
// Setting finalizer multiple times result in panic, so we guard with CAS.
|
||||
if atomic.CompareAndSwapUint32(&me.closed, 0, 1) {
|
||||
moduleName := me.name // pin to avoid leaking moduleEngine on any finalizer error.
|
||||
|
||||
// Release all the function instances declared in this module.
|
||||
for i, cf := range me.compiledFunctions[me.importedFunctionCounts:] {
|
||||
funcIdx := wasm.Index(i) // pin for any finalizer error
|
||||
|
||||
setFinalizer(cf, func(compiledFn *compiledFunction) {
|
||||
if err := munmapCodeSegment(compiledFn.codeSegment); err != nil {
|
||||
|
||||
// munmap failure cannot recover, and it happens asynchronously. Add context for troubleshooting.
|
||||
// module+funcidx should be enough, but if not, we can later propagate the func name.
|
||||
err = fmt.Errorf("jit: failed to munmap code segment for %s.function[%d]: %w", moduleName, funcIdx, err)
|
||||
panic(err)
|
||||
}
|
||||
})
|
||||
me.parentEngine.deleteCompiledFunction(cf.source)
|
||||
}
|
||||
func (me *moduleEngine) doClose() {
|
||||
// Release all the function instances declared in this module.
|
||||
for _, cf := range me.compiledFunctions[me.importedFunctionCounts:] {
|
||||
// NOTE: we still rely on the finalizer of cf until the notes on this function are addressed.
|
||||
me.parentEngine.deleteCompiledFunction(cf.source)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -444,7 +452,12 @@ func (e *engine) getCompiledFunction(f *wasm.FunctionInstance) (cf *compiledFunc
|
||||
return
|
||||
}
|
||||
|
||||
// Call implements internalwasm.ModuleEngine Call
|
||||
// Name implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Name() string {
|
||||
return me.name
|
||||
}
|
||||
|
||||
// Call implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Call(ctx *wasm.ModuleContext, f *wasm.FunctionInstance, params ...uint64) (results []uint64, err error) {
|
||||
compiled := me.compiledFunctions[f.Index]
|
||||
if compiled == nil {
|
||||
@@ -509,12 +522,24 @@ func (me *moduleEngine) Call(ctx *wasm.ModuleContext, f *wasm.FunctionInstance,
|
||||
return
|
||||
}
|
||||
|
||||
// Close implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (me *moduleEngine) Close() error {
|
||||
if !atomic.CompareAndSwapUint64(&me.closed, 0, 1) {
|
||||
return nil
|
||||
}
|
||||
me.doClose()
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewEngine() wasm.Engine {
|
||||
return newEngine()
|
||||
}
|
||||
|
||||
func newEngine() *engine {
|
||||
return &engine{compiledFunctions: map[*wasm.FunctionInstance]*compiledFunction{}}
|
||||
return &engine{
|
||||
compiledFunctions: map[*wasm.FunctionInstance]*compiledFunction{},
|
||||
setFinalizer: runtime.SetFinalizer,
|
||||
}
|
||||
}
|
||||
|
||||
// Do not make these variables as constants, otherwise there would be
|
||||
|
||||
@@ -6,7 +6,6 @@ import (
|
||||
"math"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"sync"
|
||||
"testing"
|
||||
"unsafe"
|
||||
|
||||
@@ -99,39 +98,42 @@ func TestVerifyOffsetValue(t *testing.T) {
|
||||
require.Equal(t, int(unsafe.Offsetof(eface.data)), interfaceDataOffset)
|
||||
}
|
||||
|
||||
// et is used for tests defined in the enginetest package.
|
||||
var et = &engineTester{}
|
||||
|
||||
type engineTester struct{}
|
||||
|
||||
func (e *engineTester) NewEngine() wasm.Engine {
|
||||
return newEngine()
|
||||
}
|
||||
|
||||
func (e *engineTester) InitTable(me wasm.ModuleEngine, initTableLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{} {
|
||||
table := make([]interface{}, initTableLen)
|
||||
internal := me.(*moduleEngine)
|
||||
for idx, fnidx := range initTableIdxToFnIdx {
|
||||
table[idx] = internal.compiledFunctions[fnidx]
|
||||
}
|
||||
return table
|
||||
}
|
||||
|
||||
func TestEngine_NewModuleEngine(t *testing.T) {
|
||||
requireSupportedOSArch(t)
|
||||
enginetest.RunTestEngine_NewModuleEngine(t, et)
|
||||
}
|
||||
|
||||
e := NewEngine()
|
||||
|
||||
t.Run("sets module name", func(t *testing.T) {
|
||||
me, err := e.NewModuleEngine(t.Name(), nil, nil, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer me.Close()
|
||||
require.Equal(t, t.Name(), me.(*moduleEngine).name)
|
||||
})
|
||||
func TestEngine_NewModuleEngine_InitTable(t *testing.T) {
|
||||
requireSupportedOSArch(t)
|
||||
enginetest.RunTestEngine_NewModuleEngine_InitTable(t, et)
|
||||
}
|
||||
|
||||
func TestModuleEngine_Call(t *testing.T) {
|
||||
requireSupportedOSArch(t)
|
||||
enginetest.RunTestModuleEngine_Call(t, NewEngine)
|
||||
}
|
||||
|
||||
func TestEngine_NewModuleEngine_InitTable(t *testing.T) {
|
||||
initTable := func(me wasm.ModuleEngine, tableInitLen uint32, initTableIdxToFnIdx map[wasm.Index]wasm.Index) []interface{} {
|
||||
table := make([]interface{}, tableInitLen)
|
||||
internal := me.(*moduleEngine)
|
||||
for idx, fnidx := range initTableIdxToFnIdx {
|
||||
table[idx] = internal.compiledFunctions[fnidx]
|
||||
}
|
||||
return table
|
||||
}
|
||||
enginetest.RunTestEngine_NewModuleEngine_InitTable(t, initTable, NewEngine)
|
||||
enginetest.RunTestModuleEngine_Call(t, et)
|
||||
}
|
||||
|
||||
func TestTestModuleEngine_Call_HostFn(t *testing.T) {
|
||||
requireSupportedOSArch(t)
|
||||
enginetest.RunTestModuleEngine_Call_HostFn(t, NewEngine)
|
||||
enginetest.RunTestModuleEngine_Call_HostFn(t, et)
|
||||
}
|
||||
|
||||
func requireSupportedOSArch(t *testing.T) {
|
||||
@@ -142,7 +144,7 @@ func requireSupportedOSArch(t *testing.T) {
|
||||
|
||||
func TestEngineCompile_Errors(t *testing.T) {
|
||||
t.Run("invalid import", func(t *testing.T) {
|
||||
e := newEngine()
|
||||
e := et.NewEngine()
|
||||
_, err := e.NewModuleEngine(
|
||||
t.Name(),
|
||||
[]*wasm.FunctionInstance{{Module: &wasm.ModuleInstance{Name: "uncompiled"}, Name: "fn"}},
|
||||
@@ -154,7 +156,7 @@ func TestEngineCompile_Errors(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("release on compilation error", func(t *testing.T) {
|
||||
e := newEngine()
|
||||
e := et.NewEngine().(*engine)
|
||||
|
||||
importedFunctions := []*wasm.FunctionInstance{
|
||||
{Name: "1", Type: &wasm.FunctionType{}, Body: []byte{wasm.OpcodeEnd}, Module: &wasm.ModuleInstance{}},
|
||||
@@ -186,28 +188,96 @@ func TestEngineCompile_Errors(t *testing.T) {
|
||||
})
|
||||
}
|
||||
|
||||
var fakeFinalizer = func(obj interface{}, finalizer interface{}) {
|
||||
type fakeFinalizer map[*compiledFunction]func(*compiledFunction)
|
||||
|
||||
func (f fakeFinalizer) setFinalizer(obj interface{}, finalizer interface{}) {
|
||||
cf := obj.(*compiledFunction)
|
||||
fn := finalizer.(func(compiledFn *compiledFunction))
|
||||
fn(cf)
|
||||
if _, ok := f[cf]; ok { // easier than adding a field for testing.T
|
||||
panic(fmt.Sprintf("BUG: %v already had its finalizer set", cf))
|
||||
}
|
||||
f[cf] = finalizer.(func(*compiledFunction))
|
||||
}
|
||||
|
||||
// TestModuleEngine_Close_Panic tests that an unexpected panic has some identifying information in it.
|
||||
func TestModuleEngine_Close_Panic(t *testing.T) {
|
||||
e := newEngine()
|
||||
me := &moduleEngine{
|
||||
name: t.Name(),
|
||||
compiledFunctions: []*compiledFunction{
|
||||
{codeSegment: []byte{wasm.OpcodeEnd} /* invalid because not compiled */},
|
||||
},
|
||||
parentEngine: e,
|
||||
func TestNewModuleEngine_CompiledFunctions(t *testing.T) {
|
||||
newFunctionInstance := func(id int) *wasm.FunctionInstance {
|
||||
return &wasm.FunctionInstance{
|
||||
Name: strconv.Itoa(id),
|
||||
Type: &wasm.FunctionType{},
|
||||
Body: []byte{wasm.OpcodeEnd},
|
||||
Module: &wasm.ModuleInstance{},
|
||||
}
|
||||
}
|
||||
|
||||
e := et.NewEngine().(*engine)
|
||||
|
||||
importedFinalizer := fakeFinalizer{}
|
||||
e.setFinalizer = importedFinalizer.setFinalizer
|
||||
|
||||
importedFunctions := []*wasm.FunctionInstance{
|
||||
newFunctionInstance(10),
|
||||
newFunctionInstance(20),
|
||||
}
|
||||
modE, err := e.NewModuleEngine(t.Name(), nil, importedFunctions, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer modE.Close()
|
||||
imported := modE.(*moduleEngine)
|
||||
|
||||
importingFinalizer := fakeFinalizer{}
|
||||
e.setFinalizer = importingFinalizer.setFinalizer
|
||||
|
||||
moduleFunctions := []*wasm.FunctionInstance{
|
||||
newFunctionInstance(100),
|
||||
newFunctionInstance(200),
|
||||
newFunctionInstance(300),
|
||||
}
|
||||
|
||||
modE, err = e.NewModuleEngine(t.Name(), importedFunctions, moduleFunctions, nil, nil)
|
||||
require.NoError(t, err)
|
||||
defer modE.Close()
|
||||
importing := modE.(*moduleEngine)
|
||||
|
||||
// Ensure the importing module didn't try to finalize the imported functions.
|
||||
require.Equal(t, len(importedFunctions), len(imported.compiledFunctions))
|
||||
for _, f := range importedFunctions {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
cf := e.compiledFunctions[f]
|
||||
require.Contains(t, importedFinalizer, cf)
|
||||
require.NotContains(t, importingFinalizer, cf)
|
||||
}
|
||||
|
||||
// The importing module's compiled functions include ones it compiled (module-defined) and imported ones).
|
||||
require.Equal(t, len(importedFunctions)+len(moduleFunctions), len(importing.compiledFunctions))
|
||||
|
||||
// Ensure the importing module only tried to finalize its own functions.
|
||||
for _, f := range moduleFunctions {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
cf := e.compiledFunctions[f]
|
||||
require.NotContains(t, importedFinalizer, cf)
|
||||
require.Contains(t, importingFinalizer, cf)
|
||||
}
|
||||
|
||||
// Pretend the finalizer executed, by invoking them one-by-one.
|
||||
for k, v := range importingFinalizer {
|
||||
v(k)
|
||||
}
|
||||
for k, v := range importedFinalizer {
|
||||
v(k)
|
||||
}
|
||||
for _, f := range e.compiledFunctions {
|
||||
require.Nil(t, f.codeSegment) // Set to nil if the correct finalizer was associated.
|
||||
}
|
||||
}
|
||||
|
||||
// TestReleaseCompiledFunction_Panic tests that an unexpected panic has some identifying information in it.
|
||||
func TestReleaseCompiledFunction_Panic(t *testing.T) {
|
||||
// capturePanic because there's no require.PanicsWithErrorPrefix
|
||||
errMessage := capturePanic(func() {
|
||||
me.doClose(fakeFinalizer)
|
||||
releaseCompiledFunction(&compiledFunction{
|
||||
codeSegment: []byte{wasm.OpcodeEnd}, // never compiled means it was never mapped.
|
||||
source: &wasm.FunctionInstance{Index: 2, Module: &wasm.ModuleInstance{Name: t.Name()}}, // for error string
|
||||
})
|
||||
})
|
||||
require.Contains(t, errMessage.Error(), "jit: failed to munmap code segment for TestModuleEngine_Close_Panic.function[0]:")
|
||||
require.Contains(t, errMessage.Error(), "jit: failed to munmap code segment for TestReleaseCompiledFunction_Panic.function[2]:")
|
||||
}
|
||||
|
||||
// capturePanic returns an error recovered from a panic
|
||||
@@ -223,7 +293,7 @@ func capturePanic(panics func()) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func TestModuleEngine_Close_Concurrent(t *testing.T) {
|
||||
func TestModuleEngine_Close(t *testing.T) {
|
||||
newFunctionInstance := func(id int) *wasm.FunctionInstance {
|
||||
return &wasm.FunctionInstance{
|
||||
Name: strconv.Itoa(id), Type: &wasm.FunctionType{}, Body: []byte{wasm.OpcodeEnd}, Module: &wasm.ModuleInstance{}}
|
||||
@@ -249,8 +319,8 @@ func TestModuleEngine_Close_Concurrent(t *testing.T) {
|
||||
} {
|
||||
tc := tc
|
||||
t.Run(tc.name, func(t *testing.T) {
|
||||
e := newEngine()
|
||||
var importedModuleEngine *moduleEngine
|
||||
e := et.NewEngine().(*engine)
|
||||
var imported *moduleEngine
|
||||
if len(tc.importedFunctions) > 0 {
|
||||
// Instantiate the imported module
|
||||
modEngine, err := e.NewModuleEngine(
|
||||
@@ -261,11 +331,11 @@ func TestModuleEngine_Close_Concurrent(t *testing.T) {
|
||||
nil, // tableInit
|
||||
)
|
||||
require.NoError(t, err)
|
||||
importedModuleEngine = modEngine.(*moduleEngine)
|
||||
require.Len(t, importedModuleEngine.compiledFunctions, len(tc.importedFunctions))
|
||||
imported = modEngine.(*moduleEngine)
|
||||
require.Len(t, imported.compiledFunctions, len(tc.importedFunctions))
|
||||
}
|
||||
|
||||
modEngine, err := e.NewModuleEngine(
|
||||
importing, err := e.NewModuleEngine(
|
||||
fmt.Sprintf("%s - module-defined functions", t.Name()),
|
||||
tc.importedFunctions,
|
||||
tc.moduleFunctions,
|
||||
@@ -273,58 +343,41 @@ func TestModuleEngine_Close_Concurrent(t *testing.T) {
|
||||
nil, // tableInit
|
||||
)
|
||||
require.NoError(t, err)
|
||||
require.Len(t, modEngine.(*moduleEngine).compiledFunctions, len(tc.importedFunctions)+len(tc.moduleFunctions))
|
||||
require.Len(t, importing.(*moduleEngine).compiledFunctions, len(tc.importedFunctions)+len(tc.moduleFunctions))
|
||||
|
||||
require.Len(t, e.compiledFunctions, len(tc.importedFunctions)+len(tc.moduleFunctions))
|
||||
|
||||
var importedMappedRegions [][]byte
|
||||
for _, f := range tc.importedFunctions {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
importedMappedRegions = append(importedMappedRegions, e.compiledFunctions[f].codeSegment)
|
||||
}
|
||||
var mappedRegions [][]byte
|
||||
for _, f := range tc.moduleFunctions {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
mappedRegions = append(mappedRegions, e.compiledFunctions[f].codeSegment)
|
||||
}
|
||||
|
||||
const goroutines = 100
|
||||
var wg sync.WaitGroup
|
||||
wg.Add(goroutines)
|
||||
for i := 0; i < goroutines; i++ {
|
||||
go func() {
|
||||
defer wg.Done()
|
||||
err = importing.Close()
|
||||
require.NoError(t, err)
|
||||
|
||||
// Ensure concurrent multiple execution of Close is guarded by atomic without overloading finalizer.
|
||||
modEngine.(*moduleEngine).doClose(fakeFinalizer)
|
||||
}()
|
||||
}
|
||||
wg.Wait()
|
||||
// Closing should flip the status bit, so that it cannot be closed again.
|
||||
require.Equal(t, uint64(1), importing.(*moduleEngine).closed)
|
||||
|
||||
require.True(t, modEngine.(*moduleEngine).closed == 1)
|
||||
// Closing the importing module shouldn't delete the imported functions from the engine.
|
||||
require.Len(t, e.compiledFunctions, len(tc.importedFunctions))
|
||||
for _, f := range tc.importedFunctions {
|
||||
require.Contains(t, e.compiledFunctions, f)
|
||||
}
|
||||
|
||||
// However, closing the importing module should delete its own functions from the engine.
|
||||
for i, f := range tc.moduleFunctions {
|
||||
require.NotContains(t, e.compiledFunctions, f, i)
|
||||
}
|
||||
|
||||
for _, mappedRegion := range mappedRegions {
|
||||
// munmap twice should result in error.
|
||||
err = munmapCodeSegment(mappedRegion)
|
||||
require.Error(t, err)
|
||||
}
|
||||
|
||||
if len(tc.importedFunctions) > 0 {
|
||||
importedModuleEngine.doClose(fakeFinalizer)
|
||||
|
||||
for _, mappedRegion := range importedMappedRegions {
|
||||
// munmap twice should result in error.
|
||||
err = munmapCodeSegment(mappedRegion)
|
||||
require.Error(t, err)
|
||||
}
|
||||
err = imported.Close()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// When all modules are closed, the engine should be empty.
|
||||
require.Empty(t, e.compiledFunctions)
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -351,7 +404,7 @@ func TestSliceAllocatedOnHeap(t *testing.T) {
|
||||
}
|
||||
growGoroutineStack()
|
||||
|
||||
// Trigger relocation of goroutine stack because at this point we have majority of
|
||||
// Trigger relocation of goroutine stack because at this point we have the majority of
|
||||
// goroutine stack unused after recursive call.
|
||||
runtime.GC()
|
||||
}})
|
||||
|
||||
@@ -3,6 +3,7 @@ package internalwasm
|
||||
import (
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"strconv"
|
||||
@@ -421,6 +422,7 @@ type mockEngine struct {
|
||||
}
|
||||
|
||||
type mockModuleEngine struct {
|
||||
name string
|
||||
ctx *ModuleContext
|
||||
callFailIndex int
|
||||
}
|
||||
@@ -429,7 +431,7 @@ func newStore() *Store {
|
||||
return NewStore(&mockEngine{shouldCompileFail: false, callFailIndex: -1}, Features20191205)
|
||||
}
|
||||
|
||||
// NewModuleEngine implements Engine.NewModuleEngine
|
||||
// NewModuleEngine implements the same method as documented on internalwasm.Engine.
|
||||
func (e *mockEngine) NewModuleEngine(_ string, _, _ []*FunctionInstance, _ *TableInstance, _ map[Index]Index) (ModuleEngine, error) {
|
||||
if e.shouldCompileFail {
|
||||
return nil, fmt.Errorf("some compilation error")
|
||||
@@ -437,16 +439,22 @@ func (e *mockEngine) NewModuleEngine(_ string, _, _ []*FunctionInstance, _ *Tabl
|
||||
return &mockModuleEngine{callFailIndex: e.callFailIndex}, nil
|
||||
}
|
||||
|
||||
// Call implements ModuleEngine.Call
|
||||
// Name implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (e *mockModuleEngine) Name() string {
|
||||
return e.name
|
||||
}
|
||||
|
||||
// Call implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (e *mockModuleEngine) Call(ctx *ModuleContext, f *FunctionInstance, _ ...uint64) (results []uint64, err error) {
|
||||
if e.callFailIndex >= 0 && f.Index == Index(e.callFailIndex) {
|
||||
return nil, fmt.Errorf("call failed")
|
||||
err = errors.New("call failed")
|
||||
return
|
||||
}
|
||||
e.ctx = ctx
|
||||
return
|
||||
}
|
||||
|
||||
// Close implements io.Closer
|
||||
// Close implements the same method as documented on internalwasm.ModuleEngine.
|
||||
func (e *mockModuleEngine) Close() (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user