From c8f63779e20b3bf786cad5d4bd8ed29782b7b1d7 Mon Sep 17 00:00:00 2001 From: Kenton Varda Date: Fri, 29 Nov 2024 15:17:48 -0600 Subject: [PATCH 1/5] Refactor server.c++: Make `lookupService()` return `Own`. Currently, `ServiceDesignator` can only resolve to one of a fixed set of objects that already exist, but we want to make it a little bit more dynamic in later commits. --- src/workerd/server/server.c++ | 92 +++++++++++++++++++++++------------ src/workerd/server/server.h | 9 ++-- 2 files changed, 68 insertions(+), 33 deletions(-) diff --git a/src/workerd/server/server.c++ b/src/workerd/server/server.c++ index 75129683e19..8c69541ee07 100644 --- a/src/workerd/server/server.c++ +++ b/src/workerd/server/server.c++ @@ -149,6 +149,11 @@ static kj::String escapeJsonString(kj::StringPtr text) { class ServerResolveObserver final: public jsg::ResolveObserver {}; const ServerResolveObserver serverResolveObserver; +template +static inline kj::Own fakeOwn(T& ref) { + return kj::Own(&ref, kj::NullDisposer::instance); +} + } // namespace // ======================================================================================= @@ -168,8 +173,6 @@ Server::Server(kj::Filesystem& fs, memoryCacheProvider(kj::heap(timer)), tasks(*this) {} -Server::~Server() noexcept(false) {} - struct Server::GlobalContext { jsg::V8System& v8System; capnp::ByteStreamFactory byteStreamFactory; @@ -196,6 +199,11 @@ class Server::Service { // Cross-links this service with other services. Must be called once before `startRequest()`. virtual void link() {} + // Drops any cross-links created during link(). This called just before all the services are + // destroyed. An `Own` cannot be destroyed unless the object it points to still exists, so + // we must clear all the `Own`s before we can actually destroy the `Service`s. + virtual void unlink() {} + // Begin an incoming request. Returns a `WorkerInterface` object that will be used for one // request then discarded. virtual kj::Own startRequest(IoChannelFactory::SubrequestMetadata metadata) = 0; @@ -204,6 +212,15 @@ class Server::Service { virtual bool hasHandler(kj::StringPtr handlerName) = 0; }; +Server::~Server() noexcept { + // This destructor is explicitly `noexcept` because if one of the `unlink()`s throws then we'd + // have a hard time avoiding a segfault later... and we're shutting down the server anyway so + // whatever, better to crash. + for (auto& service: services) { + service.value->unlink(); + } +} + // ======================================================================================= kj::Own Server::makeTlsContext(config::TlsOptions::Reader conf) { @@ -1572,12 +1589,12 @@ class Server::WorkerService final: public Service, // I/O channels, delivered when link() is called. struct LinkedIoChannels { - kj::Array subrequest; + kj::Array> subrequest; kj::Array> actor; // null = configuration error - kj::Maybe cache; + kj::Maybe> cache; kj::Maybe> actorStorage; AlarmScheduler& alarmScheduler; - kj::Array tails; + kj::Array> tails; }; using LinkCallback = kj::Function; using AbortActorsCallback = kj::Function; @@ -1624,6 +1641,14 @@ class Server::WorkerService final: public Service, ioChannels = callback(*this); } + void unlink() override { + // Need to tear down all actors before tearing down `ioChannels.actorStorage`. + actorNamespaces.clear(); + + // OK, now we can unlink. + ioChannels = {}; + } + kj::Maybe getActorNamespace(kj::StringPtr name) { KJ_IF_SOME(a, actorNamespaces.find(name)) { return *a; @@ -2325,7 +2350,7 @@ class Server::WorkerService final: public Service, kj::Own getCache() override { auto& channels = KJ_REQUIRE_NONNULL(ioChannels.tryGet(), "link() has not been called"); - auto& cache = JSG_REQUIRE_NONNULL(channels.cache, Error, "No Cache was configured"); + auto& cache = *JSG_REQUIRE_NONNULL(channels.cache, Error, "No Cache was configured"); return kj::heap(cache, threadContext.getHeaderIds().cfCacheNamespace); } @@ -3149,20 +3174,26 @@ kj::Own Server::makeWorker(kj::StringPtr name, WorkerService& workerService) mutable { WorkerService::LinkedIoChannels result{.alarmScheduler = *alarmScheduler}; - auto services = kj::heapArrayBuilder( + auto services = kj::heapArrayBuilder>( subrequestChannels.size() + IoContext::SPECIAL_SUBREQUEST_CHANNEL_COUNT); - Service& globalService = + kj::Own globalService = lookupService(conf.getGlobalOutbound(), kj::str("Worker \"", name, "\"'s globalOutbound")); // Bind both "next" and "null" to the global outbound. (The difference between these is a // legacy artifact that no one should be depending on.) + // + // We set up one as a fakeOwn() alias of the other. Awkwardly, it's important that real Own + // come first in the list, before the fakeOwn, because they'll be destroyed in reverse order, + // and the fakeOwn must be destroyed before the real one so that it's not dangling at time of + // destruction. static_assert(IoContext::SPECIAL_SUBREQUEST_CHANNEL_COUNT == 2); - services.add(&globalService); - services.add(&globalService); + auto globalService2 = fakeOwn(*globalService); + services.add(kj::mv(globalService)); + services.add(kj::mv(globalService2)); for (auto& channel: subrequestChannels) { - services.add(&lookupService(channel.designator, kj::mv(channel.errorContext))); + services.add(lookupService(channel.designator, kj::mv(channel.errorContext))); } result.subrequest = services.finish(); @@ -3230,7 +3261,7 @@ kj::Own Server::makeWorker(kj::StringPtr name, } result.tails = KJ_MAP(tail, conf.getTails()) { - return &lookupService(tail, kj::str("Worker \"", name, "\"'s tails")); + return lookupService(tail, kj::str("Worker \"", name, "\"'s tails")); }; return result; @@ -3276,35 +3307,35 @@ void Server::taskFailed(kj::Exception&& exception) { fatalFulfiller->reject(kj::mv(exception)); } -Server::Service& Server::lookupService( +kj::Own Server::lookupService( config::ServiceDesignator::Reader designator, kj::String errorContext) { kj::StringPtr targetName = designator.getName(); Service* service = KJ_UNWRAP_OR(services.find(targetName), { reportConfigError(kj::str(errorContext, " refers to a service \"", targetName, "\", but no such service is defined.")); - return *invalidConfigServiceSingleton; + return fakeOwn(*invalidConfigServiceSingleton); }); if (designator.hasEntrypoint()) { kj::StringPtr entrypointName = designator.getEntrypoint(); if (WorkerService* worker = dynamic_cast(service)) { KJ_IF_SOME(ep, worker->getEntrypoint(entrypointName)) { - return ep; + return fakeOwn(ep); } else { reportConfigError(kj::str(errorContext, " refers to service \"", targetName, "\" with a named entrypoint \"", entrypointName, "\", but \"", targetName, "\" has no such named entrypoint.")); - return *invalidConfigServiceSingleton; + return fakeOwn(*invalidConfigServiceSingleton); } } else { reportConfigError(kj::str(errorContext, " refers to service \"", targetName, "\" with a named entrypoint \"", entrypointName, "\", but \"", targetName, "\" is not a Worker, so does not have any " "named entrypoints.")); - return *invalidConfigServiceSingleton; + return fakeOwn(*invalidConfigServiceSingleton); } } else { - return *service; + return fakeOwn(*service); } } @@ -3314,7 +3345,7 @@ class Server::HttpListener final: public kj::Refcounted { public: HttpListener(Server& owner, kj::Own listener, - Service& service, + kj::Own service, kj::StringPtr physicalProtocol, kj::Own rewriter, kj::HttpHeaderTable& headerTable, @@ -3322,7 +3353,7 @@ class Server::HttpListener final: public kj::Refcounted { capnp::HttpOverCapnpFactory& httpOverCapnpFactory) : owner(owner), listener(kj::mv(listener)), - service(service), + service(kj::mv(service)), headerTable(headerTable), timer(timer), httpOverCapnpFactory(httpOverCapnpFactory), @@ -3390,7 +3421,7 @@ class Server::HttpListener final: public kj::Refcounted { private: Server& owner; kj::Own listener; - Service& service; + kj::Own service; kj::HttpHeaderTable& headerTable; kj::Timer& timer; capnp::HttpOverCapnpFactory& httpOverCapnpFactory; @@ -3420,7 +3451,7 @@ class Server::HttpListener final: public kj::Refcounted { // configured, which hints that this service trusts the client to provide the cf blob.) context.initResults(capnp::MessageSize{4, 1}) - .setDispatcher(kj::heap(parent, parent.service.startRequest({}))); + .setDispatcher(kj::heap(parent, parent.service->startRequest({}))); return kj::READY_NOW; } @@ -3555,11 +3586,11 @@ class Server::HttpListener final: public kj::Refcounted { auto rewrite = KJ_UNWRAP_OR(parent.rewriter->rewriteIncomingRequest( url, parent.physicalProtocol, headers, metadata.cfBlobJson), { co_return co_await response.sendError(400, "Bad Request", parent.headerTable); }); - auto worker = parent.service.startRequest(kj::mv(metadata)); + auto worker = parent.service->startRequest(kj::mv(metadata)); co_return co_await worker->request( method, url, *rewrite.headers, requestBody, *wrappedResponse); } else { - auto worker = parent.service.startRequest(kj::mv(metadata)); + auto worker = parent.service->startRequest(kj::mv(metadata)); co_return co_await worker->request(method, url, headers, requestBody, *wrappedResponse); } } @@ -3596,11 +3627,12 @@ class Server::HttpListener final: public kj::Refcounted { }; kj::Promise Server::listenHttp(kj::Own listener, - Service& service, + kj::Own service, kj::StringPtr physicalProtocol, kj::Own rewriter) { - auto obj = kj::refcounted(*this, kj::mv(listener), service, physicalProtocol, - kj::mv(rewriter), globalContext->headerTable, timer, globalContext->httpOverCapnpFactory); + auto obj = + kj::refcounted(*this, kj::mv(listener), kj::mv(service), physicalProtocol, + kj::mv(rewriter), globalContext->headerTable, timer, globalContext->httpOverCapnpFactory); co_return co_await obj->run(); } @@ -3852,7 +3884,7 @@ kj::Promise Server::listenOnSockets(config::Config::Reader config, kj::String ownAddrStr; kj::Maybe> listenerOverride; - Service& service = lookupService(sock.getService(), kj::str("Socket \"", name, "\"")); + kj::Own service = lookupService(sock.getService(), kj::str("Socket \"", name, "\"")); KJ_IF_SOME(override, socketOverrides.findEntry(name)) { KJ_SWITCH_ONEOF(override.value) { @@ -3924,7 +3956,7 @@ kj::Promise Server::listenOnSockets(config::Config::Reader config, auto rewriter = kj::heap(httpOptions, headerTableBuilder); auto handle = kj::coCapture( - [this, &service, rewriter = kj::mv(rewriter), physicalProtocol, name]( + [this, service = kj::mv(service), rewriter = kj::mv(rewriter), physicalProtocol, name]( kj::Promise> promise) mutable -> kj::Promise { TRACE_EVENT("workerd", "setup listenHttp"); auto listener = co_await promise; @@ -3937,7 +3969,7 @@ kj::Promise Server::listenOnSockets(config::Config::Reader config, KJ_LOG(ERROR, e); } } - co_await listenHttp(kj::mv(listener), service, physicalProtocol, kj::mv(rewriter)); + co_await listenHttp(kj::mv(listener), kj::mv(service), physicalProtocol, kj::mv(rewriter)); }); tasks.add(handle(kj::mv(listener)).exclusiveJoin(forkedDrainWhen.addBranch())); } diff --git a/src/workerd/server/server.h b/src/workerd/server/server.h index d329f741c41..ef18a6fef4b 100644 --- a/src/workerd/server/server.h +++ b/src/workerd/server/server.h @@ -41,7 +41,7 @@ class Server final: private kj::TaskSet::ErrorHandler { kj::EntropySource& entropySource, Worker::ConsoleMode consoleMode, kj::Function reportConfigError); - ~Server() noexcept(false); + ~Server() noexcept; // Permit experimental features to be used. These features may break backwards compatibility // in the future. @@ -216,10 +216,13 @@ class Server final: private kj::TaskSet::ErrorHandler { void abortAllActors(); // Can only be called in the link stage. - Service& lookupService(config::ServiceDesignator::Reader designator, kj::String errorContext); + // + // May return a new object or may return a fake-own around a long-lived object. + kj::Own lookupService( + config::ServiceDesignator::Reader designator, kj::String errorContext); kj::Promise listenHttp(kj::Own listener, - Service& service, + kj::Own service, kj::StringPtr physicalProtocol, kj::Own rewriter); From f16bdd4e458dbd7125fe5f0ec973ddf4f1db5d82 Mon Sep 17 00:00:00 2001 From: Kenton Varda Date: Fri, 29 Nov 2024 15:20:40 -0600 Subject: [PATCH 2/5] Refactor server.c++: Construct EntrypointService on-demand. This will allow us to extend it to support connection metadata, which might be different for each binding pointing to the same entrypoint. --- src/workerd/server/server.c++ | 30 +++++++++++++----------------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/src/workerd/server/server.c++ b/src/workerd/server/server.c++ index 8c69541ee07..873d6950ac4 100644 --- a/src/workerd/server/server.c++ +++ b/src/workerd/server/server.c++ @@ -1602,7 +1602,7 @@ class Server::WorkerService final: public Service, WorkerService(ThreadContext& threadContext, kj::Own worker, kj::Maybe> defaultEntrypointHandlers, - kj::HashMap> namedEntrypointsParam, + kj::HashMap> namedEntrypoints, const kj::HashMap& actorClasses, LinkCallback linkCallback, AbortActorsCallback abortActorsCallback) @@ -1610,15 +1610,10 @@ class Server::WorkerService final: public Service, ioChannels(kj::mv(linkCallback)), worker(kj::mv(worker)), defaultEntrypointHandlers(kj::mv(defaultEntrypointHandlers)), + namedEntrypoints(kj::mv(namedEntrypoints)), waitUntilTasks(*this), abortActorsCallback(kj::mv(abortActorsCallback)) { - namedEntrypoints.reserve(namedEntrypointsParam.size()); - for (auto& ep: namedEntrypointsParam) { - kj::StringPtr epPtr = ep.key; - namedEntrypoints.insert(kj::mv(ep.key), EntrypointService(*this, epPtr, kj::mv(ep.value))); - } - actorNamespaces.reserve(actorClasses.size()); for (auto& entry: actorClasses) { auto ns = @@ -1627,8 +1622,9 @@ class Server::WorkerService final: public Service, } } - kj::Maybe getEntrypoint(kj::StringPtr name) { - return namedEntrypoints.find(name); + kj::Maybe> getEntrypoint(kj::StringPtr name) { + auto& entry = KJ_UNWRAP_OR_RETURN(namedEntrypoints.findEntry(name), kj::none); + return kj::heap(*this, entry.key, entry.value); } kj::Array getEntrypointNames() { @@ -2221,10 +2217,10 @@ class Server::WorkerService final: public Service, class EntrypointService final: public Service { public: EntrypointService( - WorkerService& worker, kj::StringPtr entrypoint, kj::HashSet handlers) + WorkerService& worker, kj::StringPtr entrypoint, kj::HashSet& handlers) : worker(worker), entrypoint(entrypoint), - handlers(kj::mv(handlers)) {} + handlers(handlers) {} kj::Own startRequest(IoChannelFactory::SubrequestMetadata metadata) override { return worker.startRequest(kj::mv(metadata), entrypoint); @@ -2237,7 +2233,7 @@ class Server::WorkerService final: public Service, private: WorkerService& worker; kj::StringPtr entrypoint; - kj::HashSet handlers; + kj::HashSet& handlers; }; ThreadContext& threadContext; @@ -2247,7 +2243,7 @@ class Server::WorkerService final: public Service, kj::Own worker; kj::Maybe> defaultEntrypointHandlers; - kj::HashMap namedEntrypoints; + kj::HashMap> namedEntrypoints; kj::HashMap> actorNamespaces; kj::TaskSet waitUntilTasks; AbortActorsCallback abortActorsCallback; @@ -3320,7 +3316,7 @@ kj::Own Server::lookupService( kj::StringPtr entrypointName = designator.getEntrypoint(); if (WorkerService* worker = dynamic_cast(service)) { KJ_IF_SOME(ep, worker->getEntrypoint(entrypointName)) { - return fakeOwn(ep); + return kj::mv(ep); } else { reportConfigError(kj::str(errorContext, " refers to service \"", targetName, "\" with a named entrypoint \"", entrypointName, "\", but \"", targetName, @@ -4076,9 +4072,9 @@ kj::Promise Server::test(jsg::V8System& v8System, if (WorkerService* worker = dynamic_cast(service.value.get())) { for (auto& name: worker->getEntrypointNames()) { if (entrypointGlob.matches(name)) { - Service& ep = KJ_ASSERT_NONNULL(worker->getEntrypoint(name)); - if (ep.hasHandler("test"_kj)) { - co_await doTest(ep, kj::str(service.key, ':', name)); + kj::Own ep = KJ_ASSERT_NONNULL(worker->getEntrypoint(name)); + if (ep->hasHandler("test"_kj)) { + co_await doTest(*ep, kj::str(service.key, ':', name)); } } } From 3837d4ef00472ca6266788cdab86dee7443443cf Mon Sep 17 00:00:00 2001 From: Kenton Varda Date: Fri, 29 Nov 2024 16:49:50 -0600 Subject: [PATCH 3/5] Add `Frankenvalue` type: A stitched-together JS value. See comments for description. --- src/workerd/io/BUILD.bazel | 12 +++ src/workerd/io/frankenvalue-test.c++ | 56 ++++++++++ src/workerd/io/frankenvalue.c++ | 148 +++++++++++++++++++++++++++ src/workerd/io/frankenvalue.capnp | 37 +++++++ src/workerd/io/frankenvalue.h | 73 +++++++++++++ src/workerd/jsg/jsg-test.h | 22 ++++ 6 files changed, 348 insertions(+) create mode 100644 src/workerd/io/frankenvalue-test.c++ create mode 100644 src/workerd/io/frankenvalue.c++ create mode 100644 src/workerd/io/frankenvalue.capnp create mode 100644 src/workerd/io/frankenvalue.h diff --git a/src/workerd/io/BUILD.bazel b/src/workerd/io/BUILD.bazel index 386bb8a7b74..e7e5f43b876 100644 --- a/src/workerd/io/BUILD.bazel +++ b/src/workerd/io/BUILD.bazel @@ -46,6 +46,7 @@ wd_cc_library( srcs = [ "compatibility-date.c++", "features.c++", + "frankenvalue.c++", "hibernation-manager.c++", "io-context.c++", "io-own.c++", @@ -58,6 +59,7 @@ wd_cc_library( hdrs = [ "compatibility-date.h", "features.h", + "frankenvalue.h", "hibernation-manager.h", "io-channels.h", "io-context.h", @@ -87,6 +89,7 @@ wd_cc_library( ":actor-id", ":actor-storage_capnp", ":cdp_capnp", + ":frankenvalue_capnp", ":io-gate", ":io-helpers", ":limit-enforcer", @@ -273,6 +276,8 @@ wd_capnp_library(src = "compatibility-date.capnp") wd_capnp_library(src = "features.capnp") +wd_capnp_library(src = "frankenvalue.capnp") + kj_test( src = "io-gate-test.c++", deps = [ @@ -328,3 +333,10 @@ kj_test( "//src/workerd/util:thread-scopes", ], ) + +kj_test( + src = "frankenvalue-test.c++", + deps = [ + ":io", + ], +) diff --git a/src/workerd/io/frankenvalue-test.c++ b/src/workerd/io/frankenvalue-test.c++ new file mode 100644 index 00000000000..38ce219341f --- /dev/null +++ b/src/workerd/io/frankenvalue-test.c++ @@ -0,0 +1,56 @@ +#include "frankenvalue.h" + +#include + +#include +#include +#include + +namespace workerd { +namespace { + +jsg::V8System v8System; +class ContextGlobalObject: public jsg::Object, public jsg::ContextGlobal {}; + +struct TestContext: public ContextGlobalObject { + JSG_RESOURCE_TYPE(TestContext) {} +}; +JSG_DECLARE_ISOLATE_TYPE(TestIsolate, TestContext); + +KJ_TEST("Frankenvalue") { + jsg::test::Evaluator e(v8System); + + e.run([&](jsg::Lock& js) { + // Create a value based on JSON. + Frankenvalue value = Frankenvalue::fromJson(kj::str(R"({"baz": 321, "qux": "xyz"})"_kj)); + + // prop1 is empty. + value.setProperty(kj::str("prop1"), {}); + + // prop2 is a V8-serialized value. + value.setProperty(kj::str("prop2"), ({ + auto obj = js.obj(); + obj.set(js, "foo", js.num(123)); + obj.set(js, "bar", js.str("abc"_kj)); + Frankenvalue::fromJs(js, obj); + })); + + // Round trip through capnp. + { + capnp::MallocMessageBuilder message; + auto builder = message.initRoot(); + value.toCapnp(builder); + value = Frankenvalue::fromCapnp(builder.asReader()); + } + + // Use clone(). + value = value.clone(); + + // Back to JS, then JSON, then check that nothing was lost. + KJ_EXPECT(js.serializeJson(value.toJs(js)) == + R"({"baz":321,"qux":"xyz","prop1":{},"prop2":{"foo":123,"bar":"abc"}})"_kj); + }); +} + +} // namespace +} // namespace workerd diff --git a/src/workerd/io/frankenvalue.c++ b/src/workerd/io/frankenvalue.c++ new file mode 100644 index 00000000000..24733d6cadf --- /dev/null +++ b/src/workerd/io/frankenvalue.c++ @@ -0,0 +1,148 @@ +#include "frankenvalue.h" + +#include + +namespace workerd { + +Frankenvalue Frankenvalue::clone() const { + Frankenvalue result; + + KJ_SWITCH_ONEOF(value) { + KJ_CASE_ONEOF(_, EmptyObject) { + result.value = EmptyObject(); + } + KJ_CASE_ONEOF(json, Json) { + result.value = Json{kj::str(json.json)}; + } + KJ_CASE_ONEOF(v8Serialized, V8Serialized) { + result.value = V8Serialized{kj::heapArray(v8Serialized.data.asPtr())}; + } + } + + if (properties.size() > 0) { + result.properties.reserve(properties.size()); + + for (auto& property: properties) { + result.properties.add(Property{ + .name = kj::str(property.name), + .value = property.value.clone(), + }); + } + } + + return result; +} + +void Frankenvalue::toCapnp(rpc::Frankenvalue::Builder builder) const { + KJ_SWITCH_ONEOF(value) { + KJ_CASE_ONEOF(_, EmptyObject) { + builder.setEmptyObject(); + } + KJ_CASE_ONEOF(json, Json) { + builder.setJson(json.json); + } + KJ_CASE_ONEOF(v8Serialized, V8Serialized) { + builder.setV8Serialized(v8Serialized.data); + } + } + + if (properties.size() > 0) { + auto listBuilder = builder.initProperties(properties.size()); + + for (auto i: kj::indices(properties)) { + auto elemBuilder = listBuilder[i]; + elemBuilder.setName(properties[i].name); + properties[i].value.toCapnp(elemBuilder); + } + } +} + +Frankenvalue Frankenvalue::fromCapnp(rpc::Frankenvalue::Reader reader) { + Frankenvalue result; + + switch (reader.which()) { + case rpc::Frankenvalue::EMPTY_OBJECT: + result.value = EmptyObject(); + break; + case rpc::Frankenvalue::JSON: + result.value = Json{kj::str(reader.getJson())}; + break; + case rpc::Frankenvalue::V8_SERIALIZED: + result.value = V8Serialized{kj::heapArray(reader.getV8Serialized())}; + break; + } + + auto properties = reader.getProperties(); + if (properties.size() > 0) { + result.properties.reserve(properties.size()); + + for (auto property: properties) { + result.properties.add(Property{ + .name = kj::str(property.getName()), + .value = fromCapnp(property), + }); + } + } + + return result; +} + +jsg::JsValue Frankenvalue::toJs(jsg::Lock& js) const { + // TODO(cleanup): Make `withinHandleScope()` correctly support `jsg::JsValue` and friends. + return jsg::JsValue(js.withinHandleScope([&]() -> v8::Local { + jsg::JsValue result = [&]() -> jsg::JsValue { + KJ_SWITCH_ONEOF(value) { + KJ_CASE_ONEOF(_, EmptyObject) { + return js.obj(); + } + KJ_CASE_ONEOF(json, Json) { + // TODO(cleanup): Make jsg::Lock::parseJson() not return a persistent handle. + return jsg::JsValue(jsg::check(v8::JSON::Parse(js.v8Context(), js.str(json.json)))); + } + KJ_CASE_ONEOF(v8Serialized, V8Serialized) { + jsg::Deserializer deser(js, v8Serialized.data); + return deser.readValue(js); + } + } + KJ_UNREACHABLE; + }(); + + if (properties.size() > 0) { + jsg::JsObject obj = KJ_REQUIRE_NONNULL( + result.tryCast(), "non-object Frakenvalue can't have properties"); + + for (auto& property: properties) { + obj.set(js, property.name, property.value.toJs(js)); + } + } + + return result; + })); +} + +Frankenvalue Frankenvalue::fromJs(jsg::Lock& js, jsg::JsValue value) { + Frankenvalue result; + + js.withinHandleScope([&]() { + jsg::Serializer ser(js, {.treatClassInstancesAsPlainObjects = false}); + ser.write(js, value); + result.value = V8Serialized{ser.release().data}; + }); + + return result; +} + +Frankenvalue Frankenvalue::fromJson(kj::String json) { + Frankenvalue result; + result.value = Json{kj::mv(json)}; + return result; +} + +void Frankenvalue::setProperty(kj::String name, Frankenvalue value) { + properties.add(Property{ + .name = kj::mv(name), + .value = kj::mv(value), + }); +} + +} // namespace workerd diff --git a/src/workerd/io/frankenvalue.capnp b/src/workerd/io/frankenvalue.capnp new file mode 100644 index 00000000000..717ad0f33a7 --- /dev/null +++ b/src/workerd/io/frankenvalue.capnp @@ -0,0 +1,37 @@ +@0xcc3b225cb3101aba; + +using Cxx = import "/capnp/c++.capnp"; +$Cxx.namespace("workerd::rpc"); +$Cxx.allowCancellation; + +struct Frankenvalue { + # Represents a JavaScript value that has been stitched together from multiple sources outside of + # a JavaScript evaluation context. The Frankevalue can be evaluated down to a JS value as soon + # as it has a JS execution environment in which to be evaluated. + # + # This is used in particular to represent `ctx.props`. + + union { + emptyObject @0 :Void; + # Just an object with no properties. + + json @1 :Text; + # Parse this JSON-formatted text to compute the value. + + v8Serialized @2 :Data; + # Parse these V8-serialized bytes to compute the value. + } + + properties @3 :List(Frankenvalue); + # Additional properties to add to the value. The base value (specified by the union above) must + # be an object. Each property in this list must have a `name`. They will be added as properties + # of the object. + # + # If a property in the list conflicts with a property that already exists in the base value, + # the property is overwritten with the value from the `properties` list. + + name @4 :Text; + # Property name. Used only when this `Frankenvalue` represents a property, that is, it is an + # element within the `properties` list of some other `Frankenvalue`. If this is the root value, + # then `name` must be null. +} diff --git a/src/workerd/io/frankenvalue.h b/src/workerd/io/frankenvalue.h new file mode 100644 index 00000000000..9e4a69220e5 --- /dev/null +++ b/src/workerd/io/frankenvalue.h @@ -0,0 +1,73 @@ +// Copyright (c) 2024 Cloudflare, Inc. +// Licensed under the Apache 2.0 license found in the LICENSE file or at: +// https://opensource.org/licenses/Apache-2.0 + +#pragma once + +#include +#include + +namespace workerd { + +// C++ class mirroring `Frankenvalue` as defined in `frankenvalue.capnp`. +// +// Represents a JavaScript value that has been stitched together from multiple sources outside of +// a JavaScript evaluation context. The Frankevalue can be evaluated down to a JS value as soon +// as it has a JS execution environment in which to be evaluated. +// +// This is used in particular to represent `ctx.props`. +class Frankenvalue { + public: + Frankenvalue(): value(EmptyObject()) {} + + bool empty() const { + return value.is() && properties.empty(); + } + + Frankenvalue clone() const; + + // Convert to/from capnp format. + void toCapnp(rpc::Frankenvalue::Builder builder) const; + static Frankenvalue fromCapnp(rpc::Frankenvalue::Reader reader); + + // Convert to/from JavaScript values. Note that round trips here don't produce the exact same + // Frankenvalue representation: toJs() puts all the contents together into a single value, and + // fromJs() always returns a Frakenvalue containing a single V8-serialized value. + jsg::JsValue toJs(jsg::Lock& js) const; + static Frankenvalue fromJs(jsg::Lock& js, jsg::JsValue value); + + // Construct a Frakenvalue from JSON. + // + // (It's not possible to convert a Frakenvalue back to JSON, except by evaluating it in JS and + // then JSON-stringifying from there.) + static Frankenvalue fromJson(kj::String json); + + // Add a property to the value, represented as another Frankenvalue. This is how you "stitch + // together" values! + // + // This is called `set` because the new property will override any existing property with the + // same name, but note that this strictly appends content. The replacement happens only when the + // Frankenvalue is finally converted to JS. + void setProperty(kj::String name, Frankenvalue value); + + private: + struct EmptyObject {}; + struct Json { + kj::String json; + }; + struct V8Serialized { + kj::Array data; + }; + kj::OneOf value; + + struct Property; + kj::Vector properties; +}; + +// Can't be defined inline since `Frankenvalue` is still incomplete there. +struct Frankenvalue::Property { + kj::String name; + Frankenvalue value; +}; + +} // namespace workerd diff --git a/src/workerd/jsg/jsg-test.h b/src/workerd/jsg/jsg-test.h index 9c4fcb0cfa9..02de14a4217 100644 --- a/src/workerd/jsg/jsg-test.h +++ b/src/workerd/jsg/jsg-test.h @@ -135,6 +135,28 @@ class Evaluator { lock.runMicrotasks(); } + // Run some C++ code in a new lock and context. + template + void run(Func&& func) { + getIsolate().runInLockScope([&](typename IsolateType::Lock& lock) { + JSG_WITHIN_CONTEXT_SCOPE(lock, + lock.template newContext().getHandle(lock.v8Isolate), [&](jsg::Lock& js) { + v8::TryCatch tryCatch(js.v8Isolate); + + try { + func(js); + } catch (JsExceptionThrown&) { + if (tryCatch.HasTerminated()) { + KJ_FAIL_ASSERT("TerminateExecution() was called"); + } else { + KJ_ASSERT(tryCatch.HasCaught()); + jsg::throwTunneledException(js.v8Isolate, tryCatch.Exception()); + } + } + }); + }); + } + private: V8System& v8System; }; From e34ab7e73effe3fd0ab12e11e148b8005a038912 Mon Sep 17 00:00:00 2001 From: Kenton Varda Date: Fri, 29 Nov 2024 16:50:39 -0600 Subject: [PATCH 4/5] Implement connection props. This adds `ctx.props` to the `ctx` object given to `WorkerEntrypoint`s. The property receives metadata about the particular service binding over which the entrypoint was invoked. ``` class MyEntrypoint extends WorkerEntrypoint { foo() { console.log("called by: " + this.ctx.props.caller); } } ``` Service binding declarations in the workerd config may specify what metadata to pass: ``` bindings = [ ( name = "FOO", service = ( name = "my-service", entrypoint = "MyEntrypoint", props = ( json = `{"caller": "my-calling-service"} ) ) ) ] ``` Note that "caller" is just an example. The props can contain anything. Use cases include: * Authentication of the caller's identity. * Authorization / permissions (independent of caller identity). * Specifying a particular resource. For example, if the `WorkerEntrypoint` represents a chat room, `props.roomId` could be the ID of the specific chat room to access. This allows service bindings to implement a deeper capability-based security model, where bindings point to specific resources with specific permissions, instead of general APIs. On Cloudflare, only users who have permission to modify your worker will have permission to create a binding containing arbitrary metadata. Meanwhile we will be creating a mechanism by which you can grant a service binding to your worker to someone, but where you specify the metadata. Thus, you can use the metadata to authenticate requests, without the need for any cryptography. --- src/workerd/api/global-scope.h | 19 +++++ src/workerd/api/hibernatable-web-socket.c++ | 13 +-- src/workerd/api/hibernatable-web-socket.h | 1 + src/workerd/api/queue.c++ | 10 ++- src/workerd/api/queue.h | 1 + src/workerd/api/trace.c++ | 13 +-- src/workerd/api/trace.h | 1 + src/workerd/api/worker-rpc.c++ | 16 ++-- src/workerd/api/worker-rpc.h | 1 + src/workerd/io/BUILD.bazel | 1 + src/workerd/io/worker-entrypoint.c++ | 44 ++++++---- src/workerd/io/worker-entrypoint.h | 2 + src/workerd/io/worker-interface.h | 2 + src/workerd/io/worker.c++ | 14 +++- src/workerd/io/worker.h | 9 +- src/workerd/server/server-test.c++ | 47 +++++++++++ src/workerd/server/server.c++ | 92 +++++++++++++++------ src/workerd/server/workerd.capnp | 10 +++ src/workerd/tests/test-fixture.c++ | 2 +- 19 files changed, 227 insertions(+), 71 deletions(-) diff --git a/src/workerd/api/global-scope.h b/src/workerd/api/global-scope.h index ee0d5bdf620..8a3b9f61f51 100644 --- a/src/workerd/api/global-scope.h +++ b/src/workerd/api/global-scope.h @@ -204,6 +204,9 @@ class TestController: public jsg::Object { class ExecutionContext: public jsg::Object { public: + ExecutionContext(jsg::Lock& js): props(js, js.obj()) {} + ExecutionContext(jsg::Lock& js, jsg::JsValue props): props(js, props) {} + void waitUntil(kj::Promise promise); void passThroughOnException(); @@ -211,9 +214,14 @@ class ExecutionContext: public jsg::Object { // and throwing an error at the client. void abort(jsg::Lock& js, jsg::Optional reason); + jsg::JsValue getProps(jsg::Lock& js) { + return props.getHandle(js); + } + JSG_RESOURCE_TYPE(ExecutionContext, CompatibilityFlags::Reader flags) { JSG_METHOD(waitUntil); JSG_METHOD(passThroughOnException); + JSG_LAZY_INSTANCE_PROPERTY(props, getProps); if (flags.getWorkerdExperimental()) { // TODO(soon): Before making this generally available we need to: @@ -229,6 +237,17 @@ class ExecutionContext: public jsg::Object { JSG_METHOD(abort); } } + + void visitForMemoryInfo(jsg::MemoryTracker& tracker) const { + tracker.trackField("props", props); + } + + private: + jsg::JsRef props; + + void visitForGc(jsg::GcVisitor& visitor) { + visitor.visit(props); + } }; // AlarmEventInfo is a jsg::Object used to pass alarm invocation info to an alarm handler. diff --git a/src/workerd/api/hibernatable-web-socket.c++ b/src/workerd/api/hibernatable-web-socket.c++ index 2748c9c3ac9..6257c79c2f0 100644 --- a/src/workerd/api/hibernatable-web-socket.c++ +++ b/src/workerd/api/hibernatable-web-socket.c++ @@ -59,6 +59,7 @@ jsg::Ref HibernatableWebSocketEvent::claimWebSocket( kj::Promise HibernatableWebSocketCustomEventImpl::run( kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) { // Mark the request as delivered because we're about to run some JS. auto& context = incomingRequest->getContext(); @@ -100,28 +101,28 @@ kj::Promise HibernatableWebSocketCustomEve try { co_await context.run( - [entrypointName = entrypointName, &context, eventParameters = kj::mv(eventParameters)]( - Worker::Lock& lock) mutable { + [entrypointName = entrypointName, &context, eventParameters = kj::mv(eventParameters), + props = kj::mv(props)](Worker::Lock& lock) mutable { KJ_SWITCH_ONEOF(eventParameters.eventType) { KJ_CASE_ONEOF(text, HibernatableSocketParams::Text) { return lock.getGlobalScope().sendHibernatableWebSocketMessage(kj::mv(text.message), eventParameters.eventTimeoutMs, kj::mv(eventParameters.websocketId), lock, - lock.getExportedHandler(entrypointName, context.getActor())); + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); } KJ_CASE_ONEOF(data, HibernatableSocketParams::Data) { return lock.getGlobalScope().sendHibernatableWebSocketMessage(kj::mv(data.message), eventParameters.eventTimeoutMs, kj::mv(eventParameters.websocketId), lock, - lock.getExportedHandler(entrypointName, context.getActor())); + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); } KJ_CASE_ONEOF(close, HibernatableSocketParams::Close) { return lock.getGlobalScope().sendHibernatableWebSocketClose(kj::mv(close), eventParameters.eventTimeoutMs, kj::mv(eventParameters.websocketId), lock, - lock.getExportedHandler(entrypointName, context.getActor())); + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); } KJ_CASE_ONEOF(e, HibernatableSocketParams::Error) { return lock.getGlobalScope().sendHibernatableWebSocketError(kj::mv(e.error), eventParameters.eventTimeoutMs, kj::mv(eventParameters.websocketId), lock, - lock.getExportedHandler(entrypointName, context.getActor())); + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); } KJ_UNREACHABLE; } diff --git a/src/workerd/api/hibernatable-web-socket.h b/src/workerd/api/hibernatable-web-socket.h index f1b82dc6744..93b5e83a94f 100644 --- a/src/workerd/api/hibernatable-web-socket.h +++ b/src/workerd/api/hibernatable-web-socket.h @@ -66,6 +66,7 @@ class HibernatableWebSocketCustomEventImpl final: public WorkerInterface::Custom kj::Promise run(kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) override; kj::Promise sendRpc(capnp::HttpOverCapnpFactory& httpOverCapnpFactory, diff --git a/src/workerd/api/queue.c++ b/src/workerd/api/queue.c++ index 8b7b95475a9..36bc07b1af6 100644 --- a/src/workerd/api/queue.c++ +++ b/src/workerd/api/queue.c++ @@ -513,6 +513,7 @@ jsg::Ref startQueueEvent(EventTarget& globalEventTarget, kj::Promise QueueCustomEventImpl::run( kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) { incomingRequest->delivered(); auto& context = incomingRequest->getContext(); @@ -546,13 +547,14 @@ kj::Promise QueueCustomEventImpl::run( // can't just wait on their addEventListener handler to resolve because it can't be async). context.addWaitUntil(context.run( [this, entrypointName = entrypointName, &context, queueEvent = kj::addRef(*queueEventHolder), - &metrics = incomingRequest->getMetrics()](Worker::Lock& lock) mutable { + &metrics = incomingRequest->getMetrics(), + props = kj::mv(props)](Worker::Lock& lock) mutable { jsg::AsyncContextFrame::StorageScope traceScope = context.makeAsyncTraceScope(lock); auto& typeHandler = lock.getWorker().getIsolate().getApi().getQueueTypeHandler(lock); - queueEvent->event = - startQueueEvent(lock.getGlobalScope(), kj::mv(params), context.addObject(result), lock, - lock.getExportedHandler(entrypointName, context.getActor()), typeHandler); + queueEvent->event = startQueueEvent(lock.getGlobalScope(), kj::mv(params), + context.addObject(result), lock, + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor()), typeHandler); })); // TODO(soon): There's a good chance we'll want a different wall-clock timeout for queue handlers diff --git a/src/workerd/api/queue.h b/src/workerd/api/queue.h index dfcbd20662e..567a69ae15b 100644 --- a/src/workerd/api/queue.h +++ b/src/workerd/api/queue.h @@ -341,6 +341,7 @@ class QueueCustomEventImpl final: public WorkerInterface::CustomEvent, public kj kj::Promise run(kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) override; kj::Promise sendRpc(capnp::HttpOverCapnpFactory& httpOverCapnpFactory, diff --git a/src/workerd/api/trace.c++ b/src/workerd/api/trace.c++ index ca23202813c..2ba7ebab889 100644 --- a/src/workerd/api/trace.c++ +++ b/src/workerd/api/trace.c++ @@ -601,6 +601,7 @@ jsg::Ref UnsafeTraceMetrics::fromTrace(jsg::Ref item) { namespace { kj::Promise sendTracesToExportedHandler(kj::Own incomingRequest, kj::Maybe entrypointNamePtr, + Frankenvalue props, kj::ArrayPtr> traces) { // Mark the request as delivered because we're about to run some JS. incomingRequest->delivered(); @@ -624,11 +625,12 @@ kj::Promise sendTracesToExportedHandler(kj::Own sendTracesToExportedHandler(kj::Own incomingRequest, kj::Maybe entrypointNamePtr, + Frankenvalue props, kj::TaskSet& waitUntilTasks) -> kj::Promise { // Don't bother to wait around for the handler to run, just hand it off to the waitUntil tasks. - waitUntilTasks.add( - sendTracesToExportedHandler(kj::mv(incomingRequest), entrypointNamePtr, traces)); + waitUntilTasks.add(sendTracesToExportedHandler( + kj::mv(incomingRequest), entrypointNamePtr, kj::mv(props), traces)); return Result{ .outcome = EventOutcome::OK, diff --git a/src/workerd/api/trace.h b/src/workerd/api/trace.h index 812119ea2ad..dd9a804fbed 100644 --- a/src/workerd/api/trace.h +++ b/src/workerd/api/trace.h @@ -615,6 +615,7 @@ class TraceCustomEventImpl final: public WorkerInterface::CustomEvent { kj::Promise run(kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) override; kj::Promise sendRpc(capnp::HttpOverCapnpFactory& httpOverCapnpFactory, diff --git a/src/workerd/api/worker-rpc.c++ b/src/workerd/api/worker-rpc.c++ index 9d1dc2b29c2..ec304b53513 100644 --- a/src/workerd/api/worker-rpc.c++ +++ b/src/workerd/api/worker-rpc.c++ @@ -1666,18 +1666,21 @@ class EntrypointJsRpcTarget final: public JsRpcTargetBase { public: EntrypointJsRpcTarget(IoContext& ioCtx, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> tracer) : JsRpcTargetBase(ioCtx), // Most of the time we don't really have to clone this but it's hard to fully prove, so // let's be safe. entrypointName(entrypointName.map([](kj::StringPtr s) { return kj::str(s); })), + props(kj::mv(props)), tracer(kj::mv(tracer)) {} TargetInfo getTargetInfo(Worker::Lock& lock, IoContext& ioCtx) override { jsg::Lock& js = lock; - auto handler = KJ_REQUIRE_NONNULL(lock.getExportedHandler(entrypointName, ioCtx.getActor()), - "Failed to get handler to worker."); + auto handler = + KJ_REQUIRE_NONNULL(lock.getExportedHandler(entrypointName, kj::mv(props), ioCtx.getActor()), + "Failed to get handler to worker."); if (handler->missingSuperclass) { // JS RPC is not enabled on the server side, we cannot call any methods. @@ -1709,6 +1712,7 @@ class EntrypointJsRpcTarget final: public JsRpcTargetBase { private: kj::Maybe entrypointName; + Frankenvalue props; kj::Maybe> tracer; bool isReservedName(kj::StringPtr name) override { @@ -1781,15 +1785,17 @@ class JsRpcSessionCustomEventImpl::ServerTopLevelMembrane final: public capnp::M kj::Promise JsRpcSessionCustomEventImpl::run( kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) { IoContext& ioctx = incomingRequest->getContext(); incomingRequest->delivered(); auto [donePromise, doneFulfiller] = kj::newPromiseAndFulfiller(); - capFulfiller->fulfill(capnp::membrane(kj::heap(ioctx, entrypointName, - mapAddRef(incomingRequest->getWorkerTracer())), - kj::refcounted(kj::mv(doneFulfiller)))); + capFulfiller->fulfill( + capnp::membrane(kj::heap(ioctx, entrypointName, kj::mv(props), + mapAddRef(incomingRequest->getWorkerTracer())), + kj::refcounted(kj::mv(doneFulfiller)))); KJ_DEFER({ // waitUntil() should allow extending execution on the server side even when the client diff --git a/src/workerd/api/worker-rpc.h b/src/workerd/api/worker-rpc.h index e32e2d40517..3cf88f1de16 100644 --- a/src/workerd/api/worker-rpc.h +++ b/src/workerd/api/worker-rpc.h @@ -420,6 +420,7 @@ class JsRpcSessionCustomEventImpl final: public WorkerInterface::CustomEvent { kj::Promise run(kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) override; kj::Promise sendRpc(capnp::HttpOverCapnpFactory& httpOverCapnpFactory, diff --git a/src/workerd/io/BUILD.bazel b/src/workerd/io/BUILD.bazel index e7e5f43b876..6248e30f2fa 100644 --- a/src/workerd/io/BUILD.bazel +++ b/src/workerd/io/BUILD.bazel @@ -220,6 +220,7 @@ wd_cc_library( hdrs = ["worker-interface.h"], visibility = ["//visibility:public"], deps = [ + ":frankenvalue_capnp", ":worker-interface_capnp", "@capnp-cpp//src/capnp:capnp-rpc", "@capnp-cpp//src/capnp:capnpc", diff --git a/src/workerd/io/worker-entrypoint.c++ b/src/workerd/io/worker-entrypoint.c++ index ad8fa6089e5..83c7c5aed8d 100644 --- a/src/workerd/io/worker-entrypoint.c++ +++ b/src/workerd/io/worker-entrypoint.c++ @@ -45,6 +45,7 @@ class WorkerEntrypoint final: public WorkerInterface { static kj::Own construct(ThreadContext& threadContext, kj::Own worker, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> actor, kj::Own limitEnforcer, kj::Own ioContextDependency, @@ -82,6 +83,7 @@ class WorkerEntrypoint final: public WorkerInterface { kj::Maybe> incomingRequest; bool tunnelExceptions; kj::Maybe entrypointName; + Frankenvalue props; kj::Maybe cfBlobJson; // Hacky members used to hold some temporary state while processing a request. @@ -114,6 +116,7 @@ class WorkerEntrypoint final: public WorkerInterface { kj::TaskSet& waitUntilTasks, bool tunnelExceptions, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe cfBlobJson); }; @@ -152,6 +155,7 @@ class WorkerEntrypoint::ResponseSentTracker final: public kj::HttpService::Respo kj::Own WorkerEntrypoint::construct(ThreadContext& threadContext, kj::Own worker, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> actor, kj::Own limitEnforcer, kj::Own ioContextDependency, @@ -171,7 +175,7 @@ kj::Own WorkerEntrypoint::construct(ThreadContext& threadContex threadContext.getEntropySource()); auto obj = kj::heap(kj::Badge(), threadContext, - waitUntilTasks, tunnelExceptions, entrypointName, kj::mv(cfBlobJson)); + waitUntilTasks, tunnelExceptions, entrypointName, kj::mv(props), kj::mv(cfBlobJson)); obj->init(kj::mv(worker), kj::mv(actor), kj::mv(limitEnforcer), kj::mv(ioContextDependency), kj::mv(ioChannelFactory), kj::addRef(*metrics), kj::mv(workerTracer), kj::mv(invocationSpanContext)); @@ -184,11 +188,13 @@ WorkerEntrypoint::WorkerEntrypoint(kj::Badge badge, kj::TaskSet& waitUntilTasks, bool tunnelExceptions, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe cfBlobJson) : threadContext(threadContext), waitUntilTasks(waitUntilTasks), tunnelExceptions(tunnelExceptions), entrypointName(entrypointName), + props(kj::mv(props)), cfBlobJson(kj::mv(cfBlobJson)) {} void WorkerEntrypoint::init(kj::Own worker, @@ -288,7 +294,8 @@ kj::Promise WorkerEntrypoint::request(kj::HttpMethod method, jsg::AsyncContextFrame::StorageScope traceScope = context.makeAsyncTraceScope(lock); return lock.getGlobalScope().request(method, url, headers, requestBody, wrappedResponse, - cfBlobJson, lock, lock.getExportedHandler(entrypointName, context.getActor())); + cfBlobJson, lock, + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); }) .then([this](api::DeferredProxy deferredProxy) { TRACE_EVENT("workerd", "WorkerEntrypoint::request() deferred proxy step", @@ -494,14 +501,14 @@ kj::Promise WorkerEntrypoint::runScheduled( } // Scheduled handlers run entirely in waitUntil() tasks. - context.addWaitUntil( - context.run([scheduledTime, cron, entrypointName = entrypointName, &context, - &metrics = incomingRequest->getMetrics()](Worker::Lock& lock) mutable { + context.addWaitUntil(context.run( + [scheduledTime, cron, entrypointName = entrypointName, props = kj::mv(props), &context, + &metrics = incomingRequest->getMetrics()](Worker::Lock& lock) mutable { TRACE_EVENT("workerd", "WorkerEntrypoint::runScheduled() run"); jsg::AsyncContextFrame::StorageScope traceScope = context.makeAsyncTraceScope(lock); - lock.getGlobalScope().startScheduled( - scheduledTime, cron, lock, lock.getExportedHandler(entrypointName, context.getActor())); + lock.getGlobalScope().startScheduled(scheduledTime, cron, lock, + lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor())); })); static auto constexpr waitForFinished = [](IoContext& context, @@ -569,7 +576,7 @@ kj::Promise WorkerEntrypoint::runAlarmImpl( try { auto result = co_await context.run([scheduledTime, retryCount, entrypointName = entrypointName, - &context](Worker::Lock& lock) { + props = kj::mv(props), &context](Worker::Lock& lock) mutable { jsg::AsyncContextFrame::StorageScope traceScope = context.makeAsyncTraceScope(lock); // If we have an invalid timeout, set it to the default value of 15 minutes. @@ -579,7 +586,7 @@ kj::Promise WorkerEntrypoint::runAlarmImpl( timeout = 15 * kj::MINUTES; } - auto handler = lock.getExportedHandler(entrypointName, context.getActor()); + auto handler = lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor()); return lock.getGlobalScope().runAlarm(scheduledTime, timeout, retryCount, lock, handler); }); @@ -638,15 +645,15 @@ kj::Promise WorkerEntrypoint::test() { auto& context = incomingRequest->getContext(); - context.addWaitUntil(context.run( - [entrypointName = entrypointName, &context, &metrics = incomingRequest->getMetrics()]( - Worker::Lock& lock) mutable -> kj::Promise { + context.addWaitUntil(context.run([entrypointName = entrypointName, props = kj::mv(props), + &context, &metrics = incomingRequest->getMetrics()]( + Worker::Lock& lock) mutable -> kj::Promise { TRACE_EVENT("workerd", "WorkerEntrypoint::test() run"); jsg::AsyncContextFrame::StorageScope traceScope = context.makeAsyncTraceScope(lock); return context.awaitJs(lock, lock.getGlobalScope().test( - lock, lock.getExportedHandler(entrypointName, context.getActor()))); + lock, lock.getExportedHandler(entrypointName, kj::mv(props), context.getActor()))); })); static auto constexpr waitForFinished = @@ -669,8 +676,8 @@ kj::Promise WorkerEntrypoint::customEvent( this->incomingRequest = kj::none; auto& context = incomingRequest->getContext(); - auto promise = - event->run(kj::mv(incomingRequest), entrypointName, waitUntilTasks).attach(kj::mv(event)); + auto promise = event->run(kj::mv(incomingRequest), entrypointName, kj::mv(props), waitUntilTasks) + .attach(kj::mv(event)); // TODO(cleanup): In theory `context` may have been destroyed by now if `event->run()` dropped // the `incomingRequest` synchronously. No current implementation does that, and @@ -720,6 +727,7 @@ kj::Promise WorkerEntrypoint::maybeAddGcPassForTest(IoContext& context, kj::P kj::Own newWorkerEntrypoint(ThreadContext& threadContext, kj::Own worker, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> actor, kj::Own limitEnforcer, kj::Own ioContextDependency, @@ -731,9 +739,9 @@ kj::Own newWorkerEntrypoint(ThreadContext& threadContext, kj::Maybe cfBlobJson, kj::Maybe maybeTriggerInvocationSpan) { return WorkerEntrypoint::construct(threadContext, kj::mv(worker), kj::mv(entrypointName), - kj::mv(actor), kj::mv(limitEnforcer), kj::mv(ioContextDependency), kj::mv(ioChannelFactory), - kj::mv(metrics), waitUntilTasks, tunnelExceptions, kj::mv(workerTracer), kj::mv(cfBlobJson), - kj::mv(maybeTriggerInvocationSpan)); + kj::mv(props), kj::mv(actor), kj::mv(limitEnforcer), kj::mv(ioContextDependency), + kj::mv(ioChannelFactory), kj::mv(metrics), waitUntilTasks, tunnelExceptions, + kj::mv(workerTracer), kj::mv(cfBlobJson), kj::mv(maybeTriggerInvocationSpan)); } } // namespace workerd diff --git a/src/workerd/io/worker-entrypoint.h b/src/workerd/io/worker-entrypoint.h index a5f4628e736..f6ad7ccc388 100644 --- a/src/workerd/io/worker-entrypoint.h +++ b/src/workerd/io/worker-entrypoint.h @@ -4,6 +4,7 @@ #pragma once +#include #include namespace workerd { @@ -30,6 +31,7 @@ class InvocationSpanContext; kj::Own newWorkerEntrypoint(ThreadContext& threadContext, kj::Own worker, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> actor, kj::Own limitEnforcer, kj::Own ioContextDependency, diff --git a/src/workerd/io/worker-interface.h b/src/workerd/io/worker-interface.h index d49d7b61026..8b70643bc11 100644 --- a/src/workerd/io/worker-interface.h +++ b/src/workerd/io/worker-interface.h @@ -12,6 +12,7 @@ namespace workerd { +class Frankenvalue; class IoContext_IncomingRequest; // An interface representing the services made available by a worker/pipeline to handle a @@ -111,6 +112,7 @@ class WorkerInterface: public kj::HttpService { // for this event. virtual kj::Promise run(kj::Own incomingRequest, kj::Maybe entrypointName, + Frankenvalue props, kj::TaskSet& waitUntilTasks) = 0; // Forward the event over RPC. diff --git a/src/workerd/io/worker.c++ b/src/workerd/io/worker.c++ index 6ca0c3b3a1e..552467699c5 100644 --- a/src/workerd/io/worker.c++ +++ b/src/workerd/io/worker.c++ @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -1675,7 +1676,12 @@ Worker::Worker(kj::Own scriptParam, KJ_SWITCH_ONEOF(handler.value) { KJ_CASE_ONEOF(obj, api::ExportedHandler) { obj.env = lock.v8Ref(bindingsScope.As()); - obj.ctx = jsg::alloc(); + // TODO(cleanup): Unfortunately, for non-class-based handlers, we have + // always created only a single `ctx` object and reused it for all + // requests. This is weird and obviously wrong but changing it probably + // requires a compat flag. Until then, connection properties will not be + // available for non-class handlers. + obj.ctx = jsg::alloc(lock); impl->namedHandlers.insert(kj::mv(handler.name), kj::mv(obj)); } @@ -1964,7 +1970,7 @@ static inline kj::Own fakeOwn(T& ref) { } kj::Maybe> Worker::Lock::getExportedHandler( - kj::Maybe name, kj::Maybe actor) { + kj::Maybe name, Frankenvalue props, kj::Maybe actor) { KJ_IF_SOME(a, actor) { KJ_IF_SOME(h, a.getHandler()) { return fakeOwn(h); @@ -1976,8 +1982,8 @@ kj::Maybe> Worker::Lock::getExportedHandler( return fakeOwn(h); } else KJ_IF_SOME(cls, worker.impl->statelessClasses.find(n)) { jsg::Lock& js = *this; - auto handler = kj::heap(cls( - js, jsg::alloc(), KJ_ASSERT_NONNULL(worker.impl->env).addRef(js))); + auto handler = kj::heap(cls(js, jsg::alloc(js, props.toJs(js)), + KJ_ASSERT_NONNULL(worker.impl->env).addRef(js))); // HACK: We set handler.env and handler.ctx to undefined because we already passed the real // env and ctx into the constructor, and we want the handler methods to act like they take diff --git a/src/workerd/io/worker.h b/src/workerd/io/worker.h index ab31be89224..beae2e9d483 100644 --- a/src/workerd/io/worker.h +++ b/src/workerd/io/worker.h @@ -8,6 +8,7 @@ #include // because we can't forward-declare ActorCache::SharedLru. #include #include +#include #include #include #include @@ -627,10 +628,12 @@ class Worker::Lock { // default handler. Returns null if this is not a modules-syntax worker (but `entrypointName` // must be null in that case). // - // If running in an actor, the name is ignored and the entrypoint originally used to construct - // the actor is returned. + // `props` is the value to place in `ctx.props`. + // + // If running in an actor, the name and props are ignored and the entrypoint originally used to + // construct the actor is returned. kj::Maybe> getExportedHandler( - kj::Maybe entrypointName, kj::Maybe actor); + kj::Maybe entrypointName, Frankenvalue props, kj::Maybe actor); // Get the C++ object representing the global scope. api::ServiceWorkerGlobalScope& getGlobalScope(); diff --git a/src/workerd/server/server-test.c++ b/src/workerd/server/server-test.c++ index 51708b3de3c..54c8ea19748 100644 --- a/src/workerd/server/server-test.c++ +++ b/src/workerd/server/server-test.c++ @@ -3697,6 +3697,53 @@ KJ_TEST("Server: JS RPC over HTTP connections") { conn.httpGet200("/", "got: 35"); } +KJ_TEST("Server: Entrypoint binding with props") { + TestServer test(R"(( + services = [ + ( name = "hello", + worker = ( + compatibilityDate = "2024-02-23", + compatibilityFlags = ["experimental"], + modules = [ + ( name = "main.js", + esModule = + `import {WorkerEntrypoint} from "cloudflare:workers"; + `export default { + ` async fetch(request, env) { + ` return new Response("got: " + await env.MyRpc.getProps()); + ` } + `} + `export class MyRpc extends WorkerEntrypoint { + ` getProps() { return this.ctx.props.foo; } + `} + ) + ], + bindings = [ + ( name = "MyRpc", + service = ( + name = "hello", + entrypoint = "MyRpc", + props = ( + json = `{"foo": 123} + ) + ) + ) + ] + ) + ), + ], + sockets = [ + ( name = "main", address = "test-addr", service = "hello" ), + ] + ))"_kj); + + test.server.allowExperimental(); + test.start(); + + auto conn = test.connect("test-addr"); + conn.httpGet200("/", "got: 123"); +} + // ======================================================================================= // TODO(beta): Test TLS (send and receive) diff --git a/src/workerd/server/server.c++ b/src/workerd/server/server.c++ index 873d6950ac4..9be9dc570ec 100644 --- a/src/workerd/server/server.c++ +++ b/src/workerd/server/server.c++ @@ -1622,9 +1622,17 @@ class Server::WorkerService final: public Service, } } - kj::Maybe> getEntrypoint(kj::StringPtr name) { - auto& entry = KJ_UNWRAP_OR_RETURN(namedEntrypoints.findEntry(name), kj::none); - return kj::heap(*this, entry.key, entry.value); + kj::Maybe> getEntrypoint( + kj::Maybe name, kj::Maybe propsJson) { + kj::HashSet* handlers; + KJ_IF_SOME(n, name) { + auto& entry = KJ_UNWRAP_OR_RETURN(namedEntrypoints.findEntry(n), kj::none); + name = entry.key; // replace with more-permanent string + handlers = &entry.value; + } else { + handlers = &KJ_UNWRAP_OR_RETURN(defaultEntrypointHandlers, kj::none); + } + return kj::heap(*this, name, propsJson, *handlers); } kj::Array getEntrypointNames() { @@ -1658,7 +1666,7 @@ class Server::WorkerService final: public Service, } kj::Own startRequest(IoChannelFactory::SubrequestMetadata metadata) override { - return startRequest(kj::mv(metadata), kj::none); + return startRequest(kj::mv(metadata), kj::none, {}); } bool hasHandler(kj::StringPtr handlerName) override { @@ -1671,6 +1679,7 @@ class Server::WorkerService final: public Service, kj::Own startRequest(IoChannelFactory::SubrequestMetadata metadata, kj::Maybe entrypointName, + Frankenvalue props, kj::Maybe> actor = kj::none) { TRACE_EVENT("workerd", "Server::WorkerService::startRequest()"); @@ -1720,7 +1729,7 @@ class Server::WorkerService final: public Service, auto observer = kj::refcounted(kj::addRef(*workerTracer)); return newWorkerEntrypoint(threadContext, kj::atomicAddRef(*worker), entrypointName, - kj::mv(actor), kj::Own(this, kj::NullDisposer::instance), + kj::mv(props), kj::mv(actor), kj::Own(this, kj::NullDisposer::instance), {}, // ioContextDependency kj::Own(this, kj::NullDisposer::instance), kj::mv(observer), waitUntilTasks, @@ -1996,7 +2005,8 @@ class Server::WorkerService final: public Service, cleanupTask = cleanupLoop(); } - co_return service.startRequest(kj::mv(metadata), className, kj::mv(actor)) + // Actors always have empty `props`, at least for now. + co_return service.startRequest(kj::mv(metadata), className, {}, kj::mv(actor)) .attach(kj::mv(refTracker)); } @@ -2216,14 +2226,20 @@ class Server::WorkerService final: public Service, private: class EntrypointService final: public Service { public: - EntrypointService( - WorkerService& worker, kj::StringPtr entrypoint, kj::HashSet& handlers) + EntrypointService(WorkerService& worker, + kj::Maybe entrypoint, + kj::Maybe propsJson, + kj::HashSet& handlers) : worker(worker), entrypoint(entrypoint), - handlers(handlers) {} + handlers(handlers) { + KJ_IF_SOME(m, propsJson) { + props = Frankenvalue::fromJson(kj::str(m)); + } + } kj::Own startRequest(IoChannelFactory::SubrequestMetadata metadata) override { - return worker.startRequest(kj::mv(metadata), entrypoint); + return worker.startRequest(kj::mv(metadata), entrypoint, props.clone()); } bool hasHandler(kj::StringPtr handlerName) override { @@ -2232,8 +2248,9 @@ class Server::WorkerService final: public Service, private: WorkerService& worker; - kj::StringPtr entrypoint; + kj::Maybe entrypoint; kj::HashSet& handlers; + Frankenvalue props; }; ThreadContext& threadContext; @@ -3312,25 +3329,50 @@ kj::Own Server::lookupService( return fakeOwn(*invalidConfigServiceSingleton); }); + kj::Maybe entrypointName; if (designator.hasEntrypoint()) { - kj::StringPtr entrypointName = designator.getEntrypoint(); - if (WorkerService* worker = dynamic_cast(service)) { - KJ_IF_SOME(ep, worker->getEntrypoint(entrypointName)) { - return kj::mv(ep); - } else { - reportConfigError(kj::str(errorContext, " refers to service \"", targetName, - "\" with a named entrypoint \"", entrypointName, "\", but \"", targetName, - "\" has no such named entrypoint.")); - return fakeOwn(*invalidConfigServiceSingleton); - } + entrypointName = designator.getEntrypoint(); + } + + auto propsJson = [&]() -> kj::Maybe { + auto props = designator.getProps(); + switch (props.which()) { + case config::ServiceDesignator::Props::EMPTY: + return kj::none; + case config::ServiceDesignator::Props::JSON: + return props.getJson(); + } + reportConfigError(kj::str(errorContext, + " has unrecognized props type. Was the config compiled with a " + "newer version of the schema?")); + return kj::none; + }(); + + if (WorkerService* worker = dynamic_cast(service)) { + KJ_IF_SOME(ep, worker->getEntrypoint(entrypointName, propsJson)) { + return kj::mv(ep); + } else KJ_IF_SOME(ep, entrypointName) { + reportConfigError(kj::str(errorContext, " refers to service \"", targetName, + "\" with a named entrypoint \"", ep, "\", but \"", targetName, + "\" has no such named entrypoint.")); + return fakeOwn(*invalidConfigServiceSingleton); } else { reportConfigError(kj::str(errorContext, " refers to service \"", targetName, - "\" with a named entrypoint \"", entrypointName, "\", but \"", targetName, - "\" is not a Worker, so does not have any " - "named entrypoints.")); + "\", but does not specify an entrypoint, and the service does not have a " + "default entrypoint.")); return fakeOwn(*invalidConfigServiceSingleton); } } else { + KJ_IF_SOME(ep, entrypointName) { + reportConfigError(kj::str(errorContext, " refers to service \"", targetName, + "\" with a named entrypoint \"", ep, "\", but \"", targetName, + "\" is not a Worker, so does not have any named entrypoints.")); + } else if (propsJson != kj::none) { + reportConfigError(kj::str(errorContext, " refers to service \"", targetName, + "\" and provides a `props` value, but \"", targetName, + "\" is not a Worker, so cannot accept `props`")); + } + return fakeOwn(*service); } } @@ -4072,7 +4114,7 @@ kj::Promise Server::test(jsg::V8System& v8System, if (WorkerService* worker = dynamic_cast(service.value.get())) { for (auto& name: worker->getEntrypointNames()) { if (entrypointGlob.matches(name)) { - kj::Own ep = KJ_ASSERT_NONNULL(worker->getEntrypoint(name)); + kj::Own ep = KJ_ASSERT_NONNULL(worker->getEntrypoint(name, kj::none)); if (ep->hasHandler("test"_kj)) { co_await doTest(*ep, kj::str(service.key, ':', name)); } diff --git a/src/workerd/server/workerd.capnp b/src/workerd/server/workerd.capnp index 693a4f2cde0..cbf77bfc4ed 100644 --- a/src/workerd/server/workerd.capnp +++ b/src/workerd/server/workerd.capnp @@ -192,6 +192,16 @@ struct ServiceDesignator { # `entrypoint` is specified here, it names an alternate entrypoint to use on the target worker, # otherwise the default is used. + props :union { + # Value to provide in `ctx.props` in the target worker. + + empty @2 :Void; + # Empty object. (This is the default.) + + json @3 :Text; + # A JSON-encoded value. + } + # TODO(someday): Options to specify which event types are allowed. # TODO(someday): Allow adding an outgoing middleware stack here (see TODO in Service, above). } diff --git a/src/workerd/tests/test-fixture.c++ b/src/workerd/tests/test-fixture.c++ index 22aad93051f..8834f5c3123 100644 --- a/src/workerd/tests/test-fixture.c++ +++ b/src/workerd/tests/test-fixture.c++ @@ -425,7 +425,7 @@ TestFixture::Response TestFixture::runRequest( runInIoContext([&](const TestFixture::Environment& env) { auto& globalScope = env.lock.getGlobalScope(); return globalScope.request(method, url, requestHeaders, *requestBody, response, "{}"_kj, - env.lock, env.lock.getExportedHandler(kj::none, kj::none)); + env.lock, env.lock.getExportedHandler(kj::none, {}, kj::none)); }); return {.statusCode = response.statusCode, .body = response.body->str()}; From c2cce700cf6e6565ab434270d7cee49aa4d3d69d Mon Sep 17 00:00:00 2001 From: Kenton Varda Date: Fri, 6 Dec 2024 14:55:52 -0600 Subject: [PATCH 5/5] Update types snapshot to include `ctx.props`. --- types/generated-snapshot/2021-11-03/index.d.ts | 1 + types/generated-snapshot/2021-11-03/index.ts | 1 + types/generated-snapshot/2022-01-31/index.d.ts | 1 + types/generated-snapshot/2022-01-31/index.ts | 1 + types/generated-snapshot/2022-03-21/index.d.ts | 1 + types/generated-snapshot/2022-03-21/index.ts | 1 + types/generated-snapshot/2022-08-04/index.d.ts | 1 + types/generated-snapshot/2022-08-04/index.ts | 1 + types/generated-snapshot/2022-10-31/index.d.ts | 1 + types/generated-snapshot/2022-10-31/index.ts | 1 + types/generated-snapshot/2022-11-30/index.d.ts | 1 + types/generated-snapshot/2022-11-30/index.ts | 1 + types/generated-snapshot/2023-03-01/index.d.ts | 1 + types/generated-snapshot/2023-03-01/index.ts | 1 + types/generated-snapshot/2023-07-01/index.d.ts | 1 + types/generated-snapshot/2023-07-01/index.ts | 1 + types/generated-snapshot/experimental/index.d.ts | 1 + types/generated-snapshot/experimental/index.ts | 1 + types/generated-snapshot/oldest/index.d.ts | 1 + types/generated-snapshot/oldest/index.ts | 1 + 20 files changed, 20 insertions(+) diff --git a/types/generated-snapshot/2021-11-03/index.d.ts b/types/generated-snapshot/2021-11-03/index.d.ts index af5bd8fdfbf..c9c1f6f2135 100755 --- a/types/generated-snapshot/2021-11-03/index.d.ts +++ b/types/generated-snapshot/2021-11-03/index.d.ts @@ -392,6 +392,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2021-11-03/index.ts b/types/generated-snapshot/2021-11-03/index.ts index bcc6bfdea5a..68327474529 100755 --- a/types/generated-snapshot/2021-11-03/index.ts +++ b/types/generated-snapshot/2021-11-03/index.ts @@ -394,6 +394,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2022-01-31/index.d.ts b/types/generated-snapshot/2022-01-31/index.d.ts index 01fc53bf7ea..c2e5fdd92ee 100755 --- a/types/generated-snapshot/2022-01-31/index.d.ts +++ b/types/generated-snapshot/2022-01-31/index.d.ts @@ -392,6 +392,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2022-01-31/index.ts b/types/generated-snapshot/2022-01-31/index.ts index 1ce1c48dc6a..635c0a2a8c6 100755 --- a/types/generated-snapshot/2022-01-31/index.ts +++ b/types/generated-snapshot/2022-01-31/index.ts @@ -394,6 +394,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2022-03-21/index.d.ts b/types/generated-snapshot/2022-03-21/index.d.ts index 4b73489f817..c9cb2f3e095 100755 --- a/types/generated-snapshot/2022-03-21/index.d.ts +++ b/types/generated-snapshot/2022-03-21/index.d.ts @@ -395,6 +395,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2022-03-21/index.ts b/types/generated-snapshot/2022-03-21/index.ts index a23b1e72dac..8fcffcecaed 100755 --- a/types/generated-snapshot/2022-03-21/index.ts +++ b/types/generated-snapshot/2022-03-21/index.ts @@ -397,6 +397,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2022-08-04/index.d.ts b/types/generated-snapshot/2022-08-04/index.d.ts index ea2898b3657..8a76cc60163 100755 --- a/types/generated-snapshot/2022-08-04/index.d.ts +++ b/types/generated-snapshot/2022-08-04/index.d.ts @@ -395,6 +395,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2022-08-04/index.ts b/types/generated-snapshot/2022-08-04/index.ts index 1bc6008d06c..5cf2acefbbd 100755 --- a/types/generated-snapshot/2022-08-04/index.ts +++ b/types/generated-snapshot/2022-08-04/index.ts @@ -397,6 +397,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2022-10-31/index.d.ts b/types/generated-snapshot/2022-10-31/index.d.ts index a58d8c6e503..78464dfa0fe 100755 --- a/types/generated-snapshot/2022-10-31/index.d.ts +++ b/types/generated-snapshot/2022-10-31/index.d.ts @@ -395,6 +395,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2022-10-31/index.ts b/types/generated-snapshot/2022-10-31/index.ts index cb6feb9fde0..cd9ac840a6f 100755 --- a/types/generated-snapshot/2022-10-31/index.ts +++ b/types/generated-snapshot/2022-10-31/index.ts @@ -397,6 +397,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2022-11-30/index.d.ts b/types/generated-snapshot/2022-11-30/index.d.ts index 48406bac695..3a32f4de476 100755 --- a/types/generated-snapshot/2022-11-30/index.d.ts +++ b/types/generated-snapshot/2022-11-30/index.d.ts @@ -400,6 +400,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2022-11-30/index.ts b/types/generated-snapshot/2022-11-30/index.ts index 601de52bb5a..856adf5bc8e 100755 --- a/types/generated-snapshot/2022-11-30/index.ts +++ b/types/generated-snapshot/2022-11-30/index.ts @@ -402,6 +402,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2023-03-01/index.d.ts b/types/generated-snapshot/2023-03-01/index.d.ts index fb3a812a0a7..d37a10dfa4d 100755 --- a/types/generated-snapshot/2023-03-01/index.d.ts +++ b/types/generated-snapshot/2023-03-01/index.d.ts @@ -400,6 +400,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2023-03-01/index.ts b/types/generated-snapshot/2023-03-01/index.ts index c014ca490bc..acb45d143b6 100755 --- a/types/generated-snapshot/2023-03-01/index.ts +++ b/types/generated-snapshot/2023-03-01/index.ts @@ -402,6 +402,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/2023-07-01/index.d.ts b/types/generated-snapshot/2023-07-01/index.d.ts index 6db3a0ec236..1ff193e4500 100755 --- a/types/generated-snapshot/2023-07-01/index.d.ts +++ b/types/generated-snapshot/2023-07-01/index.d.ts @@ -400,6 +400,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/2023-07-01/index.ts b/types/generated-snapshot/2023-07-01/index.ts index e831fc15830..91ba7afb030 100755 --- a/types/generated-snapshot/2023-07-01/index.ts +++ b/types/generated-snapshot/2023-07-01/index.ts @@ -402,6 +402,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown, diff --git a/types/generated-snapshot/experimental/index.d.ts b/types/generated-snapshot/experimental/index.d.ts index d7e15a66da7..517ddb33076 100755 --- a/types/generated-snapshot/experimental/index.d.ts +++ b/types/generated-snapshot/experimental/index.d.ts @@ -400,6 +400,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; abort(reason?: any): void; } type ExportedHandlerFetchHandler = ( diff --git a/types/generated-snapshot/experimental/index.ts b/types/generated-snapshot/experimental/index.ts index cb1bcd71ab3..80fdc8603a8 100755 --- a/types/generated-snapshot/experimental/index.ts +++ b/types/generated-snapshot/experimental/index.ts @@ -402,6 +402,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; abort(reason?: any): void; } export type ExportedHandlerFetchHandler< diff --git a/types/generated-snapshot/oldest/index.d.ts b/types/generated-snapshot/oldest/index.d.ts index 88d32434d95..976c44f1855 100755 --- a/types/generated-snapshot/oldest/index.d.ts +++ b/types/generated-snapshot/oldest/index.d.ts @@ -392,6 +392,7 @@ interface TestController {} interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } type ExportedHandlerFetchHandler = ( request: Request>, diff --git a/types/generated-snapshot/oldest/index.ts b/types/generated-snapshot/oldest/index.ts index 91933733739..809483a2751 100755 --- a/types/generated-snapshot/oldest/index.ts +++ b/types/generated-snapshot/oldest/index.ts @@ -394,6 +394,7 @@ export interface TestController {} export interface ExecutionContext { waitUntil(promise: Promise): void; passThroughOnException(): void; + props: any; } export type ExportedHandlerFetchHandler< Env = unknown,