mirror of
https://github.com/denoland/deno.git
synced 2024-11-24 15:19:26 -05:00
dd8cbf5e29
Fixes #22995. Fixes #23000. There were a handful of bugs here causing the hang (each with a corresponding minimized test): - We were canceling recv futures when `receiveMessageOnPort` was called, but this caused the "receive loop" in the message port to exit. This was due to the fact that `CancelHandle`s are never reset (i.e., once you `cancel` a `CancelHandle`, it remains cancelled). That meant that after `receieveMessageOnPort` was called, the subsequent calls to `op_message_port_recv_message` would throw `Interrupted` exceptions, and we would exit the loop. The cancellation, however, isn't actually necessary. `op_message_port_recv_message` only borrows the underlying port for long enough to poll the receiver, so the borrow there could never overlap with `op_message_port_recv_message_sync`. - Calling `MessagePort.unref()` caused the "receive loop" in the message port to exit. This was because we were setting `messageEventListenerCount` to 0 on unref. Not only does that break the counter when multiple `MessagePort`s are present in the same thread, but we also exited the "receive loop" whenever the listener count was 0. I assume this was to prevent the recv promise from keeping the event loop open. Instead of this, I chose to just unref the recv promise as needed to control the event loop. - The last bug causing the hang (which was a doozy to debug) ended up being an unfortunate interaction between how we implement our messageport "receive loop" and a pattern found in `npm:piscina` (which angular uses). The gist of it is that piscina uses an atomic wait loop along with `receiveMessageOnPort` in its worker threads, and as the worker is getting started, the following incredibly convoluted series of events occurs: 1. Parent sends a MessagePort `p` to worker 2. Parent sends a message `m` to the port `p` 3. Parent notifies the worker with `Atomics.notify` that a new message is available 4. Worker receives message, adds "message" listener to port `p` 5. Adding the listener triggers `MessagePort.start()` on `p` 6. Receive loop in MessagePort.start receives the message `m`, but then hits an await point and yields (before dispatching the "message" event) 7. Worker continues execution, starts the atomic wait loop, and immediately receives the existing notification from the parent that a message is available 8. Worker attempts to receive the new message `m` with `receiveMessageOnPort`, but this returns `undefined` because the receive loop already took the message in 6 9. Atomic wait loop continues to next iteration, waiting for the next message with `Atomic.wait` 10. `Atomic.wait` blocks the worker thread, which prevents the receive loop from continuing and dispatching the "message" event for the received message 11. The parent waits for the worker to respond to the first message, and waits 12. The thread can't make any more progress, and the whole process hangs The fix I've chosen here (which I don't particularly love, but it works) is to just delay the `MessagePort.start` call until the end of the event loop turn, so that the atomic wait loop receives the message first. This prevents the hang. --- Those were the main issues causing the hang. There ended up being a few other small bugs as well, namely `exit` being emitted multiple times, and not patching up the message port when it's received by `receiveMessageOnPort`.
252 lines
6.5 KiB
Rust
252 lines
6.5 KiB
Rust
// Copyright 2018-2024 the Deno authors. All rights reserved. MIT license.
|
|
|
|
use std::borrow::Cow;
|
|
use std::cell::RefCell;
|
|
use std::rc::Rc;
|
|
|
|
use deno_core::error::type_error;
|
|
use deno_core::error::AnyError;
|
|
use deno_core::op2;
|
|
|
|
use deno_core::CancelFuture;
|
|
use deno_core::CancelHandle;
|
|
use deno_core::DetachedBuffer;
|
|
use deno_core::OpState;
|
|
use deno_core::RcRef;
|
|
use deno_core::Resource;
|
|
use deno_core::ResourceId;
|
|
use futures::future::poll_fn;
|
|
use serde::Deserialize;
|
|
use serde::Serialize;
|
|
use tokio::sync::mpsc::error::TryRecvError;
|
|
use tokio::sync::mpsc::unbounded_channel;
|
|
use tokio::sync::mpsc::UnboundedReceiver;
|
|
use tokio::sync::mpsc::UnboundedSender;
|
|
|
|
pub enum Transferable {
|
|
MessagePort(MessagePort),
|
|
ArrayBuffer(u32),
|
|
}
|
|
|
|
type MessagePortMessage = (DetachedBuffer, Vec<Transferable>);
|
|
|
|
pub struct MessagePort {
|
|
rx: RefCell<UnboundedReceiver<MessagePortMessage>>,
|
|
tx: RefCell<Option<UnboundedSender<MessagePortMessage>>>,
|
|
}
|
|
|
|
impl MessagePort {
|
|
pub fn send(
|
|
&self,
|
|
state: &mut OpState,
|
|
data: JsMessageData,
|
|
) -> Result<(), AnyError> {
|
|
let transferables =
|
|
deserialize_js_transferables(state, data.transferables)?;
|
|
|
|
// Swallow the failed to send error. It means the channel was disentangled,
|
|
// but not cleaned up.
|
|
if let Some(tx) = &*self.tx.borrow() {
|
|
tx.send((data.data, transferables)).ok();
|
|
}
|
|
|
|
Ok(())
|
|
}
|
|
|
|
pub async fn recv(
|
|
&self,
|
|
state: Rc<RefCell<OpState>>,
|
|
) -> Result<Option<JsMessageData>, AnyError> {
|
|
let rx = &self.rx;
|
|
|
|
let maybe_data = poll_fn(|cx| {
|
|
let mut rx = rx.borrow_mut();
|
|
rx.poll_recv(cx)
|
|
})
|
|
.await;
|
|
|
|
if let Some((data, transferables)) = maybe_data {
|
|
let js_transferables =
|
|
serialize_transferables(&mut state.borrow_mut(), transferables);
|
|
return Ok(Some(JsMessageData {
|
|
data,
|
|
transferables: js_transferables,
|
|
}));
|
|
}
|
|
Ok(None)
|
|
}
|
|
|
|
/// This forcefully disconnects the message port from its paired port. This
|
|
/// will wake up the `.recv` on the paired port, which will return `Ok(None)`.
|
|
pub fn disentangle(&self) {
|
|
let mut tx = self.tx.borrow_mut();
|
|
tx.take();
|
|
}
|
|
}
|
|
|
|
pub fn create_entangled_message_port() -> (MessagePort, MessagePort) {
|
|
let (port1_tx, port2_rx) = unbounded_channel::<MessagePortMessage>();
|
|
let (port2_tx, port1_rx) = unbounded_channel::<MessagePortMessage>();
|
|
|
|
let port1 = MessagePort {
|
|
rx: RefCell::new(port1_rx),
|
|
tx: RefCell::new(Some(port1_tx)),
|
|
};
|
|
|
|
let port2 = MessagePort {
|
|
rx: RefCell::new(port2_rx),
|
|
tx: RefCell::new(Some(port2_tx)),
|
|
};
|
|
|
|
(port1, port2)
|
|
}
|
|
|
|
pub struct MessagePortResource {
|
|
port: MessagePort,
|
|
cancel: CancelHandle,
|
|
}
|
|
|
|
impl Resource for MessagePortResource {
|
|
fn name(&self) -> Cow<str> {
|
|
"messagePort".into()
|
|
}
|
|
|
|
fn close(self: Rc<Self>) {
|
|
self.cancel.cancel();
|
|
}
|
|
}
|
|
|
|
#[op2]
|
|
#[serde]
|
|
pub fn op_message_port_create_entangled(
|
|
state: &mut OpState,
|
|
) -> (ResourceId, ResourceId) {
|
|
let (port1, port2) = create_entangled_message_port();
|
|
|
|
let port1_id = state.resource_table.add(MessagePortResource {
|
|
port: port1,
|
|
cancel: CancelHandle::new(),
|
|
});
|
|
|
|
let port2_id = state.resource_table.add(MessagePortResource {
|
|
port: port2,
|
|
cancel: CancelHandle::new(),
|
|
});
|
|
|
|
(port1_id, port2_id)
|
|
}
|
|
|
|
#[derive(Deserialize, Serialize)]
|
|
#[serde(tag = "kind", content = "data", rename_all = "camelCase")]
|
|
pub enum JsTransferable {
|
|
#[serde(rename_all = "camelCase")]
|
|
MessagePort(ResourceId),
|
|
ArrayBuffer(u32),
|
|
}
|
|
|
|
pub fn deserialize_js_transferables(
|
|
state: &mut OpState,
|
|
js_transferables: Vec<JsTransferable>,
|
|
) -> Result<Vec<Transferable>, AnyError> {
|
|
let mut transferables = Vec::with_capacity(js_transferables.len());
|
|
for js_transferable in js_transferables {
|
|
match js_transferable {
|
|
JsTransferable::MessagePort(id) => {
|
|
let resource = state
|
|
.resource_table
|
|
.take::<MessagePortResource>(id)
|
|
.map_err(|_| type_error("Invalid message port transfer"))?;
|
|
resource.cancel.cancel();
|
|
let resource = Rc::try_unwrap(resource)
|
|
.map_err(|_| type_error("Message port is not ready for transfer"))?;
|
|
transferables.push(Transferable::MessagePort(resource.port));
|
|
}
|
|
JsTransferable::ArrayBuffer(id) => {
|
|
transferables.push(Transferable::ArrayBuffer(id));
|
|
}
|
|
}
|
|
}
|
|
Ok(transferables)
|
|
}
|
|
|
|
pub fn serialize_transferables(
|
|
state: &mut OpState,
|
|
transferables: Vec<Transferable>,
|
|
) -> Vec<JsTransferable> {
|
|
let mut js_transferables = Vec::with_capacity(transferables.len());
|
|
for transferable in transferables {
|
|
match transferable {
|
|
Transferable::MessagePort(port) => {
|
|
let rid = state.resource_table.add(MessagePortResource {
|
|
port,
|
|
cancel: CancelHandle::new(),
|
|
});
|
|
js_transferables.push(JsTransferable::MessagePort(rid));
|
|
}
|
|
Transferable::ArrayBuffer(id) => {
|
|
js_transferables.push(JsTransferable::ArrayBuffer(id));
|
|
}
|
|
}
|
|
}
|
|
js_transferables
|
|
}
|
|
|
|
#[derive(Deserialize, Serialize)]
|
|
pub struct JsMessageData {
|
|
pub data: DetachedBuffer,
|
|
pub transferables: Vec<JsTransferable>,
|
|
}
|
|
|
|
#[op2]
|
|
pub fn op_message_port_post_message(
|
|
state: &mut OpState,
|
|
#[smi] rid: ResourceId,
|
|
#[serde] data: JsMessageData,
|
|
) -> Result<(), AnyError> {
|
|
for js_transferable in &data.transferables {
|
|
if let JsTransferable::MessagePort(id) = js_transferable {
|
|
if *id == rid {
|
|
return Err(type_error("Can not transfer self message port"));
|
|
}
|
|
}
|
|
}
|
|
|
|
let resource = state.resource_table.get::<MessagePortResource>(rid)?;
|
|
resource.port.send(state, data)
|
|
}
|
|
|
|
#[op2(async)]
|
|
#[serde]
|
|
pub async fn op_message_port_recv_message(
|
|
state: Rc<RefCell<OpState>>,
|
|
#[smi] rid: ResourceId,
|
|
) -> Result<Option<JsMessageData>, AnyError> {
|
|
let resource = {
|
|
let state = state.borrow();
|
|
match state.resource_table.get::<MessagePortResource>(rid) {
|
|
Ok(resource) => resource,
|
|
Err(_) => return Ok(None),
|
|
}
|
|
};
|
|
let cancel = RcRef::map(resource.clone(), |r| &r.cancel);
|
|
resource.port.recv(state).or_cancel(cancel).await?
|
|
}
|
|
|
|
#[op2]
|
|
#[serde]
|
|
pub fn op_message_port_recv_message_sync(
|
|
state: &mut OpState, // Rc<RefCell<OpState>>,
|
|
#[smi] rid: ResourceId,
|
|
) -> Result<Option<JsMessageData>, AnyError> {
|
|
let resource = state.resource_table.get::<MessagePortResource>(rid)?;
|
|
let mut rx = resource.port.rx.borrow_mut();
|
|
|
|
match rx.try_recv() {
|
|
Ok((d, t)) => Ok(Some(JsMessageData {
|
|
data: d,
|
|
transferables: serialize_transferables(state, t),
|
|
})),
|
|
Err(TryRecvError::Empty) => Ok(None),
|
|
Err(TryRecvError::Disconnected) => Ok(None),
|
|
}
|
|
}
|