From 8c3fef3d0a6fceca86eae24a7ca743100f17e010 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Fri, 7 Apr 2023 14:30:12 -0400 Subject: [PATCH 01/16] [wasm-mt] Full JSInterop on threadpool workers [wasm][threads] flip YieldFromDispatchLoop; specialize PortableThreadPool.WorkerThread [mono] Implement a LifoJSSemaphore This is a LIFO semaphore with an asynchronous wait that triggers callbacks on the JS event loop in case of Release or timeout. Make managed LowLevelJSSemaphore copy-paste PortableThreadPool.WorkerThread for threaded WASM no changes yet. just copying verbatim to a separate file fixup native code for lifo semaphore fixup managed code for LowLevelJSSemaphore Implement PortableThreadPool loop using semaphore callbacks manage emscripten event loop from PortableThreadPool.WorkerThread make sure to keep the thread alive after setting up the semaphore wait. Cleanup the thread when exiting FIXME: thread equality assertion in timeout callback XXX REVERT ME - minimal async timeout test BUGFIX: &wait_entry ===> wait_entry nit: log thread id as hex in .ts Match the C logging for easier grepping XXX minimal sample - fetch on a background thread works fix non-wasm non-threads builds Add WebWorkerEventLoop internal class to managed event loop keepalive Don't explicitly call UnwindToJs as it doesn't know about managed code. Also avoid mono_thread_exit as that also won't necessarily clean up after the interpreter Start threadpool threads with keepalive checks Add a flag to mono's thread start wrappers to keep track of threads that may not want cleanup to run after the Start function returns. Use the flag when starting threadpool threads. HACK: kind of work around the emscripten_runtime_keepalive_push/pop no-op Keep a thread local counter in mono. that we "know" will be right for us and manually call unwind_to_js and thread_exit. This is super-fragile since we don't know what emscripten internals might be trying to manipulate the keepalive count and also we are exiting the thread with active managed frames, so we might be skipping finally clauses and possibly leaking interpreter memory. This is mainly meant to keep work going on this branch and not something we necessarily want to commit support JS Semaphore with --print-icall-table cross compiler fixes smoketest on Release builds make minimal FetchBackground sample more like a unit test Share PortableThreadPool.WorkerThread common code Share the code between non-browser implementations and browser+threads. The differences are just in how the work loop is started and implemented make both kinds of lifo semaphore share a base struct Unify LowLevelLifoSemaphore for normal and async waiting WebWorkerEventLoop: remove dead code, update comments remove unused arg from async wait semaphore don't need both user data and a gchandle rename native semaphore to LifoSemaphoreAsyncWait prefix functions with mono_lifo_semaphore_asyncwait_ Rename managed file to LowLevelLifoSemaphore.AsyncWait.Browser.Threads.Mono.cs Remove unnecessary indirections and allocations from managed AsyncWait semaphore fix non-browser+threads builds Keep track of unsettled JS interop promises in threadpool workers Set WorkerThread.IsIOPending when the current thread has unsettled JS interop promises. When IsIOPending is true, the worker will not exit even if it has no more work to do. Instead it will repeatedly wait for more work to arrive or for all promises to settle. change minimal sample's fetch helper to artificially delay the delay is longer that the threadpool worker's semaphore timeout, in order to validate that the worker stays alive while there are unsettled promises --- .../System.Private.CoreLib.Shared.projitems | 4 +- .../System/Threading/ThreadPool.Portable.cs | 2 + .../System.Private.CoreLib.csproj | 3 + ...PortableThreadPool.Browser.Threads.Mono.cs | 19 +++ ...dPool.WorkerThread.Browser.Threads.Mono.cs | 122 ++++++++++++++++++ .../ThreadPool.Browser.Threads.Mono.cs | 13 ++ src/mono/mono/metadata/threads-types.h | 3 + src/mono/mono/metadata/threads.c | 52 +++++++- .../wasm/runtime/pthreads/worker/index.ts | 4 +- src/mono/wasm/runtime/startup.ts | 2 +- 10 files changed, 218 insertions(+), 6 deletions(-) create mode 100644 src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs create mode 100644 src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs create mode 100644 src/mono/System.Private.CoreLib/src/System/Threading/ThreadPool.Browser.Threads.Mono.cs diff --git a/src/libraries/System.Private.CoreLib/src/System.Private.CoreLib.Shared.projitems b/src/libraries/System.Private.CoreLib/src/System.Private.CoreLib.Shared.projitems index dd7e20bf1f2d5..7687e2b80cb9e 100644 --- a/src/libraries/System.Private.CoreLib/src/System.Private.CoreLib.Shared.projitems +++ b/src/libraries/System.Private.CoreLib/src/System.Private.CoreLib.Shared.projitems @@ -2525,9 +2525,9 @@ - + - + diff --git a/src/libraries/System.Private.CoreLib/src/System/Threading/ThreadPool.Portable.cs b/src/libraries/System.Private.CoreLib/src/System/Threading/ThreadPool.Portable.cs index 39e1d6453263e..a9c4e038129a4 100644 --- a/src/libraries/System.Private.CoreLib/src/System/Threading/ThreadPool.Portable.cs +++ b/src/libraries/System.Private.CoreLib/src/System/Threading/ThreadPool.Portable.cs @@ -19,7 +19,9 @@ public static partial class ThreadPool { // Indicates whether the thread pool should yield the thread from the dispatch loop to the runtime periodically so that // the runtime may use the thread for processing other work +#if !(TARGET_BROWSER && FEATURE_WASM_THREADS) internal static bool YieldFromDispatchLoop => false; +#endif #if NATIVEAOT private const bool IsWorkerTrackingEnabledInConfig = false; diff --git a/src/mono/System.Private.CoreLib/System.Private.CoreLib.csproj b/src/mono/System.Private.CoreLib/System.Private.CoreLib.csproj index 1cc7d38b44fb1..bd5ea789f963f 100644 --- a/src/mono/System.Private.CoreLib/System.Private.CoreLib.csproj +++ b/src/mono/System.Private.CoreLib/System.Private.CoreLib.csproj @@ -281,6 +281,9 @@ + + + diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs new file mode 100644 index 0000000000000..d459c992f810f --- /dev/null +++ b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs @@ -0,0 +1,19 @@ +// Licensed to the .NET Foundation under one or more agreements. +// The .NET Foundation licenses this file to you under the MIT license. + +namespace System.Threading; + +internal sealed partial class PortableThreadPool +{ + private static partial class WorkerThread + { + private static bool IsIOPending => WebWorkerEventLoop.HasUnsettledInteropPromises; + } + + private struct CpuUtilizationReader + { +#pragma warning disable CA1822 + public double CurrentUtilization => 0.0; // FIXME: can we do better +#pragma warning restore CA1822 + } +} diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs new file mode 100644 index 0000000000000..52f7b03e699c6 --- /dev/null +++ b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs @@ -0,0 +1,122 @@ +// Licensed to the .NET Foundation under one or more agreements. +// The .NET Foundation licenses this file to you under the MIT license. + +using System.Diagnostics; +using System.Diagnostics.CodeAnalysis; +using System.Diagnostics.Tracing; +using System.Runtime.CompilerServices; + +namespace System.Threading +{ + internal sealed partial class PortableThreadPool + { + /// + /// The worker thread infastructure for the CLR thread pool. + /// + private static partial class WorkerThread + { + /// + /// Semaphore for controlling how many threads are currently working. + /// + private static readonly LowLevelLifoSemaphore s_semaphore = + LowLevelLifoSemaphore.CreateAsyncWaitSemaphore( + 0, + MaxPossibleThreadCount, + AppContextConfigHelper.GetInt32Config( + "System.Threading.ThreadPool.UnfairSemaphoreSpinLimit", + SemaphoreSpinCountDefault, + false), + onWait: () => + { + if (NativeRuntimeEventSource.Log.IsEnabled()) + { + NativeRuntimeEventSource.Log.ThreadPoolWorkerThreadWait( + (uint)ThreadPoolInstance._separated.counts.VolatileRead().NumExistingThreads); + } + }); + + private static readonly ThreadStart s_workerThreadStart = WorkerThreadStart; + + private sealed record SemaphoreWaitState(PortableThreadPool ThreadPoolInstance, LowLevelLock ThreadAdjustmentLock, WebWorkerEventLoop.KeepaliveToken KeepaliveToken) + { + public bool SpinWait = true; + + public void ResetIteration() { + SpinWait = true; + } + } + + private static void WorkerThreadStart() + { + Thread.CurrentThread.SetThreadPoolWorkerThreadName(); + + PortableThreadPool threadPoolInstance = ThreadPoolInstance; + + if (NativeRuntimeEventSource.Log.IsEnabled()) + { + NativeRuntimeEventSource.Log.ThreadPoolWorkerThreadStart( + (uint)threadPoolInstance._separated.counts.VolatileRead().NumExistingThreads); + } + + LowLevelLock threadAdjustmentLock = threadPoolInstance._threadAdjustmentLock; + var keepaliveToken = WebWorkerEventLoop.KeepalivePush(); + SemaphoreWaitState state = new(threadPoolInstance, threadAdjustmentLock, keepaliveToken) { SpinWait = true }; + // set up the callbacks for semaphore waits, tell + // emscripten to keep the thread alive, and return to + // the JS event loop. + WaitForWorkLoop(s_semaphore, state); + // return from thread start with keepalive - the thread will stay alive in the JS event loop + } + + private static readonly Action s_WorkLoopSemaphoreSuccess = new(WorkLoopSemaphoreSuccess); + private static readonly Action s_WorkLoopSemaphoreTimedOut = new(WorkLoopSemaphoreTimedOut); + + private static void WaitForWorkLoop(LowLevelLifoSemaphore semaphore, SemaphoreWaitState state) + { + semaphore.PrepareAsyncWait(ThreadPoolThreadTimeoutMs, s_WorkLoopSemaphoreSuccess, s_WorkLoopSemaphoreTimedOut, state); + // thread should still be kept alive + Debug.Assert(state.KeepaliveToken.Valid); + } + + private static void WorkLoopSemaphoreSuccess(LowLevelLifoSemaphore semaphore, object? stateObject) + { + SemaphoreWaitState state = (SemaphoreWaitState)stateObject!; + WorkerDoWork(state.ThreadPoolInstance, ref state.SpinWait); + // Go around the loop one more time, keeping existing mutated state + WaitForWorkLoop(semaphore, state); + } + + private static void WorkLoopSemaphoreTimedOut(LowLevelLifoSemaphore semaphore, object? stateObject) + { + SemaphoreWaitState state = (SemaphoreWaitState)stateObject!; + if (WorkerTimedOutMaybeStop(state.ThreadPoolInstance, state.ThreadAdjustmentLock)) { + // we're done, kill the thread. + + // we're wrapped in an emscripten eventloop handler which will consult the + // keepalive count, destroy the thread and run the TLS dtor which will + // unregister the thread from Mono + state.KeepaliveToken.Pop(); + return; + } else { + // more work showed up while we were shutting down, go around one more time + state.ResetIteration(); + WaitForWorkLoop(semaphore, state); + } + } + + private static void CreateWorkerThread() + { + // Thread pool threads must start in the default execution context without transferring the context, so + // using captureContext: false. + Thread workerThread = new Thread(s_workerThreadStart); + workerThread.IsThreadPoolThread = true; + workerThread.IsBackground = true; + // thread name will be set in thread proc + + // This thread will return to the JS event loop - tell the runtime not to cleanup + // after the start function returns, if the Emscripten keepalive is non-zero. + WebWorkerEventLoop.StartExitable(workerThread, captureContext: false); + } + } + } +} diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/ThreadPool.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/ThreadPool.Browser.Threads.Mono.cs new file mode 100644 index 0000000000000..7933e49db422b --- /dev/null +++ b/src/mono/System.Private.CoreLib/src/System/Threading/ThreadPool.Browser.Threads.Mono.cs @@ -0,0 +1,13 @@ +// Licensed to the .NET Foundation under one or more agreements. +// The .NET Foundation licenses this file to you under the MIT license. + +namespace System.Threading +{ + public static partial class ThreadPool + { + // Indicates that the threadpool should yield the thread from the dispatch loop to the + // runtime periodically. We use this to return back to the JS event loop so that the JS + // event queue can be drained + internal static bool YieldFromDispatchLoop => true; + } +} diff --git a/src/mono/mono/metadata/threads-types.h b/src/mono/mono/metadata/threads-types.h index fe57d74a02e39..b9652aa33eb32 100644 --- a/src/mono/mono/metadata/threads-types.h +++ b/src/mono/mono/metadata/threads-types.h @@ -78,6 +78,9 @@ typedef enum { MONO_THREAD_CREATE_FLAGS_DEBUGGER = 0x02, MONO_THREAD_CREATE_FLAGS_FORCE_CREATE = 0x04, MONO_THREAD_CREATE_FLAGS_SMALL_STACK = 0x08, +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP = 0x10, +#endif } MonoThreadCreateFlags; MONO_COMPONENT_API MonoInternalThread* diff --git a/src/mono/mono/metadata/threads.c b/src/mono/mono/metadata/threads.c index 08dc87fd8a62c..dad7323149f99 100644 --- a/src/mono/mono/metadata/threads.c +++ b/src/mono/mono/metadata/threads.c @@ -1088,6 +1088,9 @@ typedef struct { MonoThreadStart start_func; gpointer start_func_arg; gboolean force_attach; +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + gboolean returns_to_js_event_loop; +#endif gboolean failed; MonoCoopSem registered; } StartInfo; @@ -1173,6 +1176,10 @@ start_wrapper_internal (StartInfo *start_info, gsize *stack_ptr) /* Let the thread that called Start() know we're ready */ mono_coop_sem_post (&start_info->registered); +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + gboolean returns_to_js_event_loop = start_info->returns_to_js_event_loop; +#endif + if (mono_atomic_dec_i32 (&start_info->ref) == 0) { mono_coop_sem_destroy (&start_info->registered); g_free (start_info); @@ -1240,6 +1247,20 @@ start_wrapper_internal (StartInfo *start_info, gsize *stack_ptr) THREAD_DEBUG (g_message ("%s: (%" G_GSIZE_FORMAT ") Start wrapper terminating", __func__, mono_native_thread_id_get ())); +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + if (returns_to_js_event_loop) { + /* if the thread wants to stay alive, don't clean up after it */ +#ifdef MONO_EMSCRIPTEN_KEEPALIVE_WORKAROUND_HACK + /* we "know" that threadpool threads set their keepalive count correctly and will return here */ + g_assert (mono_emscripten_keepalive_hack_count > 0); + return 0; +#else + if (emscripten_runtime_keepalive_check()) + return 0; +#endif + } +#endif + /* Do any cleanup needed for apartment state. This * cannot be done in mono_thread_detach_internal since * mono_thread_detach_internal could be called for a @@ -1266,9 +1287,27 @@ start_wrapper (gpointer data) info = mono_thread_info_attach (); info->runtime_thread = TRUE; +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + gboolean returns_to_js_event_loop = start_info->returns_to_js_event_loop; +#endif /* Run the actual main function of the thread */ res = start_wrapper_internal (start_info, (gsize*)info->stack_end); +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + if (returns_to_js_event_loop) { + /* if the thread wants to stay alive, don't clean up after it */ +#ifdef MONO_EMSCRIPTEN_KEEPALIVE_WORKAROUND_HACK + /* we "know" the keepalive count is positive at this point for threadpool threads. Keep it alive */ + g_assert (mono_emscripten_keepalive_hack_count > 0); + emscripten_unwind_to_js_event_loop (); + g_assert_not_reached(); +#else + if (emscripten_runtime_keepalive_check()) + return 0; +#endif + } +#endif + mono_thread_info_exit (res); g_assert_not_reached (); @@ -1355,6 +1394,9 @@ create_thread (MonoThread *thread, MonoInternalThread *internal, MonoThreadStart start_info->start_func_arg = start_func_arg; start_info->force_attach = flags & MONO_THREAD_CREATE_FLAGS_FORCE_CREATE; start_info->failed = FALSE; +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + start_info->returns_to_js_event_loop = (flags & MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP) != 0; +#endif mono_coop_sem_init (&start_info->registered, 0); if (flags != MONO_THREAD_CREATE_FLAGS_SMALL_STACK) @@ -4913,7 +4955,15 @@ ves_icall_System_Threading_Thread_StartInternal (MonoThreadObjectHandle thread_h return; } - res = create_thread (internal, internal, NULL, NULL, stack_size, MONO_THREAD_CREATE_FLAGS_NONE, error); + MonoThreadCreateFlags create_flags = MONO_THREAD_CREATE_FLAGS_NONE; +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) + // HACK: threadpool threads can return to the JS event loop + // WISH: support this for other threads, too + if (internal->threadpool_thread) + create_flags |= MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP; +#endif + + res = create_thread (internal, internal, NULL, NULL, stack_size, create_flags, error); if (!res) { UNLOCK_THREAD (internal); return; diff --git a/src/mono/wasm/runtime/pthreads/worker/index.ts b/src/mono/wasm/runtime/pthreads/worker/index.ts index da4c780804c55..71a6f3bcfbe7e 100644 --- a/src/mono/wasm/runtime/pthreads/worker/index.ts +++ b/src/mono/wasm/runtime/pthreads/worker/index.ts @@ -114,7 +114,7 @@ function onMonoConfigReceived(config: MonoConfigInternal): void { export function mono_wasm_pthread_on_pthread_attached(pthread_id: pthread_ptr): void { const self = pthread_self; mono_assert(self !== null && self.pthread_id == pthread_id, "expected pthread_self to be set already when attaching"); - console.debug("MONO_WASM: attaching pthread to runtime", pthread_id); + console.debug("MONO_WASM: attaching pthread to runtime 0x" + pthread_id.toString(16)); preRunWorker(); currentWorkerThreadEvents.dispatchEvent(makeWorkerThreadEvent(dotnetPthreadAttached, self)); } @@ -127,7 +127,7 @@ export function afterThreadInitTLS(): void { if (ENVIRONMENT_IS_PTHREAD) { const pthread_ptr = (Module)["_pthread_self"](); mono_assert(!is_nullish(pthread_ptr), "pthread_self() returned null"); - console.debug("MONO_WASM: after thread init, pthread ptr", pthread_ptr); + console.debug("MONO_WASM: after thread init, pthread ptr 0x" + pthread_ptr.toString(16)); const self = setupChannelToMainThread(pthread_ptr); currentWorkerThreadEvents.dispatchEvent(makeWorkerThreadEvent(dotnetPthreadCreated, self)); } diff --git a/src/mono/wasm/runtime/startup.ts b/src/mono/wasm/runtime/startup.ts index 92c644c64a182..4cdf6dbc080b4 100644 --- a/src/mono/wasm/runtime/startup.ts +++ b/src/mono/wasm/runtime/startup.ts @@ -761,7 +761,7 @@ export async function mono_wasm_pthread_worker_init(module: DotnetModule, export pthreads_worker.setupPreloadChannelToMainThread(); // This is a good place for subsystems to attach listeners for pthreads_worker.currentWorkerThreadEvents pthreads_worker.currentWorkerThreadEvents.addEventListener(pthreads_worker.dotnetPthreadCreated, (ev) => { - console.debug("MONO_WASM: pthread created", ev.pthread_self.pthread_id); + console.debug("MONO_WASM: pthread created 0x" + ev.pthread_self.pthread_id.toString(16)); }); // this is the only event which is called on worker From a3db8861b3734b06a72c97b8035c8b49f011fd98 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Fri, 7 Apr 2023 15:02:13 -0400 Subject: [PATCH 02/16] [wasm-mt] Add background interop to smoketest --- .../wasm/browser-threads-minimal/Program.cs | 76 ++++++++++++++++++- ...Wasm.Browser.Threads.Minimal.Sample.csproj | 2 + .../wasm/browser-threads-minimal/blurst.txt | 1 + .../browser-threads-minimal/fetchhelper.js | 11 +++ .../wasm/browser-threads-minimal/main.js | 35 +++++++-- 5 files changed, 115 insertions(+), 10 deletions(-) create mode 100644 src/mono/sample/wasm/browser-threads-minimal/blurst.txt create mode 100644 src/mono/sample/wasm/browser-threads-minimal/fetchhelper.js diff --git a/src/mono/sample/wasm/browser-threads-minimal/Program.cs b/src/mono/sample/wasm/browser-threads-minimal/Program.cs index 0b9784836bbd7..b9ef3854ef163 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/Program.cs +++ b/src/mono/sample/wasm/browser-threads-minimal/Program.cs @@ -18,6 +18,61 @@ public static int Main(string[] args) return 0; } + [JSImport("globalThis.setTimeout")] + static partial void GlobalThisSetTimeout([JSMarshalAs] Action cb, int timeoutMs); + + [JSImport("globalThis.fetch")] + private static partial Task GlobalThisFetch(string url); + + [JSExport] + public static async Task Hello() + { + var t = Task.Run(TimeOutThenComplete); + await t; + Console.WriteLine ($"XYZ: Main Thread caught task tid:{Thread.CurrentThread.ManagedThreadId}"); + } + + const string fetchhelper = "./fetchelper.js"; + + [JSImport("responseText", fetchhelper)] + private static partial Task FetchHelperResponseText(JSObject response); + + [JSExport] + public static async Task FetchBackground(string url) + { + var t = Task.Run(async () => + { + await JSHost.ImportAsync(fetchhelper, "./fetchhelper.js"); + var r = await GlobalThisFetch(url); + var ok = (bool)r.GetPropertyAsBoolean("ok"); + + Console.WriteLine($"XYZ: FetchBackground fetch returned to thread:{Thread.CurrentThread.ManagedThreadId}, ok: {ok}"); + if (ok) + { + var text = await FetchHelperResponseText(r); + Console.WriteLine($"XYZ: FetchBackground fetch returned to thread:{Thread.CurrentThread.ManagedThreadId}, text: {text}"); + return text; + } + return "not-ok"; + }); + var r = await t; + Console.WriteLine($"XYZ: FetchBackground thread:{Thread.CurrentThread.ManagedThreadId} background thread returned"); + return r; + } + + private static async Task TimeOutThenComplete() + { + var tcs = new TaskCompletionSource(); + Console.WriteLine ($"XYZ: Task running tid:{Thread.CurrentThread.ManagedThreadId}"); + GlobalThisSetTimeout(() => { + tcs.SetResult(); + Console.WriteLine ($"XYZ: Timeout fired tid:{Thread.CurrentThread.ManagedThreadId}"); + }, 250); + Console.WriteLine ($"XYZ: Task sleeping tid:{Thread.CurrentThread.ManagedThreadId}"); + await tcs.Task; + Console.WriteLine ($"XYZ: Task resumed tid:{Thread.CurrentThread.ManagedThreadId}"); + } + [JSExport] public static async Task RunBackgroundThreadCompute() { @@ -41,10 +96,27 @@ public static async Task RunBackgroundLongRunningTaskCompute() return await t; } + [JSExport] + public static async Task RunBackgroundTaskRunCompute() + { + var t1 = Task.Run (() => { + var n = CountingCollatzTest(); + return n; + }); + var t2 = Task.Run (() => { + var n = CountingCollatzTest(); + return n; + }); + var rs = await Task.WhenAll (new [] { t1, t2 }); + if (rs[0] != rs[1]) + throw new Exception ($"Results from two tasks {rs[0]}, {rs[1]}, differ"); + return rs[0]; + } + public static int CountingCollatzTest() { const int limit = 5000; - const int maxInput = 500_000; + const int maxInput = 200_000; int bigly = 0; int hugely = 0; int maxSteps = 0; @@ -60,7 +132,7 @@ public static int CountingCollatzTest() Console.WriteLine ($"Bigly: {bigly}, Hugely: {hugely}, maxSteps: {maxSteps}"); - if (bigly == 241677 && hugely == 0 && maxSteps == 448) + if (bigly == 86187 && hugely == 0 && maxSteps == 382) return 524; else return 0; diff --git a/src/mono/sample/wasm/browser-threads-minimal/Wasm.Browser.Threads.Minimal.Sample.csproj b/src/mono/sample/wasm/browser-threads-minimal/Wasm.Browser.Threads.Minimal.Sample.csproj index f9c81f4b40e71..defce7521ac7f 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/Wasm.Browser.Threads.Minimal.Sample.csproj +++ b/src/mono/sample/wasm/browser-threads-minimal/Wasm.Browser.Threads.Minimal.Sample.csproj @@ -6,6 +6,8 @@ + + diff --git a/src/mono/sample/wasm/browser-threads-minimal/blurst.txt b/src/mono/sample/wasm/browser-threads-minimal/blurst.txt new file mode 100644 index 0000000000000..6679d914da1c7 --- /dev/null +++ b/src/mono/sample/wasm/browser-threads-minimal/blurst.txt @@ -0,0 +1 @@ +It was the best of times, it was the blurst of times. diff --git a/src/mono/sample/wasm/browser-threads-minimal/fetchhelper.js b/src/mono/sample/wasm/browser-threads-minimal/fetchhelper.js new file mode 100644 index 0000000000000..928492378fc6c --- /dev/null +++ b/src/mono/sample/wasm/browser-threads-minimal/fetchhelper.js @@ -0,0 +1,11 @@ + +function delay(timeoutMs) { + return new Promise(resolve => setTimeout(resolve, timeoutMs)); +} + +export async function responseText(response) /* Promise */ { + console.log("artificially waiting for response for 25 seconds"); + await delay(25000); + console.log("artificial waiting done"); + return await response.text(); +} diff --git a/src/mono/sample/wasm/browser-threads-minimal/main.js b/src/mono/sample/wasm/browser-threads-minimal/main.js index f607d96c2846a..4c6a28e916052 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/main.js +++ b/src/mono/sample/wasm/browser-threads-minimal/main.js @@ -15,18 +15,37 @@ try { const exports = await getAssemblyExports(assemblyName); - const r1 = await exports.Sample.Test.RunBackgroundThreadCompute(); - if (r1 !== 524) { - const msg = `Unexpected result ${r1} from RunBackgroundThreadCompute()`; + //console.log ("XYZ: running hello"); + //await exports.Sample.Test.Hello(); + //console.log ("XYZ: hello done"); + + console.log ("XYZ: running FetchBackground"); + let s = await exports.Sample.Test.FetchBackground("./blurst.txt"); + console.log ("XYZ: FetchBackground done"); + if (s !== "It was the best of times, it was the blurst of times.\n") { + const msg = `Unexpected FetchBackground result ${s}`; document.getElementById("out").innerHTML = msg; - throw new Error(msg); + throw new Error (msg); } - const r2 = await exports.Sample.Test.RunBackgroundLongRunningTaskCompute(); - if (r2 !== 524) { - const msg = `Unexpected result ${r2} from RunBackgorundLongRunningTaskCompute()`; + + console.log ("XYZ: running FetchBackground(missing)"); + s = await exports.Sample.Test.FetchBackground("./missing.txt"); + console.log ("XYZ: FetchBackground(missing) done"); + if (s !== "not-ok") { + const msg = `Unexpected FetchBackground(missing) result ${s}`; document.getElementById("out").innerHTML = msg; - throw new Error(msg); + throw new Error (msg); } + + //console.log ("HHH: running TaskRunCompute"); + //const r1 = await exports.Sample.Test.RunBackgroundTaskRunCompute(); + //if (r1 !== 524) { + // const msg = `Unexpected result ${r1} from RunBackgorundTaskRunCompute()`; + // document.getElementById("out").innerHTML = msg; + // throw new Error(msg); + //} + //console.log ("HHH: TaskRunCompute done"); + let exit_code = await runMain(assemblyName, []); exit(exit_code); From 847ecd55bcd6574e5d190ec1e528176d571aca77 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Fri, 7 Apr 2023 23:17:30 -0400 Subject: [PATCH 03/16] update to use the LowLevelLifoAsyncWaitSemaphore --- ...ThreadPool.WorkerThread.Browser.Threads.Mono.cs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs index 52f7b03e699c6..cf561eb875f65 100644 --- a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs +++ b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs @@ -18,8 +18,8 @@ private static partial class WorkerThread /// /// Semaphore for controlling how many threads are currently working. /// - private static readonly LowLevelLifoSemaphore s_semaphore = - LowLevelLifoSemaphore.CreateAsyncWaitSemaphore( + private static readonly LowLevelLifoAsyncWaitSemaphore s_semaphore = + new LowLevelLifoAsyncWaitSemaphore( 0, MaxPossibleThreadCount, AppContextConfigHelper.GetInt32Config( @@ -68,17 +68,17 @@ private static void WorkerThreadStart() // return from thread start with keepalive - the thread will stay alive in the JS event loop } - private static readonly Action s_WorkLoopSemaphoreSuccess = new(WorkLoopSemaphoreSuccess); - private static readonly Action s_WorkLoopSemaphoreTimedOut = new(WorkLoopSemaphoreTimedOut); + private static readonly Action s_WorkLoopSemaphoreSuccess = new(WorkLoopSemaphoreSuccess); + private static readonly Action s_WorkLoopSemaphoreTimedOut = new(WorkLoopSemaphoreTimedOut); - private static void WaitForWorkLoop(LowLevelLifoSemaphore semaphore, SemaphoreWaitState state) + private static void WaitForWorkLoop(LowLevelLifoAsyncWaitSemaphore semaphore, SemaphoreWaitState state) { semaphore.PrepareAsyncWait(ThreadPoolThreadTimeoutMs, s_WorkLoopSemaphoreSuccess, s_WorkLoopSemaphoreTimedOut, state); // thread should still be kept alive Debug.Assert(state.KeepaliveToken.Valid); } - private static void WorkLoopSemaphoreSuccess(LowLevelLifoSemaphore semaphore, object? stateObject) + private static void WorkLoopSemaphoreSuccess(LowLevelLifoAsyncWaitSemaphore semaphore, object? stateObject) { SemaphoreWaitState state = (SemaphoreWaitState)stateObject!; WorkerDoWork(state.ThreadPoolInstance, ref state.SpinWait); @@ -86,7 +86,7 @@ private static void WorkLoopSemaphoreSuccess(LowLevelLifoSemaphore semaphore, ob WaitForWorkLoop(semaphore, state); } - private static void WorkLoopSemaphoreTimedOut(LowLevelLifoSemaphore semaphore, object? stateObject) + private static void WorkLoopSemaphoreTimedOut(LowLevelLifoAsyncWaitSemaphore semaphore, object? stateObject) { SemaphoreWaitState state = (SemaphoreWaitState)stateObject!; if (WorkerTimedOutMaybeStop(state.ThreadPoolInstance, state.ThreadAdjustmentLock)) { From dd69c8faa4951bac97eb9ea86958b7168cc3829a Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Tue, 18 Apr 2023 12:01:06 -0400 Subject: [PATCH 04/16] adjust to renamed PortableThreadPool helper methods --- .../PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs index cf561eb875f65..b45dee7fa2fd6 100644 --- a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs +++ b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.WorkerThread.Browser.Threads.Mono.cs @@ -89,7 +89,7 @@ private static void WorkLoopSemaphoreSuccess(LowLevelLifoAsyncWaitSemaphore sema private static void WorkLoopSemaphoreTimedOut(LowLevelLifoAsyncWaitSemaphore semaphore, object? stateObject) { SemaphoreWaitState state = (SemaphoreWaitState)stateObject!; - if (WorkerTimedOutMaybeStop(state.ThreadPoolInstance, state.ThreadAdjustmentLock)) { + if (ShouldExitWorker(state.ThreadPoolInstance, state.ThreadAdjustmentLock)) { // we're done, kill the thread. // we're wrapped in an emscripten eventloop handler which will consult the From 451d21b2a457bbab2372588d5fa8d948d48c2891 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Wed, 19 Apr 2023 12:50:19 -0400 Subject: [PATCH 05/16] adjust to renamed WebWorkerEventLoop.HasJavaScriptInteropDependents --- .../System/Threading/PortableThreadPool.Browser.Threads.Mono.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs index d459c992f810f..632b0c934ee4c 100644 --- a/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs +++ b/src/mono/System.Private.CoreLib/src/System/Threading/PortableThreadPool.Browser.Threads.Mono.cs @@ -7,7 +7,7 @@ internal sealed partial class PortableThreadPool { private static partial class WorkerThread { - private static bool IsIOPending => WebWorkerEventLoop.HasUnsettledInteropPromises; + private static bool IsIOPending => WebWorkerEventLoop.HasJavaScriptInteropDependents; } private struct CpuUtilizationReader From 2889a1fe7557179aa6e27a58e637112a803ac988 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Wed, 19 Apr 2023 15:20:03 -0400 Subject: [PATCH 06/16] extend and rationalize the smoke test a bit Add a test that just starts a thread and asserts that it has a different thread ID than the main thread. This should allow us to rule out accidentally having the test pass on a single-threaded runtime --- .../wasm/browser-threads-minimal/Program.cs | 18 +++++++++- .../wasm/browser-threads-minimal/main.js | 36 ++++++++++--------- 2 files changed, 37 insertions(+), 17 deletions(-) diff --git a/src/mono/sample/wasm/browser-threads-minimal/Program.cs b/src/mono/sample/wasm/browser-threads-minimal/Program.cs index b9ef3854ef163..d6ccecd2e1eed 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/Program.cs +++ b/src/mono/sample/wasm/browser-threads-minimal/Program.cs @@ -18,6 +18,22 @@ public static int Main(string[] args) return 0; } + [JSExport] + public static async Task TestCanStartThread() + { + var tcs = new TaskCompletionSource(); + var t = new Thread(() => + { + var childTid = Thread.CurrentThread.ManagedThreadId; + tcs.SetResult(childTid); + }); + t.Start(); + var childTid = await tcs.Task; + t.Join(); + if (childTid == Thread.CurrentThread.ManagedThreadId) + throw new Exception("Child thread ran on same thread as parent"); + } + [JSImport("globalThis.setTimeout")] static partial void GlobalThisSetTimeout([JSMarshalAs] Action cb, int timeoutMs); @@ -25,7 +41,7 @@ public static int Main(string[] args) private static partial Task GlobalThisFetch(string url); [JSExport] - public static async Task Hello() + public static async Task TestCallSetTimeoutOnWorker() { var t = Task.Run(TimeOutThenComplete); await t; diff --git a/src/mono/sample/wasm/browser-threads-minimal/main.js b/src/mono/sample/wasm/browser-threads-minimal/main.js index 4c6a28e916052..3179fd5739e5b 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/main.js +++ b/src/mono/sample/wasm/browser-threads-minimal/main.js @@ -15,36 +15,40 @@ try { const exports = await getAssemblyExports(assemblyName); - //console.log ("XYZ: running hello"); - //await exports.Sample.Test.Hello(); - //console.log ("XYZ: hello done"); + console.log("smoke: running TestCanStartThread"); + await exports.Sample.Test.TestCanStartThread(); + console.log("smoke: TestCanStartThread done"); - console.log ("XYZ: running FetchBackground"); + console.log ("smoke: running TestCallSetTimeoutOnWorker"); + await exports.Sample.Test.TestCallSetTimeoutOnWorker(); + console.log ("smoke: TestCallSetTimeoutOnWorker done"); + + console.log ("smoke: running FetchBackground(blurst.txt)"); let s = await exports.Sample.Test.FetchBackground("./blurst.txt"); - console.log ("XYZ: FetchBackground done"); + console.log ("smoke: FetchBackground(blurst.txt) done"); if (s !== "It was the best of times, it was the blurst of times.\n") { const msg = `Unexpected FetchBackground result ${s}`; document.getElementById("out").innerHTML = msg; throw new Error (msg); } - console.log ("XYZ: running FetchBackground(missing)"); + console.log ("smoke: running FetchBackground(missing)"); s = await exports.Sample.Test.FetchBackground("./missing.txt"); - console.log ("XYZ: FetchBackground(missing) done"); + console.log ("smoke: FetchBackground(missing) done"); if (s !== "not-ok") { const msg = `Unexpected FetchBackground(missing) result ${s}`; document.getElementById("out").innerHTML = msg; throw new Error (msg); } - - //console.log ("HHH: running TaskRunCompute"); - //const r1 = await exports.Sample.Test.RunBackgroundTaskRunCompute(); - //if (r1 !== 524) { - // const msg = `Unexpected result ${r1} from RunBackgorundTaskRunCompute()`; - // document.getElementById("out").innerHTML = msg; - // throw new Error(msg); - //} - //console.log ("HHH: TaskRunCompute done"); + + console.log ("smoke: running TaskRunCompute"); + const r1 = await exports.Sample.Test.RunBackgroundTaskRunCompute(); + if (r1 !== 524) { + const msg = `Unexpected result ${r1} from RunBackgorundTaskRunCompute()`; + document.getElementById("out").innerHTML = msg; + throw new Error(msg); + } + console.log ("smoke: TaskRunCompute done"); let exit_code = await runMain(assemblyName, []); From 92d80a510adbc9b85be2590f9f0554e1f139f8f8 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Wed, 19 Apr 2023 15:27:55 -0400 Subject: [PATCH 07/16] remove old-Emscripten workaround hack --- src/mono/mono/metadata/threads.c | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/src/mono/mono/metadata/threads.c b/src/mono/mono/metadata/threads.c index dad7323149f99..a7eeb4daea735 100644 --- a/src/mono/mono/metadata/threads.c +++ b/src/mono/mono/metadata/threads.c @@ -1250,14 +1250,8 @@ start_wrapper_internal (StartInfo *start_info, gsize *stack_ptr) #if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) if (returns_to_js_event_loop) { /* if the thread wants to stay alive, don't clean up after it */ -#ifdef MONO_EMSCRIPTEN_KEEPALIVE_WORKAROUND_HACK - /* we "know" that threadpool threads set their keepalive count correctly and will return here */ - g_assert (mono_emscripten_keepalive_hack_count > 0); - return 0; -#else if (emscripten_runtime_keepalive_check()) return 0; -#endif } #endif @@ -1296,15 +1290,8 @@ start_wrapper (gpointer data) #if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) if (returns_to_js_event_loop) { /* if the thread wants to stay alive, don't clean up after it */ -#ifdef MONO_EMSCRIPTEN_KEEPALIVE_WORKAROUND_HACK - /* we "know" the keepalive count is positive at this point for threadpool threads. Keep it alive */ - g_assert (mono_emscripten_keepalive_hack_count > 0); - emscripten_unwind_to_js_event_loop (); - g_assert_not_reached(); -#else if (emscripten_runtime_keepalive_check()) return 0; -#endif } #endif From 5f4acc83e39b41fad8b2233b2edb2471a6dda039 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 20 Apr 2023 11:26:05 -0400 Subject: [PATCH 08/16] hide some debug output --- src/mono/wasm/runtime/pthreads/worker/index.ts | 6 ++++-- src/mono/wasm/runtime/startup.ts | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/mono/wasm/runtime/pthreads/worker/index.ts b/src/mono/wasm/runtime/pthreads/worker/index.ts index 71a6f3bcfbe7e..205c78a62fa97 100644 --- a/src/mono/wasm/runtime/pthreads/worker/index.ts +++ b/src/mono/wasm/runtime/pthreads/worker/index.ts @@ -114,7 +114,8 @@ function onMonoConfigReceived(config: MonoConfigInternal): void { export function mono_wasm_pthread_on_pthread_attached(pthread_id: pthread_ptr): void { const self = pthread_self; mono_assert(self !== null && self.pthread_id == pthread_id, "expected pthread_self to be set already when attaching"); - console.debug("MONO_WASM: attaching pthread to runtime 0x" + pthread_id.toString(16)); + if (runtimeHelpers.diagnosticTracing) + console.debug("MONO_WASM: attaching pthread to runtime 0x" + pthread_id.toString(16)); preRunWorker(); currentWorkerThreadEvents.dispatchEvent(makeWorkerThreadEvent(dotnetPthreadAttached, self)); } @@ -127,7 +128,8 @@ export function afterThreadInitTLS(): void { if (ENVIRONMENT_IS_PTHREAD) { const pthread_ptr = (Module)["_pthread_self"](); mono_assert(!is_nullish(pthread_ptr), "pthread_self() returned null"); - console.debug("MONO_WASM: after thread init, pthread ptr 0x" + pthread_ptr.toString(16)); + if (runtimeHelpers.diagnosticTracing) + console.debug("MONO_WASM: after thread init, pthread ptr 0x" + pthread_ptr.toString(16)); const self = setupChannelToMainThread(pthread_ptr); currentWorkerThreadEvents.dispatchEvent(makeWorkerThreadEvent(dotnetPthreadCreated, self)); } diff --git a/src/mono/wasm/runtime/startup.ts b/src/mono/wasm/runtime/startup.ts index 4cdf6dbc080b4..b76a105009616 100644 --- a/src/mono/wasm/runtime/startup.ts +++ b/src/mono/wasm/runtime/startup.ts @@ -761,7 +761,8 @@ export async function mono_wasm_pthread_worker_init(module: DotnetModule, export pthreads_worker.setupPreloadChannelToMainThread(); // This is a good place for subsystems to attach listeners for pthreads_worker.currentWorkerThreadEvents pthreads_worker.currentWorkerThreadEvents.addEventListener(pthreads_worker.dotnetPthreadCreated, (ev) => { - console.debug("MONO_WASM: pthread created 0x" + ev.pthread_self.pthread_id.toString(16)); + if (runtimeHelpers.diagnosticTracing) + console.debug("MONO_WASM: pthread created 0x" + ev.pthread_self.pthread_id.toString(16)); }); // this is the only event which is called on worker From 806b7705bc6b09bbefcc4fde0dc8f49728149d22 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 20 Apr 2023 15:29:40 -0400 Subject: [PATCH 09/16] smoke test: dispose of the ImportAsync result after the task is done --- src/mono/sample/wasm/browser-threads-minimal/Program.cs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/mono/sample/wasm/browser-threads-minimal/Program.cs b/src/mono/sample/wasm/browser-threads-minimal/Program.cs index d6ccecd2e1eed..4379c9092bf61 100644 --- a/src/mono/sample/wasm/browser-threads-minimal/Program.cs +++ b/src/mono/sample/wasm/browser-threads-minimal/Program.cs @@ -58,7 +58,7 @@ public static async Task FetchBackground(string url) { var t = Task.Run(async () => { - await JSHost.ImportAsync(fetchhelper, "./fetchhelper.js"); + using var import = await JSHost.ImportAsync(fetchhelper, "./fetchhelper.js"); var r = await GlobalThisFetch(url); var ok = (bool)r.GetPropertyAsBoolean("ok"); From d2811f7144bc690b5b5cf359435eb631cc45701c Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 20 Apr 2023 15:31:55 -0400 Subject: [PATCH 10/16] [wasm-mt] make JSHostImplementation.s_csOwnedObjects ThreadStatic The integer jsHandles are not global - they are essentially indices into a JS array. So the mapping from a jsHandle to a JSObject must be per-thread. This fixes the thread affinity assertions in the smoketest (which were false positives - we looked up a worker's jsHandle and got back the main thread's JSObject - and then asserted that it was accessed from the wrong thread) --- .../JavaScript/JSHostImplementation.cs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs index 7738bb834fc21..66dd12e470c40 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs @@ -15,7 +15,21 @@ internal static partial class JSHostImplementation private const string TaskGetResultName = "get_Result"; private static MethodInfo? s_taskGetResultMethodInfo; // we use this to maintain identity of JSHandle for a JSObject proxy +#if FEATURE_WASM_THREADS + [ThreadStatic] + private static Dictionary>? s_csOwnedObjects_impl; + + public static Dictionary> s_csOwnedObjects + { + get + { + s_csOwnedObjects_impl ??= new (); + return s_csOwnedObjects_impl; + } + } +#else public static readonly Dictionary> s_csOwnedObjects = new Dictionary>(); +#endif // we use this to maintain identity of GCHandle for a managed object public static Dictionary s_gcHandleFromJSOwnedObject = new Dictionary(ReferenceEqualityComparer.Instance); From 48d22aea63cd5db77f1c87a1f7c9d87fac75e9ff Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Fri, 21 Apr 2023 12:07:33 -0400 Subject: [PATCH 11/16] remove locking on JSHostImplementation.CsOwnedObjects In single-threaded wasm, there is no need to lock since there is only one caller at a time. In multi-threaded wasm, the underlying dictionary is thread-static --- .../JavaScript/Interop/LegacyExports.cs | 28 +++++++--------- .../JavaScript/JSHostImplementation.cs | 33 ++++++++----------- .../Legacy/LegacyHostImplementation.cs | 5 +-- 3 files changed, 25 insertions(+), 41 deletions(-) diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs index 38605743fad93..2f1d630340ed0 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs @@ -32,18 +32,15 @@ internal static void PreventTrimming() public static void GetCSOwnedObjectByJSHandleRef(nint jsHandle, int shouldAddInflight, out JSObject? result) { - lock (JSHostImplementation.s_csOwnedObjects) + if (JSHostImplementation.CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference)) { - if (JSHostImplementation.s_csOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference)) + reference.TryGetTarget(out JSObject? jsObject); + if (shouldAddInflight != 0) { - reference.TryGetTarget(out JSObject? jsObject); - if (shouldAddInflight != 0) - { - jsObject?.AddInFlight(); - } - result = jsObject; - return; + jsObject?.AddInFlight(); } + result = jsObject; + return; } result = null; } @@ -77,14 +74,12 @@ public static void CreateCSOwnedProxyRef(nint jsHandle, LegacyHostImplementation JSObject? res = null; - lock (JSHostImplementation.s_csOwnedObjects) + if (!JSHostImplementation.CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || + !reference.TryGetTarget(out res) || + res.IsDisposed) { - if (!JSHostImplementation.s_csOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || - !reference.TryGetTarget(out res) || - res.IsDisposed) - { #pragma warning disable CS0612 // Type or member is obsolete - res = mappedType switch + res = mappedType switch { LegacyHostImplementation.MappedType.JSObject => new JSObject(jsHandle), LegacyHostImplementation.MappedType.Array => new Array(jsHandle), @@ -95,8 +90,7 @@ public static void CreateCSOwnedProxyRef(nint jsHandle, LegacyHostImplementation _ => throw new ArgumentOutOfRangeException(nameof(mappedType)) }; #pragma warning restore CS0612 // Type or member is obsolete - JSHostImplementation.s_csOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); - } + JSHostImplementation.CsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); } if (shouldAddInflight != 0) { diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs index 66dd12e470c40..6d59ba394f275 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs @@ -17,19 +17,18 @@ internal static partial class JSHostImplementation // we use this to maintain identity of JSHandle for a JSObject proxy #if FEATURE_WASM_THREADS [ThreadStatic] - private static Dictionary>? s_csOwnedObjects_impl; +#endif + private static Dictionary>? s_csOwnedObjects; - public static Dictionary> s_csOwnedObjects + public static Dictionary> CsOwnedObjects { get { - s_csOwnedObjects_impl ??= new (); - return s_csOwnedObjects_impl; + s_csOwnedObjects ??= new (); + return s_csOwnedObjects; } } -#else - public static readonly Dictionary> s_csOwnedObjects = new Dictionary>(); -#endif + // we use this to maintain identity of GCHandle for a managed object public static Dictionary s_gcHandleFromJSOwnedObject = new Dictionary(ReferenceEqualityComparer.Instance); @@ -38,10 +37,7 @@ public static void ReleaseCSOwnedObject(nint jsHandle) { if (jsHandle != IntPtr.Zero) { - lock (s_csOwnedObjects) - { - s_csOwnedObjects.Remove((int)jsHandle); - } + CsOwnedObjects.Remove((int)jsHandle); Interop.Runtime.ReleaseCSOwnedObject(jsHandle); } } @@ -189,17 +185,14 @@ public static unsafe void FreeMethodSignatureBuffer(JSFunctionBinding signature) public static JSObject CreateCSOwnedProxy(nint jsHandle) { - JSObject? res = null; + JSObject? res; - lock (s_csOwnedObjects) + if (!CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || + !reference.TryGetTarget(out res) || + res.IsDisposed) { - if (!s_csOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || - !reference.TryGetTarget(out res) || - res.IsDisposed) - { - res = new JSObject(jsHandle); - s_csOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); - } + res = new JSObject(jsHandle); + CsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); } return res; } diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs index 6a5de6a03e503..a9b522a7c8b18 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs @@ -21,10 +21,7 @@ public static void ReleaseInFlight(object obj) [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void RegisterCSOwnedObject(JSObject proxy) { - lock (JSHostImplementation.s_csOwnedObjects) - { - JSHostImplementation.s_csOwnedObjects[(int)proxy.JSHandle] = new WeakReference(proxy, trackResurrection: true); - } + JSHostImplementation.CsOwnedObjects[(int)proxy.JSHandle] = new WeakReference(proxy, trackResurrection: true); } public static MarshalType GetMarshalTypeFromType(Type type) From 77a54ffc1e1d5c131ccd6c56407250bd44aac800 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 27 Apr 2023 16:42:15 -0400 Subject: [PATCH 12/16] [threads] make the "external eventloop" platform independent It only does something on WASM, but in principle if other platforms allow us to run some code after returning from a thread start function, we could do it there, too. --- src/mono/mono/metadata/threads-types.h | 7 +++-- src/mono/mono/metadata/threads.c | 32 +++++++--------------- src/mono/mono/utils/mono-threads-posix.c | 9 ++++++ src/mono/mono/utils/mono-threads-wasm.c | 10 +++++++ src/mono/mono/utils/mono-threads-windows.c | 9 ++++++ src/mono/mono/utils/mono-threads.h | 3 ++ 6 files changed, 45 insertions(+), 25 deletions(-) diff --git a/src/mono/mono/metadata/threads-types.h b/src/mono/mono/metadata/threads-types.h index b9652aa33eb32..5e89f84bef211 100644 --- a/src/mono/mono/metadata/threads-types.h +++ b/src/mono/mono/metadata/threads-types.h @@ -78,9 +78,10 @@ typedef enum { MONO_THREAD_CREATE_FLAGS_DEBUGGER = 0x02, MONO_THREAD_CREATE_FLAGS_FORCE_CREATE = 0x04, MONO_THREAD_CREATE_FLAGS_SMALL_STACK = 0x08, -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP = 0x10, -#endif + // "external eventloop" means the thread main function can return without killing the thread + // and the thread will continue to be attached to the runtime and may invoke embedding APIs + // and managed calls. There is usually some platform-specific way to shut down the thread. + MONO_THREAD_CREATE_FLAGS_EXTERNAL_EVENTLOOP = 0x10, } MonoThreadCreateFlags; MONO_COMPONENT_API MonoInternalThread* diff --git a/src/mono/mono/metadata/threads.c b/src/mono/mono/metadata/threads.c index a7eeb4daea735..72038b9d2c879 100644 --- a/src/mono/mono/metadata/threads.c +++ b/src/mono/mono/metadata/threads.c @@ -1088,9 +1088,7 @@ typedef struct { MonoThreadStart start_func; gpointer start_func_arg; gboolean force_attach; -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - gboolean returns_to_js_event_loop; -#endif + gboolean external_eventloop; gboolean failed; MonoCoopSem registered; } StartInfo; @@ -1176,9 +1174,7 @@ start_wrapper_internal (StartInfo *start_info, gsize *stack_ptr) /* Let the thread that called Start() know we're ready */ mono_coop_sem_post (&start_info->registered); -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - gboolean returns_to_js_event_loop = start_info->returns_to_js_event_loop; -#endif + gboolean external_eventloop = start_info->external_eventloop; if (mono_atomic_dec_i32 (&start_info->ref) == 0) { mono_coop_sem_destroy (&start_info->registered); @@ -1247,13 +1243,11 @@ start_wrapper_internal (StartInfo *start_info, gsize *stack_ptr) THREAD_DEBUG (g_message ("%s: (%" G_GSIZE_FORMAT ") Start wrapper terminating", __func__, mono_native_thread_id_get ())); -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - if (returns_to_js_event_loop) { - /* if the thread wants to stay alive, don't clean up after it */ - if (emscripten_runtime_keepalive_check()) + if (G_UNLIKELY (external_eventloop)) { + /* if the thread wants to stay alive in an external eventloop, don't clean up after it */ + if (mono_thread_platform_external_eventloop_keepalive_check ()) return 0; } -#endif /* Do any cleanup needed for apartment state. This * cannot be done in mono_thread_detach_internal since @@ -1281,19 +1275,15 @@ start_wrapper (gpointer data) info = mono_thread_info_attach (); info->runtime_thread = TRUE; -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - gboolean returns_to_js_event_loop = start_info->returns_to_js_event_loop; -#endif + gboolean external_eventloop = start_info->external_eventloop; /* Run the actual main function of the thread */ res = start_wrapper_internal (start_info, (gsize*)info->stack_end); -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - if (returns_to_js_event_loop) { + if (G_UNLIKELY (external_eventloop)) { /* if the thread wants to stay alive, don't clean up after it */ - if (emscripten_runtime_keepalive_check()) + if (mono_thread_platform_external_eventloop_keepalive_check ()) return 0; } -#endif mono_thread_info_exit (res); @@ -1381,9 +1371,7 @@ create_thread (MonoThread *thread, MonoInternalThread *internal, MonoThreadStart start_info->start_func_arg = start_func_arg; start_info->force_attach = flags & MONO_THREAD_CREATE_FLAGS_FORCE_CREATE; start_info->failed = FALSE; -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - start_info->returns_to_js_event_loop = (flags & MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP) != 0; -#endif + start_info->external_eventloop = (flags & MONO_THREAD_CREATE_FLAGS_EXTERNAL_EVENTLOOP) != 0; mono_coop_sem_init (&start_info->registered, 0); if (flags != MONO_THREAD_CREATE_FLAGS_SMALL_STACK) @@ -4947,7 +4935,7 @@ ves_icall_System_Threading_Thread_StartInternal (MonoThreadObjectHandle thread_h // HACK: threadpool threads can return to the JS event loop // WISH: support this for other threads, too if (internal->threadpool_thread) - create_flags |= MONO_THREAD_CREATE_FLAGS_RETURNS_TO_JS_EVENT_LOOP; + create_flags |= MONO_THREAD_CREATE_FLAGS_EXTERNAL_EVENTLOOP; #endif res = create_thread (internal, internal, NULL, NULL, stack_size, create_flags, error); diff --git a/src/mono/mono/utils/mono-threads-posix.c b/src/mono/mono/utils/mono-threads-posix.c index 08e843cdd7023..a70691d01e8fe 100644 --- a/src/mono/mono/utils/mono-threads-posix.c +++ b/src/mono/mono/utils/mono-threads-posix.c @@ -133,6 +133,15 @@ mono_threads_platform_exit (gsize exit_code) pthread_exit ((gpointer) exit_code); } +gboolean +mono_thread_platform_external_eventloop_keepalive_check (void) +{ + /* vanilla POSIX thread creation doesn't support an external eventloop: when the thread main + function returns, the thread is done. + */ + return FALSE; +} + #if HOST_FUCHSIA int mono_thread_info_get_system_max_stack_size (void) diff --git a/src/mono/mono/utils/mono-threads-wasm.c b/src/mono/mono/utils/mono-threads-wasm.c index 96e5446388a43..371d156b9e823 100644 --- a/src/mono/mono/utils/mono-threads-wasm.c +++ b/src/mono/mono/utils/mono-threads-wasm.c @@ -301,6 +301,16 @@ mono_thread_platform_create_thread (MonoThreadStart thread_fn, gpointer thread_d #endif } +gboolean +mono_thread_platform_external_eventloop_keepalive_check (void) +{ + /* if someone called emscripten_runtime_keepalive_push (), the + * thread will stay alive in the JS event loop after returning + * from the thread's main function. + */ + return emscripten_runtime_keepalive_check (); +} + void mono_threads_platform_init (void) { } diff --git a/src/mono/mono/utils/mono-threads-windows.c b/src/mono/mono/utils/mono-threads-windows.c index 3e56205c0ab88..169449b831e83 100644 --- a/src/mono/mono/utils/mono-threads-windows.c +++ b/src/mono/mono/utils/mono-threads-windows.c @@ -501,6 +501,15 @@ typedef BOOL (WINAPI *LPFN_ISWOW64PROCESS) (HANDLE, PBOOL); static gboolean is_wow64 = FALSE; #endif +gboolean +mono_thread_platform_external_eventloop_keepalive_check (void) +{ + /* We don't support thread creation with an external eventloop on WIN32: when the thread start + function returns, the thread is done. + */ + return FALSE; +} + /* We do this at init time to avoid potential races with module opening */ void mono_threads_platform_init (void) diff --git a/src/mono/mono/utils/mono-threads.h b/src/mono/mono/utils/mono-threads.h index 6a548b1838c34..cdfc9f6133671 100644 --- a/src/mono/mono/utils/mono-threads.h +++ b/src/mono/mono/utils/mono-threads.h @@ -632,6 +632,9 @@ gboolean mono_threads_platform_in_critical_region (THREAD_INFO_TYPE *info); gboolean mono_threads_platform_yield (void); void mono_threads_platform_exit (gsize exit_code); +gboolean +mono_thread_platform_external_eventloop_keepalive_check (void); + void mono_threads_coop_begin_global_suspend (void); void mono_threads_coop_end_global_suspend (void); From fd99953ef793f380a4ff9908e6bd2a38d9820e4d Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 27 Apr 2023 18:16:08 -0400 Subject: [PATCH 13/16] fix wasi and singlethreaded browser-wasm --- src/mono/mono/utils/mono-threads-wasm.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/mono/mono/utils/mono-threads-wasm.c b/src/mono/mono/utils/mono-threads-wasm.c index 371d156b9e823..6a33dfa0d5bc2 100644 --- a/src/mono/mono/utils/mono-threads-wasm.c +++ b/src/mono/mono/utils/mono-threads-wasm.c @@ -304,11 +304,15 @@ mono_thread_platform_create_thread (MonoThreadStart thread_fn, gpointer thread_d gboolean mono_thread_platform_external_eventloop_keepalive_check (void) { +#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) /* if someone called emscripten_runtime_keepalive_push (), the * thread will stay alive in the JS event loop after returning * from the thread's main function. */ return emscripten_runtime_keepalive_check (); +#else + return FALSE; +#endif } void mono_threads_platform_init (void) From 4671e03b194b456c7fa48b56c1127f92ae5bd99c Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 27 Apr 2023 18:26:05 -0400 Subject: [PATCH 14/16] Add a Thread.HasExternalEventLoop managed property Set it from WebWorkerEventLoop.StartExitable. In native code, use it to set the `MONO_THREAD_CREATE_FLAGS_EXTERNAL_EVENTLOOP` flag when starting the thread. --- .../src/System/Threading/Thread.Mono.cs | 13 +++++++++++++ .../WebWorkerEventLoop.Browser.Threads.Mono.cs | 4 ++-- src/mono/mono/metadata/object-internals.h | 1 + src/mono/mono/metadata/threads.c | 6 +----- 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/Thread.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/Thread.Mono.cs index 31bc824008607..d5b1918608b77 100644 --- a/src/mono/System.Private.CoreLib/src/System/Threading/Thread.Mono.cs +++ b/src/mono/System.Private.CoreLib/src/System/Threading/Thread.Mono.cs @@ -37,6 +37,7 @@ public partial class Thread private int interruption_requested; private IntPtr longlived; internal bool threadpool_thread; + internal bool external_eventloop; // browser-wasm: thread will return to the JS eventloop /* These are used from managed code */ internal byte apartment_state; internal int managed_id; @@ -352,5 +353,17 @@ private static void SpinWait_nop() private static extern void SetPriority(Thread thread, int priority); internal int GetSmallId() => small_id; + + internal bool HasExternalEventLoop + { + get + { + return external_eventloop; + } + set + { + external_eventloop = value; + } + } } } diff --git a/src/mono/System.Private.CoreLib/src/System/Threading/WebWorkerEventLoop.Browser.Threads.Mono.cs b/src/mono/System.Private.CoreLib/src/System/Threading/WebWorkerEventLoop.Browser.Threads.Mono.cs index 5a49c076271d1..73c2959293d52 100644 --- a/src/mono/System.Private.CoreLib/src/System/Threading/WebWorkerEventLoop.Browser.Threads.Mono.cs +++ b/src/mono/System.Private.CoreLib/src/System/Threading/WebWorkerEventLoop.Browser.Threads.Mono.cs @@ -67,10 +67,10 @@ internal static void StartExitable(Thread thread, bool captureContext) // not needed by PortableThreadPool.WorkerThread if (captureContext) throw new InvalidOperationException(); - // hack: threadpool threads are exitable, and nothing else is. - // see create_thread() in mono/metadata/threads.c + // for now, threadpool threads are exitable, and nothing else is. if (!thread.IsThreadPoolThread) throw new InvalidOperationException(); + thread.HasExternalEventLoop = true; thread.UnsafeStart(); } diff --git a/src/mono/mono/metadata/object-internals.h b/src/mono/mono/metadata/object-internals.h index be4ce486c91a7..b968762c210cc 100644 --- a/src/mono/mono/metadata/object-internals.h +++ b/src/mono/mono/metadata/object-internals.h @@ -616,6 +616,7 @@ struct _MonoInternalThread { * longer */ MonoLongLivedThreadData *longlived; MonoBoolean threadpool_thread; + MonoBoolean external_eventloop; guint8 apartment_state; gint32 managed_id; guint32 small_id; diff --git a/src/mono/mono/metadata/threads.c b/src/mono/mono/metadata/threads.c index 72038b9d2c879..9821eabb38500 100644 --- a/src/mono/mono/metadata/threads.c +++ b/src/mono/mono/metadata/threads.c @@ -4931,12 +4931,8 @@ ves_icall_System_Threading_Thread_StartInternal (MonoThreadObjectHandle thread_h } MonoThreadCreateFlags create_flags = MONO_THREAD_CREATE_FLAGS_NONE; -#if defined(HOST_BROWSER) && !defined(DISABLE_THREADS) - // HACK: threadpool threads can return to the JS event loop - // WISH: support this for other threads, too - if (internal->threadpool_thread) + if (G_UNLIKELY (internal->external_eventloop)) create_flags |= MONO_THREAD_CREATE_FLAGS_EXTERNAL_EVENTLOOP; -#endif res = create_thread (internal, internal, NULL, NULL, stack_size, create_flags, error); if (!res) { From e22ca465f84b1992701a3235a184c476db82fe71 Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Thu, 27 Apr 2023 18:33:39 -0400 Subject: [PATCH 15/16] rename JSHostImplementation.ThreadCsOwnedObjects (used to be CsOwnedObjects) Rename to make it clear that it's objects owned by the current thread, not the runtime globally --- .../InteropServices/JavaScript/Interop/LegacyExports.cs | 6 +++--- .../InteropServices/JavaScript/JSHostImplementation.cs | 8 ++++---- .../JavaScript/Legacy/LegacyHostImplementation.cs | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs index 2f1d630340ed0..c15b81826b0fb 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Interop/LegacyExports.cs @@ -32,7 +32,7 @@ internal static void PreventTrimming() public static void GetCSOwnedObjectByJSHandleRef(nint jsHandle, int shouldAddInflight, out JSObject? result) { - if (JSHostImplementation.CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference)) + if (JSHostImplementation.ThreadCsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference)) { reference.TryGetTarget(out JSObject? jsObject); if (shouldAddInflight != 0) @@ -74,7 +74,7 @@ public static void CreateCSOwnedProxyRef(nint jsHandle, LegacyHostImplementation JSObject? res = null; - if (!JSHostImplementation.CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || + if (!JSHostImplementation.ThreadCsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || !reference.TryGetTarget(out res) || res.IsDisposed) { @@ -90,7 +90,7 @@ public static void CreateCSOwnedProxyRef(nint jsHandle, LegacyHostImplementation _ => throw new ArgumentOutOfRangeException(nameof(mappedType)) }; #pragma warning restore CS0612 // Type or member is obsolete - JSHostImplementation.CsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); + JSHostImplementation.ThreadCsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); } if (shouldAddInflight != 0) { diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs index 6d59ba394f275..b439586d21301 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/JSHostImplementation.cs @@ -20,7 +20,7 @@ internal static partial class JSHostImplementation #endif private static Dictionary>? s_csOwnedObjects; - public static Dictionary> CsOwnedObjects + public static Dictionary> ThreadCsOwnedObjects { get { @@ -37,7 +37,7 @@ public static void ReleaseCSOwnedObject(nint jsHandle) { if (jsHandle != IntPtr.Zero) { - CsOwnedObjects.Remove((int)jsHandle); + ThreadCsOwnedObjects.Remove((int)jsHandle); Interop.Runtime.ReleaseCSOwnedObject(jsHandle); } } @@ -187,12 +187,12 @@ public static JSObject CreateCSOwnedProxy(nint jsHandle) { JSObject? res; - if (!CsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || + if (!ThreadCsOwnedObjects.TryGetValue((int)jsHandle, out WeakReference? reference) || !reference.TryGetTarget(out res) || res.IsDisposed) { res = new JSObject(jsHandle); - CsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); + ThreadCsOwnedObjects[(int)jsHandle] = new WeakReference(res, trackResurrection: true); } return res; } diff --git a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs index a9b522a7c8b18..661b21690670a 100644 --- a/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs +++ b/src/libraries/System.Runtime.InteropServices.JavaScript/src/System/Runtime/InteropServices/JavaScript/Legacy/LegacyHostImplementation.cs @@ -21,7 +21,7 @@ public static void ReleaseInFlight(object obj) [MethodImpl(MethodImplOptions.AggressiveInlining)] public static void RegisterCSOwnedObject(JSObject proxy) { - JSHostImplementation.CsOwnedObjects[(int)proxy.JSHandle] = new WeakReference(proxy, trackResurrection: true); + JSHostImplementation.ThreadCsOwnedObjects[(int)proxy.JSHandle] = new WeakReference(proxy, trackResurrection: true); } public static MarshalType GetMarshalTypeFromType(Type type) From c5915016e721d25c174ab9c6ece0248f1479687f Mon Sep 17 00:00:00 2001 From: Aleksey Kliger Date: Fri, 28 Apr 2023 13:19:11 -0400 Subject: [PATCH 16/16] [checked] assert GC Safe mode, when returning to external eventloop --- src/mono/mono/metadata/threads.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/mono/mono/metadata/threads.c b/src/mono/mono/metadata/threads.c index 9821eabb38500..0788d9ccdac13 100644 --- a/src/mono/mono/metadata/threads.c +++ b/src/mono/mono/metadata/threads.c @@ -1281,8 +1281,11 @@ start_wrapper (gpointer data) if (G_UNLIKELY (external_eventloop)) { /* if the thread wants to stay alive, don't clean up after it */ - if (mono_thread_platform_external_eventloop_keepalive_check ()) + if (mono_thread_platform_external_eventloop_keepalive_check ()) { + /* while we wait in the external eventloop, we're GC safe */ + MONO_REQ_GC_SAFE_MODE; return 0; + } } mono_thread_info_exit (res);