LibWeb: Implement ReadableStreamPipeTo according to spec

Our existing implementation of stream piping was extremely ad-hoc. It
did nothing to handle closed/errored streams, and did not read from or
write to streams in a way required by the spec.

This new implementation uses a custom JS::Cell to drive the read/write
loop.
This commit is contained in:
Timothy Flynn 2025-04-09 12:01:51 -04:00 committed by Tim Flynn
commit eb0a51faf0
Notes: github-actions[bot] 2025-04-11 16:11:49 +00:00
33 changed files with 3926 additions and 85 deletions

View file

@ -289,8 +289,383 @@ bool readable_stream_has_default_reader(ReadableStream const& stream)
return false;
}
// https://streams.spec.whatwg.org/#ref-for-in-parallel
class ReadableStreamPipeTo final : public JS::Cell {
GC_CELL(ReadableStreamPipeTo, JS::Cell);
GC_DECLARE_ALLOCATOR(ReadableStreamPipeTo);
public:
void process()
{
if (check_for_error_and_close_states())
return;
auto ready_promise = m_writer->ready();
if (ready_promise && WebIDL::is_promise_fulfilled(*ready_promise)) {
read_chunk();
return;
}
auto when_ready = GC::create_function(m_realm->heap(), [this](JS::Value) -> WebIDL::ExceptionOr<JS::Value> {
read_chunk();
return JS::js_undefined();
});
auto shutdown = GC::create_function(heap(), [this](JS::Value) -> WebIDL::ExceptionOr<JS::Value> {
check_for_error_and_close_states();
return JS::js_undefined();
});
if (ready_promise)
WebIDL::react_to_promise(*ready_promise, when_ready, shutdown);
if (auto promise = m_reader->closed())
WebIDL::react_to_promise(*promise, shutdown, shutdown);
}
void set_abort_signal(GC::Ref<DOM::AbortSignal> signal, DOM::AbortSignal::AbortSignal::AbortAlgorithmID signal_id)
{
m_signal = signal;
m_signal_id = signal_id;
}
// https://streams.spec.whatwg.org/#rs-pipeTo-shutdown-with-action
void shutdown_with_action(GC::Ref<GC::Function<GC::Ref<WebIDL::Promise>()>> action, Optional<JS::Value> original_error = {})
{
// 1. If shuttingDown is true, abort these substeps.
if (m_shutting_down)
return;
// 2. Set shuttingDown to true.
m_shutting_down = true;
auto on_pending_writes_complete = [this, action, original_error = move(original_error)]() mutable {
HTML::TemporaryExecutionContext execution_context { m_realm, HTML::TemporaryExecutionContext::CallbacksEnabled::Yes };
// 4. Let p be the result of performing action.
auto promise = action->function()();
WebIDL::react_to_promise(promise,
// 5. Upon fulfillment of p, finalize, passing along originalError if it was given.
GC::create_function(heap(), [this, original_error = move(original_error)](JS::Value) mutable -> WebIDL::ExceptionOr<JS::Value> {
finish(move(original_error));
return JS::js_undefined();
}),
// 6. Upon rejection of p with reason newError, finalize with newError.
GC::create_function(heap(), [this](JS::Value new_error) -> WebIDL::ExceptionOr<JS::Value> {
finish(new_error);
return JS::js_undefined();
}));
};
// 3. If dest.[[state]] is "writable" and ! WritableStreamCloseQueuedOrInFlight(dest) is false,
if (m_destination->state() == WritableStream::State::Writable && !writable_stream_close_queued_or_in_flight(m_destination)) {
// 1. If any chunks have been read but not yet written, write them to dest.
write_unwritten_chunks();
// 2. Wait until every chunk that has been read has been written (i.e. the corresponding promises have settled).
wait_for_pending_writes_to_complete(move(on_pending_writes_complete));
} else {
on_pending_writes_complete();
}
}
// https://streams.spec.whatwg.org/#rs-pipeTo-shutdown
void shutdown(Optional<JS::Value> error = {})
{
// 1. If shuttingDown is true, abort these substeps.
if (m_shutting_down)
return;
// 2. Set shuttingDown to true.
m_shutting_down = true;
auto on_pending_writes_complete = [this, error = move(error)]() mutable {
HTML::TemporaryExecutionContext execution_context { m_realm, HTML::TemporaryExecutionContext::CallbacksEnabled::Yes };
// 4. Finalize, passing along error if it was given.
finish(move(error));
};
// 3. If dest.[[state]] is "writable" and ! WritableStreamCloseQueuedOrInFlight(dest) is false,
if (m_destination->state() == WritableStream::State::Writable && !writable_stream_close_queued_or_in_flight(m_destination)) {
// 1. If any chunks have been read but not yet written, write them to dest.
write_unwritten_chunks();
// 2. Wait until every chunk that has been read has been written (i.e. the corresponding promises have settled).
wait_for_pending_writes_to_complete(move(on_pending_writes_complete));
} else {
on_pending_writes_complete();
}
}
private:
ReadableStreamPipeTo(
GC::Ref<JS::Realm> realm,
GC::Ref<WebIDL::Promise> promise,
GC::Ref<ReadableStream> source,
GC::Ref<WritableStream> destination,
GC::Ref<ReadableStreamDefaultReader> reader,
GC::Ref<WritableStreamDefaultWriter> writer,
bool prevent_close,
bool prevent_abort,
bool prevent_cancel)
: m_realm(realm)
, m_promise(promise)
, m_source(source)
, m_destination(destination)
, m_reader(reader)
, m_writer(writer)
, m_prevent_close(prevent_close)
, m_prevent_abort(prevent_abort)
, m_prevent_cancel(prevent_cancel)
{
m_reader->set_readable_stream_pipe_to_operation({}, this);
}
virtual void visit_edges(Cell::Visitor& visitor) override
{
Base::visit_edges(visitor);
visitor.visit(m_realm);
visitor.visit(m_promise);
visitor.visit(m_source);
visitor.visit(m_destination);
visitor.visit(m_reader);
visitor.visit(m_writer);
visitor.visit(m_signal);
visitor.visit(m_pending_writes);
visitor.visit(m_unwritten_chunks);
}
void read_chunk()
{
// Shutdown must stop activity: if shuttingDown becomes true, the user agent must not initiate further reads from
// reader, and must only perform writes of already-read chunks, as described below. In particular, the user agent
// must check the below conditions before performing any reads or writes, since they might lead to immediate shutdown.
if (check_for_error_and_close_states())
return;
auto when_ready = GC::create_function(heap(), [this](JS::Value value) -> WebIDL::ExceptionOr<JS::Value> {
auto& vm = this->vm();
VERIFY(value.is_object());
auto& object = value.as_object();
auto done = MUST(JS::iterator_complete(vm, object));
if (done) {
if (!check_for_error_and_close_states())
finish();
} else {
auto chunk = MUST(JS::iterator_value(vm, object));
m_unwritten_chunks.append(chunk);
write_chunk();
process();
}
return JS::js_undefined();
});
auto shutdown = GC::create_function(heap(), [this](JS::Value) -> WebIDL::ExceptionOr<JS::Value> {
check_for_error_and_close_states();
return JS::js_undefined();
});
WebIDL::react_to_promise(m_reader->read(), when_ready, shutdown);
if (auto promise = m_writer->closed())
WebIDL::react_to_promise(*promise, shutdown, shutdown);
}
void write_chunk()
{
// Shutdown must stop activity: if shuttingDown becomes true, the user agent must not initiate further reads from
// reader, and must only perform writes of already-read chunks, as described below. In particular, the user agent
// must check the below conditions before performing any reads or writes, since they might lead to immediate shutdown.
if (!m_shutting_down && check_for_error_and_close_states())
return;
auto promise = m_writer->write(m_unwritten_chunks.take_first());
WebIDL::mark_promise_as_handled(promise);
m_pending_writes.append(promise);
}
void write_unwritten_chunks()
{
while (!m_unwritten_chunks.is_empty())
write_chunk();
}
void wait_for_pending_writes_to_complete(Function<void()> on_complete)
{
auto handler = GC::create_function(heap(), [this, on_complete = move(on_complete)]() {
m_pending_writes.clear();
on_complete();
});
auto success_steps = [handler](Vector<JS::Value> const&) { handler->function()(); };
auto failure_steps = [handler](JS::Value) { handler->function()(); };
WebIDL::wait_for_all(m_realm, m_pending_writes, move(success_steps), move(failure_steps));
}
// https://streams.spec.whatwg.org/#rs-pipeTo-finalize
// We call this `finish` instead of `finalize` to avoid conflicts with GC::Cell::finalize.
void finish(Optional<JS::Value> error = {})
{
// 1. Perform ! WritableStreamDefaultWriterRelease(writer).
writable_stream_default_writer_release(m_writer);
// 2. If reader implements ReadableStreamBYOBReader, perform ! ReadableStreamBYOBReaderRelease(reader).
// 3. Otherwise, perform ! ReadableStreamDefaultReaderRelease(reader).
readable_stream_default_reader_release(m_reader);
// 4. If signal is not undefined, remove abortAlgorithm from signal.
if (m_signal)
m_signal->remove_abort_algorithm(m_signal_id);
// 5. If error was given, reject promise with error.
if (error.has_value()) {
WebIDL::reject_promise(m_realm, m_promise, *error);
}
// 6. Otherwise, resolve promise with undefined.
else {
WebIDL::resolve_promise(m_realm, m_promise, JS::js_undefined());
}
m_reader->set_readable_stream_pipe_to_operation({}, nullptr);
}
bool check_for_error_and_close_states()
{
// Error and close states must be propagated: the following conditions must be applied in order.
return m_shutting_down
|| check_for_forward_errors()
|| check_for_backward_errors()
|| check_for_forward_close()
|| check_for_backward_close();
}
bool check_for_forward_errors()
{
// 1. Errors must be propagated forward: if source.[[state]] is or becomes "errored", then
if (m_source->state() == ReadableStream::State::Errored) {
// 1. If preventAbort is false, shutdown with an action of ! WritableStreamAbort(dest, source.[[storedError]])
// and with source.[[storedError]].
if (!m_prevent_abort) {
auto action = GC::create_function(heap(), [this]() {
return writable_stream_abort(m_destination, m_source->stored_error());
});
shutdown_with_action(action, m_source->stored_error());
}
// 2. Otherwise, shutdown with source.[[storedError]].
else {
shutdown(m_source->stored_error());
}
}
return m_shutting_down;
}
bool check_for_backward_errors()
{
// 2. Errors must be propagated backward: if dest.[[state]] is or becomes "errored", then
if (m_destination->state() == WritableStream::State::Errored) {
// 1. If preventCancel is false, shutdown with an action of ! ReadableStreamCancel(source, dest.[[storedError]])
// and with dest.[[storedError]].
if (!m_prevent_cancel) {
auto action = GC::create_function(heap(), [this]() {
return readable_stream_cancel(m_source, m_destination->stored_error());
});
shutdown_with_action(action, m_destination->stored_error());
}
// 2. Otherwise, shutdown with dest.[[storedError]].
else {
shutdown(m_destination->stored_error());
}
}
return m_shutting_down;
}
bool check_for_forward_close()
{
// 3. Closing must be propagated forward: if source.[[state]] is or becomes "closed", then
if (m_source->state() == ReadableStream::State::Closed) {
// 1. If preventClose is false, shutdown with an action of ! WritableStreamDefaultWriterCloseWithErrorPropagation(writer).
if (!m_prevent_close) {
auto action = GC::create_function(heap(), [this]() {
return writable_stream_default_writer_close_with_error_propagation(m_writer);
});
shutdown_with_action(action);
}
// 2. Otherwise, shutdown.
else {
shutdown();
}
}
return m_shutting_down;
}
bool check_for_backward_close()
{
// 4. Closing must be propagated backward: if ! WritableStreamCloseQueuedOrInFlight(dest) is true or dest.[[state]] is "closed", then
if (writable_stream_close_queued_or_in_flight(m_destination) || m_destination->state() == WritableStream::State::Closed) {
// 1. Assert: no chunks have been read or written.
// 2. Let destClosed be a new TypeError.
auto destination_closed = JS::TypeError::create(m_realm, "Destination stream was closed during piping operation"sv);
// 3. If preventCancel is false, shutdown with an action of ! ReadableStreamCancel(source, destClosed) and with destClosed.
if (!m_prevent_cancel) {
auto action = GC::create_function(heap(), [this, destination_closed]() {
return readable_stream_cancel(m_source, destination_closed);
});
shutdown_with_action(action, destination_closed);
}
// 4. Otherwise, shutdown with destClosed.
else {
shutdown(destination_closed);
}
}
return m_shutting_down;
}
GC::Ref<JS::Realm> m_realm;
GC::Ref<WebIDL::Promise> m_promise;
GC::Ref<ReadableStream> m_source;
GC::Ref<WritableStream> m_destination;
GC::Ref<ReadableStreamDefaultReader> m_reader;
GC::Ref<WritableStreamDefaultWriter> m_writer;
GC::Ptr<DOM::AbortSignal> m_signal;
DOM::AbortSignal::AbortAlgorithmID m_signal_id { 0 };
Vector<GC::Ref<WebIDL::Promise>> m_pending_writes;
Vector<JS::Value, 1> m_unwritten_chunks;
bool m_prevent_close { false };
bool m_prevent_abort { false };
bool m_prevent_cancel { false };
bool m_shutting_down { false };
};
GC_DEFINE_ALLOCATOR(ReadableStreamPipeTo);
// https://streams.spec.whatwg.org/#readable-stream-pipe-to
GC::Ref<WebIDL::Promise> readable_stream_pipe_to(ReadableStream& source, WritableStream& dest, bool, bool, bool, GC::Ptr<DOM::AbortSignal> signal)
GC::Ref<WebIDL::Promise> readable_stream_pipe_to(ReadableStream& source, WritableStream& dest, bool prevent_close, bool prevent_abort, bool prevent_cancel, GC::Ptr<DOM::AbortSignal> signal)
{
auto& realm = source.realm();
@ -300,7 +675,6 @@ GC::Ref<WebIDL::Promise> readable_stream_pipe_to(ReadableStream& source, Writabl
// 4. If signal was not given, let signal be undefined.
// 5. Assert: either signal is undefined, or signal implements AbortSignal.
(void)signal;
// 6. Assert: ! IsReadableStreamLocked(source) is false.
VERIFY(!is_readable_stream_locked(source));
@ -322,57 +696,81 @@ GC::Ref<WebIDL::Promise> readable_stream_pipe_to(ReadableStream& source, Writabl
// 11. Set source.[[disturbed]] to true.
source.set_disturbed(true);
// FIXME: 12. Let shuttingDown be false.
// 12. Let shuttingDown be false.
// NOTE: This is internal to the ReadableStreamPipeTo class.
// 13. Let promise be a new promise.
auto promise = WebIDL::create_promise(realm);
// FIXME 14. If signal is not undefined,
// 1. Let abortAlgorithm be the following steps:
// 1. Let error be signals abort reason.
// 2. Let actions be an empty ordered set.
// 3. If preventAbort is false, append the following action to actions:
// 1. If dest.[[state]] is "writable", return ! WritableStreamAbort(dest, error).
// 2. Otherwise, return a promise resolved with undefined.
// 4. If preventCancel is false, append the following action to actions:
// 1. If source.[[state]] is "readable", return ! ReadableStreamCancel(source, error).
// 2. Otherwise, return a promise resolved with undefined.
// 5. Shutdown with an action consisting of getting a promise to wait for all of the actions in actions, and with error.
// 2. If signal is aborted, perform abortAlgorithm and return promise.
// 3. Add abortAlgorithm to signal.
auto operation = realm.heap().allocate<ReadableStreamPipeTo>(realm, promise, source, dest, reader, writer, prevent_close, prevent_abort, prevent_cancel);
// 15. In parallel but not really; see #905, using reader and writer, read all chunks from source and write them to
// dest. Due to the locking provided by the reader and writer, the exact manner in which this happens is not
// observable to author code, and so there is flexibility in how this is done. The following constraints apply
// regardless of the exact algorithm used:
// - Public API must not be used: while reading or writing, or performing any of the operations below, the
// JavaScript-modifiable reader, writer, and stream APIs (i.e. methods on the appropriate prototypes) must not
// be used. Instead, the streams must be manipulated directly.
// 14. If signal is not undefined,
if (signal) {
// 1. Let abortAlgorithm be the following steps:
auto abort_algorithm = [&realm, operation, source = GC::Ref { source }, dest = GC::Ref { dest }, prevent_abort, prevent_cancel, signal]() {
HTML::TemporaryExecutionContext execution_context { realm, HTML::TemporaryExecutionContext::CallbacksEnabled::Yes };
// FIXME: Currently a naive implementation that uses ReadableStreamDefaultReader::read_all_chunks() to read all chunks
// from the source and then through the callback success_steps writes those chunks to the destination.
auto chunk_steps = GC::create_function(realm.heap(), [&realm, writer](JS::Value chunk) {
auto promise = writable_stream_default_writer_write(writer, chunk);
WebIDL::resolve_promise(realm, promise, JS::js_undefined());
});
// 1. Let error be signals abort reason.
auto error = signal->reason();
auto success_steps = GC::create_function(realm.heap(), [promise, &realm, reader, writer]() {
// Make sure we close the acquired writer.
WebIDL::resolve_promise(realm, writable_stream_default_writer_close(*writer), JS::js_undefined());
readable_stream_default_reader_release(*reader);
// 2. Let actions be an empty ordered set.
GC::Ptr<GC::Function<GC::Ref<WebIDL::Promise>()>> abort_destination;
GC::Ptr<GC::Function<GC::Ref<WebIDL::Promise>()>> cancel_source;
WebIDL::resolve_promise(realm, promise, JS::js_undefined());
});
// 3. If preventAbort is false, append the following action to actions:
if (!prevent_abort) {
abort_destination = GC::create_function(realm.heap(), [&realm, dest, error]() {
// 1. If dest.[[state]] is "writable", return ! WritableStreamAbort(dest, error).
if (dest->state() == WritableStream::State::Writable)
return writable_stream_abort(dest, error);
auto failure_steps = GC::create_function(realm.heap(), [promise, &realm, reader, writer](JS::Value error) {
// Make sure we close the acquired writer.
WebIDL::resolve_promise(realm, writable_stream_default_writer_close(*writer), JS::js_undefined());
readable_stream_default_reader_release(*reader);
// 2. Otherwise, return a promise resolved with undefined.
return WebIDL::create_resolved_promise(realm, JS::js_undefined());
});
}
WebIDL::reject_promise(realm, promise, error);
});
// 4. If preventCancel is false, append the following action action to actions:
if (!prevent_cancel) {
cancel_source = GC::create_function(realm.heap(), [&realm, source, error]() {
// 1. If source.[[state]] is "readable", return ! ReadableStreamCancel(source, error).
if (source->state() == ReadableStream::State::Readable)
return readable_stream_cancel(source, error);
reader->read_all_chunks(chunk_steps, success_steps, failure_steps);
// 2. Otherwise, return a promise resolved with undefined.
return WebIDL::create_resolved_promise(realm, JS::js_undefined());
});
}
// 5. Shutdown with an action consisting of getting a promise to wait for all of the actions in actions, and with error.
auto action = GC::create_function(realm.heap(), [&realm, abort_destination, cancel_source]() {
GC::RootVector<GC::Ref<WebIDL::Promise>> actions(realm.heap());
if (abort_destination)
actions.append(abort_destination->function()());
if (cancel_source)
actions.append(cancel_source->function()());
return WebIDL::get_promise_for_wait_for_all(realm, actions);
});
operation->shutdown_with_action(action, error);
};
// 2. If signal is aborted, perform abortAlgorithm and return promise.
if (signal->aborted()) {
abort_algorithm();
return promise;
}
// 3. Add abortAlgorithm to signal.
auto signal_id = signal->add_abort_algorithm(move(abort_algorithm));
operation->set_abort_signal(*signal, signal_id.value());
}
// 15. In parallel (but not really; see #905), using reader and writer, read all chunks from source and write them
// to dest. Due to the locking provided by the reader and writer, the exact manner in which this happens is not
// observable to author code, and so there is flexibility in how this is done.
operation->process();
// 16. Return promise.
return promise;