summaryrefslogtreecommitdiff
path: root/core/runtime.rs
diff options
context:
space:
mode:
authorNayeem Rahman <nayeemrmn99@gmail.com>2023-06-03 21:22:32 +0100
committerGitHub <noreply@github.com>2023-06-03 14:22:32 -0600
commit34dac6c6efa75f38c29031a65db1ee3332a67259 (patch)
tree5facbc9c0631feebc23ed2f2d9ea37643d0e7086 /core/runtime.rs
parent7d0853d15863b2fb61bcf5927139cfdd3d869d73 (diff)
refactor(core): remove force_op_registration and cleanup JsRuntimeForSnapshot (#19353)
Addresses https://github.com/denoland/deno/pull/19308#discussion_r1212248194. Removes force_op_registration as it is no longer necessary.
Diffstat (limited to 'core/runtime.rs')
-rw-r--r--core/runtime.rs390
1 files changed, 153 insertions, 237 deletions
diff --git a/core/runtime.rs b/core/runtime.rs
index f95c4a8ef..fdcb81e9e 100644
--- a/core/runtime.rs
+++ b/core/runtime.rs
@@ -1,7 +1,6 @@
// Copyright 2018-2023 the Deno authors. All rights reserved. MIT license.
use crate::bindings;
-use crate::bindings::BindingsMode;
use crate::error::generic_error;
use crate::error::to_v8_type_error;
use crate::error::JsError;
@@ -24,8 +23,6 @@ use crate::realm::ContextState;
use crate::realm::JsRealm;
use crate::realm::JsRealmInner;
use crate::snapshot_util;
-use crate::snapshot_util::SnapshotOptions;
-use crate::snapshot_util::SnapshottedData;
use crate::source_map::SourceMapCache;
use crate::source_map::SourceMapGetter;
use crate::Extension;
@@ -63,7 +60,6 @@ use std::sync::Mutex;
use std::sync::Once;
use std::task::Context;
use std::task::Poll;
-use v8::CreateParams;
const STATE_DATA_OFFSET: u32 = 0;
const MODULE_MAP_DATA_OFFSET: u32 = 1;
@@ -121,7 +117,7 @@ impl<T> DerefMut for ManuallyDropRc<T> {
/// This inner struct allows us to let the outer JsRuntime drop normally without a Drop impl, while we
/// control dropping more closely here using ManuallyDrop.
struct InnerIsolateState {
- snapshotting: bool,
+ will_snapshot: bool,
state: ManuallyDropRc<RefCell<JsRuntimeState>>,
v8_isolate: ManuallyDrop<v8::OwnedIsolate>,
}
@@ -182,7 +178,7 @@ impl Drop for InnerIsolateState {
// SAFETY: We gotta drop these
unsafe {
ManuallyDrop::drop(&mut self.state.0);
- if self.snapshotting {
+ if self.will_snapshot {
// Create the snapshot and just drop it.
eprintln!("WARNING: v8::OwnedIsolate for snapshot was leaked");
} else {
@@ -192,47 +188,46 @@ impl Drop for InnerIsolateState {
}
}
+#[derive(Copy, Clone, Debug, Eq, PartialEq)]
+pub(crate) enum InitMode {
+ /// We have no snapshot -- this is a pristine context.
+ New,
+ /// We are using a snapshot, thus certain initialization steps are skipped.
+ FromSnapshot,
+}
+
+impl InitMode {
+ fn from_options(options: &RuntimeOptions) -> Self {
+ match options.startup_snapshot {
+ None => Self::New,
+ Some(_) => Self::FromSnapshot,
+ }
+ }
+}
+
/// A single execution context of JavaScript. Corresponds roughly to the "Web
/// Worker" concept in the DOM.
////
-/// The JsRuntimeImpl future completes when there is an error or when all
+/// The JsRuntime future completes when there is an error or when all
/// pending ops have completed.
///
-/// API consumers will want to use either the [`JsRuntime`] or [`JsRuntimeForSnapshot`]
-/// type aliases.
-pub struct JsRuntimeImpl<const FOR_SNAPSHOT: bool = false> {
+/// Use [`JsRuntimeForSnapshot`] to be able to create a snapshot.
+pub struct JsRuntime {
inner: InnerIsolateState,
module_map: Rc<RefCell<ModuleMap>>,
allocations: IsolateAllocations,
extensions: Vec<Extension>,
event_loop_middlewares: Vec<Box<OpEventLoopFn>>,
- bindings_mode: BindingsMode,
+ init_mode: InitMode,
// Marks if this is considered the top-level runtime. Used only be inspector.
is_main: bool,
}
-/// The runtime type that most users will use when not creating a snapshot.
-pub struct JsRuntime(JsRuntimeImpl<false>);
-
-impl Deref for JsRuntime {
- type Target = JsRuntimeImpl<false>;
-
- fn deref(&self) -> &Self::Target {
- &self.0
- }
-}
-
-impl DerefMut for JsRuntime {
- fn deref_mut(&mut self) -> &mut Self::Target {
- &mut self.0
- }
-}
-
/// The runtime type used for snapshot creation.
-pub struct JsRuntimeForSnapshot(JsRuntimeImpl<true>);
+pub struct JsRuntimeForSnapshot(JsRuntime);
impl Deref for JsRuntimeForSnapshot {
- type Target = JsRuntimeImpl<true>;
+ type Target = JsRuntime;
fn deref(&self) -> &Self::Target {
&self.0
@@ -301,7 +296,7 @@ pub type SharedArrayBufferStore =
pub type CompiledWasmModuleStore = CrossIsolateStore<v8::CompiledWasmModule>;
-/// Internal state for JsRuntimeImpl which is stored in one of v8::Isolate's
+/// Internal state for JsRuntime which is stored in one of v8::Isolate's
/// embedder slots.
pub struct JsRuntimeState {
global_realm: Option<JsRealm>,
@@ -398,7 +393,7 @@ pub struct RuntimeOptions {
/// executed tries to load modules.
pub module_loader: Option<Rc<dyn ModuleLoader>>,
- /// JsRuntimeImpl extensions, not to be confused with ES modules.
+ /// JsRuntime extensions, not to be confused with ES modules.
/// Only ops registered by extensions will be initialized. If you need
/// to execute JS code from extensions, pass source files in `js` or `esm`
/// option on `ExtensionBuilder`.
@@ -448,21 +443,60 @@ pub struct RuntimeSnapshotOptions {
pub snapshot_module_load_cb: Option<ExtModuleLoaderCb>,
}
-trait JsRuntimeInternalTrait {
- fn create_raw_isolate(
- refs: &'static v8::ExternalReferences,
- params: Option<CreateParams>,
- snapshot: SnapshotOptions,
- ) -> v8::OwnedIsolate;
-}
+impl JsRuntime {
+ /// Only constructor, configuration is done through `options`.
+ pub fn new(mut options: RuntimeOptions) -> JsRuntime {
+ JsRuntime::init_v8(options.v8_platform.take(), cfg!(test));
+ JsRuntime::new_inner(options, false, None)
+ }
+
+ pub(crate) fn state_from(
+ isolate: &v8::Isolate,
+ ) -> Rc<RefCell<JsRuntimeState>> {
+ let state_ptr = isolate.get_data(STATE_DATA_OFFSET);
+ let state_rc =
+ // SAFETY: We are sure that it's a valid pointer for whole lifetime of
+ // the runtime.
+ unsafe { Rc::from_raw(state_ptr as *const RefCell<JsRuntimeState>) };
+ let state = state_rc.clone();
+ std::mem::forget(state_rc);
+ state
+ }
-impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
- fn init_v8(v8_platform: Option<v8::SharedRef<v8::Platform>>) {
+ pub(crate) fn module_map_from(
+ isolate: &v8::Isolate,
+ ) -> Rc<RefCell<ModuleMap>> {
+ let module_map_ptr = isolate.get_data(MODULE_MAP_DATA_OFFSET);
+ let module_map_rc =
+ // SAFETY: We are sure that it's a valid pointer for whole lifetime of
+ // the runtime.
+ unsafe { Rc::from_raw(module_map_ptr as *const RefCell<ModuleMap>) };
+ let module_map = module_map_rc.clone();
+ std::mem::forget(module_map_rc);
+ module_map
+ }
+
+ pub(crate) fn event_loop_pending_state_from_scope(
+ scope: &mut v8::HandleScope,
+ ) -> EventLoopPendingState {
+ let state = JsRuntime::state_from(scope);
+ let module_map = JsRuntime::module_map_from(scope);
+ let state = EventLoopPendingState::new(
+ scope,
+ &mut state.borrow_mut(),
+ &module_map.borrow(),
+ );
+ state
+ }
+
+ fn init_v8(
+ v8_platform: Option<v8::SharedRef<v8::Platform>>,
+ predictable: bool,
+ ) {
static DENO_INIT: Once = Once::new();
static DENO_PREDICTABLE: AtomicBool = AtomicBool::new(false);
static DENO_PREDICTABLE_SET: AtomicBool = AtomicBool::new(false);
- let predictable = FOR_SNAPSHOT || cfg!(test);
if DENO_PREDICTABLE_SET.load(Ordering::SeqCst) {
let current = DENO_PREDICTABLE.load(Ordering::SeqCst);
assert_eq!(current, predictable, "V8 may only be initialized once in either snapshotting or non-snapshotting mode. Either snapshotting or non-snapshotting mode may be used in a single process, not both.");
@@ -473,15 +507,13 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
DENO_INIT.call_once(move || v8_init(v8_platform, predictable));
}
- fn new_runtime(
+ fn new_inner(
mut options: RuntimeOptions,
- snapshot_options: SnapshotOptions,
+ will_snapshot: bool,
maybe_load_callback: Option<ExtModuleLoaderCb>,
- ) -> JsRuntimeImpl<FOR_SNAPSHOT>
- where
- JsRuntimeImpl<FOR_SNAPSHOT>: JsRuntimeInternalTrait,
- {
- let (op_state, ops) = Self::create_opstate(&mut options, &snapshot_options);
+ ) -> JsRuntime {
+ let init_mode = InitMode::from_options(&options);
+ let (op_state, ops) = Self::create_opstate(&mut options, init_mode);
let op_state = Rc::new(RefCell::new(op_state));
// Collect event-loop middleware
@@ -546,19 +578,56 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
// V8 takes ownership of external_references.
let refs: &'static v8::ExternalReferences = Box::leak(Box::new(refs));
- let bindings_mode = match snapshot_options {
- SnapshotOptions::None => bindings::BindingsMode::New,
- SnapshotOptions::Create => bindings::BindingsMode::New,
- SnapshotOptions::Load(_) => bindings::BindingsMode::LoadedFinal,
- SnapshotOptions::CreateFromExisting(_) => bindings::BindingsMode::Loaded,
+ let mut isolate = if will_snapshot {
+ snapshot_util::create_snapshot_creator(
+ refs,
+ options.startup_snapshot.take(),
+ )
+ } else {
+ let mut params = options
+ .create_params
+ .take()
+ .unwrap_or_default()
+ .embedder_wrapper_type_info_offsets(
+ V8_WRAPPER_TYPE_INDEX,
+ V8_WRAPPER_OBJECT_INDEX,
+ )
+ .external_references(&**refs);
+ if let Some(snapshot) = options.startup_snapshot.take() {
+ params = match snapshot {
+ Snapshot::Static(data) => params.snapshot_blob(data),
+ Snapshot::JustCreated(data) => params.snapshot_blob(data),
+ Snapshot::Boxed(data) => params.snapshot_blob(data),
+ };
+ }
+ v8::Isolate::new(params)
};
- let snapshotting = snapshot_options.will_snapshot();
-
- let (mut isolate, global_context, snapshotted_data) = Self::create_isolate(
- refs,
- options.create_params.take(),
- snapshot_options,
+ isolate.set_capture_stack_trace_for_uncaught_exceptions(true, 10);
+ isolate.set_promise_reject_callback(bindings::promise_reject_callback);
+ isolate.set_host_initialize_import_meta_object_callback(
+ bindings::host_initialize_import_meta_object_callback,
);
+ isolate.set_host_import_module_dynamically_callback(
+ bindings::host_import_module_dynamically_callback,
+ );
+ isolate.set_wasm_async_resolve_promise_callback(
+ bindings::wasm_async_resolve_promise_callback,
+ );
+
+ let (global_context, snapshotted_data) = {
+ let scope = &mut v8::HandleScope::new(&mut isolate);
+ let context = v8::Context::new(scope);
+
+ // Get module map data from the snapshot
+ let snapshotted_data = if init_mode == InitMode::FromSnapshot {
+ Some(snapshot_util::get_snapshotted_data(scope, context))
+ } else {
+ None
+ };
+
+ (v8::Global::new(scope, context), snapshotted_data)
+ };
+
// SAFETY: this is first use of `isolate_ptr` so we are sure we're
// not overwriting an existing pointer.
isolate = unsafe {
@@ -575,7 +644,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
scope,
context,
&context_state.borrow().op_ctxs,
- bindings_mode,
+ init_mode,
);
context.set_slot(scope, context_state.clone());
@@ -619,13 +688,13 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
drop(context_scope);
- let mut js_runtime = JsRuntimeImpl {
+ let mut js_runtime = JsRuntime {
inner: InnerIsolateState {
- snapshotting,
+ will_snapshot,
state: ManuallyDropRc(ManuallyDrop::new(state_rc)),
v8_isolate: ManuallyDrop::new(isolate),
},
- bindings_mode,
+ init_mode,
allocations: IsolateAllocations::default(),
event_loop_middlewares,
extensions: options.extensions,
@@ -641,51 +710,6 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
js_runtime
}
- /// Create a new [`v8::OwnedIsolate`] and its global [`v8::Context`] from optional parameters and snapshot.
- fn create_isolate(
- refs: &'static v8::ExternalReferences,
- params: Option<CreateParams>,
- snapshot: SnapshotOptions,
- ) -> (
- v8::OwnedIsolate,
- v8::Global<v8::Context>,
- Option<SnapshottedData>,
- )
- where
- JsRuntimeImpl<FOR_SNAPSHOT>: JsRuntimeInternalTrait,
- {
- let has_snapshot = snapshot.loaded();
- let mut isolate = Self::create_raw_isolate(refs, params, snapshot);
-
- isolate.set_capture_stack_trace_for_uncaught_exceptions(true, 10);
- isolate.set_promise_reject_callback(bindings::promise_reject_callback);
- isolate.set_host_initialize_import_meta_object_callback(
- bindings::host_initialize_import_meta_object_callback,
- );
- isolate.set_host_import_module_dynamically_callback(
- bindings::host_import_module_dynamically_callback,
- );
- isolate.set_wasm_async_resolve_promise_callback(
- bindings::wasm_async_resolve_promise_callback,
- );
-
- let (context, snapshotted_data) = {
- let scope = &mut v8::HandleScope::new(&mut isolate);
- let context = v8::Context::new(scope);
-
- // Get module map data from the snapshot
- let snapshotted_data = if has_snapshot {
- Some(snapshot_util::get_snapshotted_data(scope, context))
- } else {
- None
- };
-
- (v8::Global::new(scope, context), snapshotted_data)
- };
-
- (isolate, context, snapshotted_data)
- }
-
#[cfg(test)]
#[inline]
pub(crate) fn module_map(&self) -> &Rc<RefCell<ModuleMap>> {
@@ -728,7 +752,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
/// Creates a new realm (V8 context) in this JS execution context,
/// pre-initialized with all of the extensions that were passed in
- /// [`RuntimeOptions::extensions`] when the [`JsRuntimeImpl`] was
+ /// [`RuntimeOptions::extensions`] when the [`JsRuntime`] was
/// constructed.
pub fn create_realm(&mut self) -> Result<JsRealm, Error> {
let realm = {
@@ -768,7 +792,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
scope,
context,
&context_state.borrow().op_ctxs,
- self.bindings_mode,
+ self.init_mode,
);
context.set_slot(scope, context_state.clone());
let realm = JsRealmInner::new(
@@ -946,10 +970,10 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
/// Initializes ops of provided Extensions
fn create_opstate(
options: &mut RuntimeOptions,
- snapshot_options: &SnapshotOptions,
+ init_mode: InitMode,
) -> (OpState, Vec<OpDecl>) {
// Add built-in extension
- if snapshot_options.loaded() {
+ if init_mode == InitMode::FromSnapshot {
options
.extensions
.insert(0, crate::ops_builtin::core::init_ops());
@@ -1470,78 +1494,15 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
}
}
-impl JsRuntime {
- /// Only constructor, configuration is done through `options`.
- pub fn new(mut options: RuntimeOptions) -> JsRuntime {
- JsRuntimeImpl::<false>::init_v8(options.v8_platform.take());
-
- let snapshot_options = snapshot_util::SnapshotOptions::new_from(
- options.startup_snapshot.take(),
- false,
- );
-
- JsRuntime(JsRuntimeImpl::<false>::new_runtime(
- options,
- snapshot_options,
- None,
- ))
- }
-
- pub(crate) fn state_from(
- isolate: &v8::Isolate,
- ) -> Rc<RefCell<JsRuntimeState>> {
- let state_ptr = isolate.get_data(STATE_DATA_OFFSET);
- let state_rc =
- // SAFETY: We are sure that it's a valid pointer for whole lifetime of
- // the runtime.
- unsafe { Rc::from_raw(state_ptr as *const RefCell<JsRuntimeState>) };
- let state = state_rc.clone();
- std::mem::forget(state_rc);
- state
- }
-
- pub(crate) fn module_map_from(
- isolate: &v8::Isolate,
- ) -> Rc<RefCell<ModuleMap>> {
- let module_map_ptr = isolate.get_data(MODULE_MAP_DATA_OFFSET);
- let module_map_rc =
- // SAFETY: We are sure that it's a valid pointer for whole lifetime of
- // the runtime.
- unsafe { Rc::from_raw(module_map_ptr as *const RefCell<ModuleMap>) };
- let module_map = module_map_rc.clone();
- std::mem::forget(module_map_rc);
- module_map
- }
-
- pub(crate) fn event_loop_pending_state_from_scope(
- scope: &mut v8::HandleScope,
- ) -> EventLoopPendingState {
- let state = JsRuntime::state_from(scope);
- let module_map = JsRuntime::module_map_from(scope);
- let state = EventLoopPendingState::new(
- scope,
- &mut state.borrow_mut(),
- &module_map.borrow(),
- );
- state
- }
-}
-
impl JsRuntimeForSnapshot {
pub fn new(
mut options: RuntimeOptions,
runtime_snapshot_options: RuntimeSnapshotOptions,
) -> JsRuntimeForSnapshot {
- JsRuntimeImpl::<true>::init_v8(options.v8_platform.take());
-
- let snapshot_options = snapshot_util::SnapshotOptions::new_from(
- options.startup_snapshot.take(),
- true,
- );
-
- JsRuntimeForSnapshot(JsRuntimeImpl::<true>::new_runtime(
+ JsRuntime::init_v8(options.v8_platform.take(), true);
+ JsRuntimeForSnapshot(JsRuntime::new_inner(
options,
- snapshot_options,
+ true,
runtime_snapshot_options.snapshot_module_load_cb,
))
}
@@ -1590,42 +1551,6 @@ impl JsRuntimeForSnapshot {
}
}
-impl JsRuntimeInternalTrait for JsRuntimeImpl<true> {
- fn create_raw_isolate(
- refs: &'static v8::ExternalReferences,
- _params: Option<CreateParams>,
- snapshot: SnapshotOptions,
- ) -> v8::OwnedIsolate {
- snapshot_util::create_snapshot_creator(refs, snapshot)
- }
-}
-
-impl JsRuntimeInternalTrait for JsRuntimeImpl<false> {
- fn create_raw_isolate(
- refs: &'static v8::ExternalReferences,
- params: Option<CreateParams>,
- snapshot: SnapshotOptions,
- ) -> v8::OwnedIsolate {
- let mut params = params
- .unwrap_or_default()
- .embedder_wrapper_type_info_offsets(
- V8_WRAPPER_TYPE_INDEX,
- V8_WRAPPER_OBJECT_INDEX,
- )
- .external_references(&**refs);
-
- if let Some(snapshot) = snapshot.snapshot() {
- params = match snapshot {
- Snapshot::Static(data) => params.snapshot_blob(data),
- Snapshot::JustCreated(data) => params.snapshot_blob(data),
- Snapshot::Boxed(data) => params.snapshot_blob(data),
- };
- }
-
- v8::Isolate::new(params)
- }
-}
-
fn get_stalled_top_level_await_message_for_module(
scope: &mut v8::HandleScope,
module_id: ModuleId,
@@ -1731,7 +1656,7 @@ where
F: FnMut(usize, usize) -> usize,
{
// SAFETY: The data is a pointer to the Rust callback function. It is stored
- // in `JsRuntimeImpl::allocations` and thus is guaranteed to outlive the isolate.
+ // in `JsRuntime::allocations` and thus is guaranteed to outlive the isolate.
let callback = unsafe { &mut *(data as *mut F) };
callback(current_heap_limit, initial_heap_limit)
}
@@ -1800,7 +1725,7 @@ pub(crate) fn exception_to_err_result<T>(
}
// Related to module loading
-impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
+impl JsRuntime {
pub(crate) fn instantiate_module(
&mut self,
id: ModuleId,
@@ -1916,11 +1841,11 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
/// Evaluates an already instantiated ES module.
///
/// Returns a receiver handle that resolves when module promise resolves.
- /// Implementors must manually call [`JsRuntimeImpl::run_event_loop`] to drive
+ /// Implementors must manually call [`JsRuntime::run_event_loop`] to drive
/// module evaluation future.
///
/// `Error` can usually be downcast to `JsError` and should be awaited and
- /// checked after [`JsRuntimeImpl::run_event_loop`] completion.
+ /// checked after [`JsRuntime::run_event_loop`] completion.
///
/// This function panics if module has not been instantiated.
pub fn mod_evaluate(
@@ -1953,7 +1878,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
// Because that promise is created internally by V8, when error occurs during
// module evaluation the promise is rejected, and since the promise has no rejection
// handler it will result in call to `bindings::promise_reject_callback` adding
- // the promise to pending promise rejection table - meaning JsRuntimeImpl will return
+ // the promise to pending promise rejection table - meaning JsRuntime will return
// error on next poll().
//
// This situation is not desirable as we want to manually return error at the
@@ -2362,7 +2287,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
/// The module will be marked as "main", and because of that
/// "import.meta.main" will return true when checked inside that module.
///
- /// User must call [`JsRuntimeImpl::mod_evaluate`] with returned `ModuleId`
+ /// User must call [`JsRuntime::mod_evaluate`] with returned `ModuleId`
/// manually after load is finished.
pub async fn load_main_module(
&mut self,
@@ -2417,7 +2342,7 @@ impl<const FOR_SNAPSHOT: bool> JsRuntimeImpl<FOR_SNAPSHOT> {
/// This method is meant to be used when loading some utility code that
/// might be later imported by the main module (ie. an entry point module).
///
- /// User must call [`JsRuntimeImpl::mod_evaluate`] with returned `ModuleId`
+ /// User must call [`JsRuntime::mod_evaluate`] with returned `ModuleId`
/// manually after load is finished.
pub async fn load_side_module(
&mut self,
@@ -2563,7 +2488,7 @@ pub fn queue_fast_async_op<R: serde::Serialize + 'static>(
) {
let runtime_state = match ctx.runtime_state.upgrade() {
Some(rc_state) => rc_state,
- // at least 1 Rc is held by the JsRuntimeImpl.
+ // at least 1 Rc is held by the JsRuntime.
None => unreachable!(),
};
let get_class = {
@@ -2661,7 +2586,7 @@ pub fn queue_async_op<'s>(
) -> Option<v8::Local<'s, v8::Value>> {
let runtime_state = match ctx.runtime_state.upgrade() {
Some(rc_state) => rc_state,
- // at least 1 Rc is held by the JsRuntimeImpl.
+ // at least 1 Rc is held by the JsRuntime.
None => unreachable!(),
};
@@ -3559,8 +3484,8 @@ pub mod tests {
}
}
- fn create_module<const FOR_SNAPSHOT: bool>(
- runtime: &mut JsRuntimeImpl<FOR_SNAPSHOT>,
+ fn create_module(
+ runtime: &mut JsRuntime,
i: usize,
main: bool,
) -> ModuleInfo {
@@ -3603,10 +3528,7 @@ pub mod tests {
}
}
- fn assert_module_map<const FOR_SNAPSHOT: bool>(
- runtime: &mut JsRuntimeImpl<FOR_SNAPSHOT>,
- modules: &Vec<ModuleInfo>,
- ) {
+ fn assert_module_map(runtime: &mut JsRuntime, modules: &Vec<ModuleInfo>) {
let module_map = runtime.module_map.borrow();
assert_eq!(module_map.handles.len(), modules.len());
assert_eq!(module_map.info.len(), modules.len());
@@ -4679,13 +4601,7 @@ Deno.core.opAsync("op_async_serialize_object_with_numbers_as_keys", {
Ok(String::from("Test"))
}
- deno_core::extension!(
- test_ext,
- ops = [op_test],
- customizer = |ext: &mut deno_core::ExtensionBuilder| {
- ext.force_op_registration();
- },
- );
+ deno_core::extension!(test_ext, ops = [op_test]);
let mut runtime = JsRuntime::new(RuntimeOptions {
startup_snapshot: Some(Snapshot::Boxed(snapshot)),
extensions: vec![test_ext::init_ops()],