Commit bf5dbebf authored by Kenton Varda's avatar Kenton Varda

Define additional type aliases AnswerId and ImportId. In the RPC protocol,…

Define additional type aliases AnswerId and ImportId.  In the RPC protocol, always use the type corresponding to the sender's point of view.  In the implementation, always use the type corresponding to the local point of view.
parent b15789d8
...@@ -39,6 +39,9 @@ namespace { ...@@ -39,6 +39,9 @@ namespace {
class RpcDumper { class RpcDumper {
// Class which stringifies RPC messages for debugging purposes, including decoding params and // Class which stringifies RPC messages for debugging purposes, including decoding params and
// results based on the call's interface and method IDs and extracting cap descriptors. // results based on the call's interface and method IDs and extracting cap descriptors.
//
// TODO(cleanup): Clean this code up and move it to someplace reusable, so it can be used as
// a packet inspector / debugging tool for Cap'n Proto network traffic.
public: public:
void addSchema(InterfaceSchema schema) { void addSchema(InterfaceSchema schema) {
...@@ -103,7 +106,7 @@ public: ...@@ -103,7 +106,7 @@ public:
auto ret = message.getReturn(); auto ret = message.getReturn();
auto iter = returnTypes.find( auto iter = returnTypes.find(
std::make_pair(sender == CLIENT ? SERVER : CLIENT, ret.getQuestionId())); std::make_pair(sender == CLIENT ? SERVER : CLIENT, ret.getAnswerId()));
if (iter == returnTypes.end()) { if (iter == returnTypes.end()) {
break; break;
} }
...@@ -129,11 +132,11 @@ public: ...@@ -129,11 +132,11 @@ public:
if (schema.getProto().isStruct()) { if (schema.getProto().isStruct()) {
auto results = kj::str(imbued.getAs<DynamicStruct>(schema.asStruct())); auto results = kj::str(imbued.getAs<DynamicStruct>(schema.asStruct()));
return kj::str(senderName, "(", ret.getQuestionId(), "): return ", results, return kj::str(senderName, "(", ret.getAnswerId(), "): return ", results,
" caps:[", kj::strArray(payload.getCapTable(), ", "), "]"); " caps:[", kj::strArray(payload.getCapTable(), ", "), "]");
} else if (schema.getProto().isInterface()) { } else if (schema.getProto().isInterface()) {
imbued.getAs<DynamicCapability>(schema.asInterface()); imbued.getAs<DynamicCapability>(schema.asInterface());
return kj::str(senderName, "(", ret.getQuestionId(), "): return cap ", return kj::str(senderName, "(", ret.getAnswerId(), "): return cap ",
kj::strArray(payload.getCapTable(), ", ")); kj::strArray(payload.getCapTable(), ", "));
} else { } else {
break; break;
......
...@@ -337,7 +337,7 @@ public: ...@@ -337,7 +337,7 @@ public:
} }
}); });
answers.forEach([&](QuestionId id, Answer& answer) { answers.forEach([&](AnswerId id, Answer& answer) {
KJ_IF_MAYBE(p, answer.pipeline) { KJ_IF_MAYBE(p, answer.pipeline) {
pipelinesToRelease.add(kj::mv(*p)); pipelinesToRelease.add(kj::mv(*p));
} }
...@@ -357,7 +357,7 @@ public: ...@@ -357,7 +357,7 @@ public:
exp = Export(); exp = Export();
}); });
imports.forEach([&](ExportId id, Import& import) { imports.forEach([&](ImportId id, Import& import) {
KJ_IF_MAYBE(f, import.promiseFulfiller) { KJ_IF_MAYBE(f, import.promiseFulfiller) {
f->get()->reject(kj::cp(networkException)); f->get()->reject(kj::cp(networkException));
} }
...@@ -399,7 +399,18 @@ private: ...@@ -399,7 +399,18 @@ private:
// We have to define these before we can define the class's fields. // We have to define these before we can define the class's fields.
typedef uint32_t QuestionId; typedef uint32_t QuestionId;
typedef QuestionId AnswerId;
typedef uint32_t ExportId; typedef uint32_t ExportId;
typedef ExportId ImportId;
// See equivalent definitions in rpc.capnp.
//
// We always use the type that refers to the local table of the same name. So e.g. although
// QuestionId and AnswerId are the same type, we use QuestionId when referring to an entry in
// the local question table (which corresponds to the peer's answer table) and use AnswerId
// to refer to an entry in our answer table (which corresponds to the peer's question table).
// Since all messages in the RPC protocol are defined from the sender's point of view, this
// means that any time we read an ID from a received message, its type should invert.
// TODO(cleanup): Perhaps we could enforce that in a type-safe way? Hmm...
struct Question { struct Question {
kj::Array<ExportId> paramExports; kj::Array<ExportId> paramExports;
...@@ -503,8 +514,8 @@ private: ...@@ -503,8 +514,8 @@ private:
ExportTable<ExportId, Export> exports; ExportTable<ExportId, Export> exports;
ExportTable<QuestionId, Question> questions; ExportTable<QuestionId, Question> questions;
ImportTable<QuestionId, Answer> answers; ImportTable<AnswerId, Answer> answers;
ImportTable<ExportId, Import> imports; ImportTable<ImportId, Import> imports;
// The Four Tables! // The Four Tables!
// The order of the tables is important for correct destruction. // The order of the tables is important for correct destruction.
...@@ -598,7 +609,7 @@ private: ...@@ -598,7 +609,7 @@ private:
// A ClientHook that wraps an entry in the import table. // A ClientHook that wraps an entry in the import table.
public: public:
ImportClient(RpcConnectionState& connectionState, ExportId importId) ImportClient(RpcConnectionState& connectionState, ImportId importId)
: RpcClient(connectionState), importId(importId) {} : RpcClient(connectionState), importId(importId) {}
~ImportClient() noexcept(false) { ~ImportClient() noexcept(false) {
...@@ -636,7 +647,7 @@ private: ...@@ -636,7 +647,7 @@ private:
kj::Maybe<kj::Own<ClientHook>> writeTarget( kj::Maybe<kj::Own<ClientHook>> writeTarget(
rpc::MessageTarget::Builder target) override { rpc::MessageTarget::Builder target) override {
target.setExportedCap(importId); target.setImportedCap(importId);
return nullptr; return nullptr;
} }
...@@ -655,7 +666,7 @@ private: ...@@ -655,7 +666,7 @@ private:
} }
private: private:
ExportId importId; ImportId importId;
uint remoteRefcount = 0; uint remoteRefcount = 0;
// Number of times we've received this import from the peer. // Number of times we've received this import from the peer.
...@@ -715,7 +726,7 @@ private: ...@@ -715,7 +726,7 @@ private:
PromiseClient(RpcConnectionState& connectionState, PromiseClient(RpcConnectionState& connectionState,
kj::Own<ClientHook> initial, kj::Own<ClientHook> initial,
kj::Promise<kj::Own<ClientHook>> eventual, kj::Promise<kj::Own<ClientHook>> eventual,
kj::Maybe<ExportId> importId) kj::Maybe<ImportId> importId)
: RpcClient(connectionState), : RpcClient(connectionState),
isResolved(false), isResolved(false),
cap(kj::mv(initial)), cap(kj::mv(initial)),
...@@ -801,7 +812,7 @@ private: ...@@ -801,7 +812,7 @@ private:
bool isResolved; bool isResolved;
kj::Own<ClientHook> cap; kj::Own<ClientHook> cap;
kj::Maybe<ExportId> importId; kj::Maybe<ImportId> importId;
kj::ForkedPromise<kj::Own<ClientHook>> fork; kj::ForkedPromise<kj::Own<ClientHook>> fork;
// Keep this last, because the continuation uses *this, so it should be destroyed first to // Keep this last, because the continuation uses *this, so it should be destroyed first to
...@@ -1017,7 +1028,7 @@ private: ...@@ -1017,7 +1028,7 @@ private:
// ===================================================================================== // =====================================================================================
// Interpreting CapDescriptor // Interpreting CapDescriptor
kj::Own<ClientHook> import(ExportId importId, bool isPromise) { kj::Own<ClientHook> import(ImportId importId, bool isPromise) {
// Receive a new import. // Receive a new import.
auto& import = imports[importId]; auto& import = imports[importId];
...@@ -1515,12 +1526,12 @@ private: ...@@ -1515,12 +1526,12 @@ private:
class RpcCallContext final: public CallContextHook, public kj::Refcounted { class RpcCallContext final: public CallContextHook, public kj::Refcounted {
public: public:
RpcCallContext(RpcConnectionState& connectionState, QuestionId questionId, RpcCallContext(RpcConnectionState& connectionState, AnswerId answerId,
kj::Own<IncomingRpcMessage>&& request, const AnyPointer::Reader& params, kj::Own<IncomingRpcMessage>&& request, const AnyPointer::Reader& params,
kj::Array<kj::Own<ClientHook>>&& requestCapTable, bool redirectResults, kj::Array<kj::Own<ClientHook>>&& requestCapTable, bool redirectResults,
kj::Own<kj::PromiseFulfiller<void>>&& cancelFulfiller) kj::Own<kj::PromiseFulfiller<void>>&& cancelFulfiller)
: connectionState(kj::addRef(connectionState)), : connectionState(kj::addRef(connectionState)),
questionId(questionId), answerId(answerId),
request(kj::mv(request)), request(kj::mv(request)),
requestCapContext(kj::mv(requestCapTable)), requestCapContext(kj::mv(requestCapTable)),
params(requestCapContext.imbue(params)), params(requestCapContext.imbue(params)),
...@@ -1538,7 +1549,7 @@ private: ...@@ -1538,7 +1549,7 @@ private:
messageSizeHint<rpc::Return>() + sizeInWords<rpc::Payload>()); messageSizeHint<rpc::Return>() + sizeInWords<rpc::Payload>());
auto builder = message->getBody().initAs<rpc::Message>().initReturn(); auto builder = message->getBody().initAs<rpc::Message>().initReturn();
builder.setQuestionId(questionId); builder.setAnswerId(answerId);
builder.setReleaseParamCaps(false); builder.setReleaseParamCaps(false);
if (redirectResults) { if (redirectResults) {
...@@ -1575,7 +1586,7 @@ private: ...@@ -1575,7 +1586,7 @@ private:
if (!(cancellationFlags & CANCEL_REQUESTED) && isFirstResponder()) { if (!(cancellationFlags & CANCEL_REQUESTED) && isFirstResponder()) {
if (response == nullptr) getResults(MessageSize{0, 0}); // force initialization of response if (response == nullptr) getResults(MessageSize{0, 0}); // force initialization of response
returnMessage.setQuestionId(questionId); returnMessage.setAnswerId(answerId);
returnMessage.setReleaseParamCaps(false); returnMessage.setReleaseParamCaps(false);
auto exports = kj::downcast<RpcServerResponseImpl>(*KJ_ASSERT_NONNULL(response)).send(); auto exports = kj::downcast<RpcServerResponseImpl>(*KJ_ASSERT_NONNULL(response)).send();
...@@ -1595,7 +1606,7 @@ private: ...@@ -1595,7 +1606,7 @@ private:
messageSizeHint<rpc::Return>() + exceptionSizeHint(exception)); messageSizeHint<rpc::Return>() + exceptionSizeHint(exception));
auto builder = message->getBody().initAs<rpc::Message>().initReturn(); auto builder = message->getBody().initAs<rpc::Message>().initReturn();
builder.setQuestionId(questionId); builder.setAnswerId(answerId);
builder.setReleaseParamCaps(false); builder.setReleaseParamCaps(false);
fromException(exception, builder.initException()); fromException(exception, builder.initException());
...@@ -1676,9 +1687,9 @@ private: ...@@ -1676,9 +1687,9 @@ private:
messageSizeHint<rpc::Return>()); messageSizeHint<rpc::Return>());
auto builder = message->getBody().initAs<rpc::Message>().initReturn(); auto builder = message->getBody().initAs<rpc::Message>().initReturn();
builder.setQuestionId(questionId); builder.setAnswerId(answerId);
builder.setReleaseParamCaps(false); builder.setReleaseParamCaps(false);
builder.setTakeFromOtherAnswer(tailInfo->questionId); builder.setTakeFromOtherQuestion(tailInfo->questionId);
message->send(); message->send();
...@@ -1724,7 +1735,7 @@ private: ...@@ -1724,7 +1735,7 @@ private:
private: private:
kj::Own<RpcConnectionState> connectionState; kj::Own<RpcConnectionState> connectionState;
QuestionId questionId; AnswerId answerId;
// Request --------------------------------------------- // Request ---------------------------------------------
...@@ -1777,10 +1788,10 @@ private: ...@@ -1777,10 +1788,10 @@ private:
// Already received `Finish` so it's our job to erase the table entry. We shouldn't have // Already received `Finish` so it's our job to erase the table entry. We shouldn't have
// sent results if canceled, so we shouldn't have an export list to deal with. // sent results if canceled, so we shouldn't have an export list to deal with.
KJ_ASSERT(resultExports.size() == 0); KJ_ASSERT(resultExports.size() == 0);
connectionState->answers.erase(questionId); connectionState->answers.erase(answerId);
} else { } else {
// We just have to null out callContext and set the exports. // We just have to null out callContext and set the exports.
auto& answer = connectionState->answers[questionId]; auto& answer = connectionState->answers[answerId];
answer.callContext = nullptr; answer.callContext = nullptr;
answer.resultExports = kj::mv(resultExports); answer.resultExports = kj::mv(resultExports);
...@@ -1930,16 +1941,16 @@ private: ...@@ -1930,16 +1941,16 @@ private:
auto capTable = receiveCaps(payload.getCapTable()); auto capTable = receiveCaps(payload.getCapTable());
auto cancelPaf = kj::newPromiseAndFulfiller<void>(); auto cancelPaf = kj::newPromiseAndFulfiller<void>();
QuestionId questionId = call.getQuestionId(); AnswerId answerId = call.getQuestionId();
auto context = kj::refcounted<RpcCallContext>( auto context = kj::refcounted<RpcCallContext>(
*this, questionId, kj::mv(message), payload.getContent(), *this, answerId, kj::mv(message), payload.getContent(),
kj::mv(capTable), redirectResults, kj::mv(cancelPaf.fulfiller)); kj::mv(capTable), redirectResults, kj::mv(cancelPaf.fulfiller));
// No more using `call` after this point, as it now belongs to the context. // No more using `call` after this point, as it now belongs to the context.
{ {
auto& answer = answers[questionId]; auto& answer = answers[answerId];
KJ_REQUIRE(!answer.active, "questionId is already in use") { KJ_REQUIRE(!answer.active, "questionId is already in use") {
return; return;
...@@ -1956,7 +1967,7 @@ private: ...@@ -1956,7 +1967,7 @@ private:
// context->directTailCall(). // context->directTailCall().
{ {
auto& answer = answers[questionId]; auto& answer = answers[answerId];
answer.pipeline = kj::mv(promiseAndPipeline.pipeline); answer.pipeline = kj::mv(promiseAndPipeline.pipeline);
...@@ -1998,8 +2009,8 @@ private: ...@@ -1998,8 +2009,8 @@ private:
kj::Maybe<kj::Own<ClientHook>> getMessageTarget(const rpc::MessageTarget::Reader& target) { kj::Maybe<kj::Own<ClientHook>> getMessageTarget(const rpc::MessageTarget::Reader& target) {
switch (target.which()) { switch (target.which()) {
case rpc::MessageTarget::EXPORTED_CAP: { case rpc::MessageTarget::IMPORTED_CAP: {
KJ_IF_MAYBE(exp, exports.find(target.getExportedCap())) { KJ_IF_MAYBE(exp, exports.find(target.getImportedCap())) {
return exp->clientHook->addRef(); return exp->clientHook->addRef();
} else { } else {
KJ_FAIL_REQUIRE("Message target is not a current export ID.") { KJ_FAIL_REQUIRE("Message target is not a current export ID.") {
...@@ -2050,7 +2061,7 @@ private: ...@@ -2050,7 +2061,7 @@ private:
KJ_DEFER(releaseExports(exportsToRelease)); KJ_DEFER(releaseExports(exportsToRelease));
kj::Maybe<kj::Promise<kj::Own<RpcResponse>>> promiseToRelease; kj::Maybe<kj::Promise<kj::Own<RpcResponse>>> promiseToRelease;
KJ_IF_MAYBE(question, questions.find(ret.getQuestionId())) { KJ_IF_MAYBE(question, questions.find(ret.getAnswerId())) {
KJ_REQUIRE(question->isAwaitingReturn, "Duplicate Return.") { return; } KJ_REQUIRE(question->isAwaitingReturn, "Duplicate Return.") { return; }
question->isAwaitingReturn = false; question->isAwaitingReturn = false;
...@@ -2098,8 +2109,8 @@ private: ...@@ -2098,8 +2109,8 @@ private:
questionRef->fulfill(kj::Own<RpcResponse>()); questionRef->fulfill(kj::Own<RpcResponse>());
break; break;
case rpc::Return::TAKE_FROM_OTHER_ANSWER: case rpc::Return::TAKE_FROM_OTHER_QUESTION:
KJ_IF_MAYBE(answer, answers.find(ret.getTakeFromOtherAnswer())) { KJ_IF_MAYBE(answer, answers.find(ret.getTakeFromOtherQuestion())) {
KJ_IF_MAYBE(response, answer->redirectedResults) { KJ_IF_MAYBE(response, answer->redirectedResults) {
questionRef->fulfill(kj::mv(*response)); questionRef->fulfill(kj::mv(*response));
} else { } else {
...@@ -2116,16 +2127,16 @@ private: ...@@ -2116,16 +2127,16 @@ private:
KJ_FAIL_REQUIRE("Unknown 'Return' type.") { return; } KJ_FAIL_REQUIRE("Unknown 'Return' type.") { return; }
} }
} else { } else {
if (ret.isTakeFromOtherAnswer()) { if (ret.isTakeFromOtherQuestion()) {
// Be sure to release the tail call's promise. // Be sure to release the tail call's promise.
KJ_IF_MAYBE(answer, answers.find(ret.getTakeFromOtherAnswer())) { KJ_IF_MAYBE(answer, answers.find(ret.getTakeFromOtherQuestion())) {
promiseToRelease = kj::mv(answer->redirectedResults); promiseToRelease = kj::mv(answer->redirectedResults);
} }
} }
// Looks like this question was canceled earlier, so `Finish` was already sent. We can go // Looks like this question was canceled earlier, so `Finish` was already sent. We can go
// ahead and delete it from the table. // ahead and delete it from the table.
questions.erase(ret.getQuestionId(), *question); questions.erase(ret.getAnswerId(), *question);
} }
} else { } else {
...@@ -2328,13 +2339,13 @@ private: ...@@ -2328,13 +2339,13 @@ private:
}; };
void handleRestore(kj::Own<IncomingRpcMessage>&& message, const rpc::Restore::Reader& restore) { void handleRestore(kj::Own<IncomingRpcMessage>&& message, const rpc::Restore::Reader& restore) {
QuestionId questionId = restore.getQuestionId(); AnswerId answerId = restore.getQuestionId();
auto response = connection->newOutgoingMessage( auto response = connection->newOutgoingMessage(
messageSizeHint<rpc::Return>() + sizeInWords<rpc::CapDescriptor>() + 32); messageSizeHint<rpc::Return>() + sizeInWords<rpc::CapDescriptor>() + 32);
rpc::Return::Builder ret = response->getBody().getAs<rpc::Message>().initReturn(); rpc::Return::Builder ret = response->getBody().getAs<rpc::Message>().initReturn();
ret.setQuestionId(questionId); ret.setAnswerId(answerId);
CapBuilderContext context; CapBuilderContext context;
...@@ -2365,7 +2376,7 @@ private: ...@@ -2365,7 +2376,7 @@ private:
message = nullptr; message = nullptr;
// Add the answer to the answer table for pipelining and send the response. // Add the answer to the answer table for pipelining and send the response.
auto& answer = answers[questionId]; auto& answer = answers[answerId];
KJ_REQUIRE(!answer.active, "questionId is already in use") { KJ_REQUIRE(!answer.active, "questionId is already in use") {
return; return;
} }
......
...@@ -30,64 +30,74 @@ ...@@ -30,64 +30,74 @@
# capability, the caller can begin calling methods on that capability _before the first call has # capability, the caller can begin calling methods on that capability _before the first call has
# returned_. The caller essentially sends a message saying "Hey server, as soon as you finish # returned_. The caller essentially sends a message saying "Hey server, as soon as you finish
# that previous call, do this with the result!". Cap'n Proto's RPC protocol makes this possible. # that previous call, do this with the result!". Cap'n Proto's RPC protocol makes this possible.
# As a result, it is more complicated than most.
# #
# Cap'n Proto RPC is based heavily on CapTP: # The protocol is significantly more complicated than most RPC protocols. However, this is
# implementation complexity that underlies an easy-to-grasp higher-level model of object oriented
# programming. That is, just like TCP is a surprisingly complicated protocol that implements a
# conceptually-simple byte stream abstraction, Cap'n Proto is a surprisingly complicated protocol
# that implements a conceptually-simple object abstraction.
#
# Cap'n Proto RPC is based heavily on CapTP, the object-capability protocol used by the E
# programming language:
# http://www.erights.org/elib/distrib/captp/index.html # http://www.erights.org/elib/distrib/captp/index.html
# #
# Cap'n Proto RPC takes place between "vats". A vat hosts some set of capabilities and talks to # Cap'n Proto RPC takes place between "vats". A vat hosts some set of objects and talks to other
# other vats through direct bilateral connections. Typically, there is a 1:1 correspondence # vats through direct bilateral connections. Typically, there is a 1:1 correspondence between vats
# between vats and processes (in the unix sense of the word), although this is not strictly always # and processes (in the unix sense of the word), although this is not strictly always true (one
# true (one process could run multiple vats, or a distributed vat might live across many processes). # process could run multiple vats, or a distributed virtual vat might live across many processes).
# #
# Cap'n Proto does not distinguish between "clients" and "servers" -- this is up to the application. # Cap'n Proto does not distinguish between "clients" and "servers" -- this is up to the application.
# Either end of any connection can potentially hold capabilities pointing to the other end, and # Either end of any connection can potentially hold capabilities pointing to the other end, and
# can call methods on those capabilities. In the doc comments below, we use the words "sender" # can call methods on those capabilities. In the doc comments below, we use the words "sender"
# and "receiver". These refer to the sender and receiver of an instance of the struct or field # and "receiver". These refer to the sender and receiver of an instance of the struct or field
# being documented. Sometimes we refer to a "third-party" which is neither the sender nor the # being documented. Sometimes we refer to a "third-party" which is neither the sender nor the
# receiver. # receiver. Documentation is generally written from the point of view of the sender.
# #
# It is generally up to the vat network implementation to securely verify that connections are made # It is generally up to the vat network implementation to securely verify that connections are made
# to the intended vat as well as to encrypt transmitted data for privacy and integrity. See the # to the intended vat as well as to encrypt transmitted data for privacy and integrity. See the
# `VatNetwork` example interface near the end of this file. # `VatNetwork` example interface near the end of this file.
# #
# Once a connection is formed, nothing interesting can happen until one side sends a Restore # Once a connection is formed, nothing interesting can happen until one side sends a Restore
# message to obtain a persistent capability. # message to convert a persistent capability reference into a live one.
# #
# Unless otherwise specified, messages must be delivered to the receiving application in the same # Unless otherwise specified, messages must be delivered to the receiving application in the same
# order in which they were initiated by the sending application, just like in E: # order in which they were initiated by the sending application. The goal is to support "E-Order",
# which states that two calls made on the same reference must be delivered in the order which they
# were made:
# http://erights.org/elib/concurrency/partial-order.html # http://erights.org/elib/concurrency/partial-order.html
# #
# Since the full protocol is complicated, we define multiple levels of support which an # Since the full protocol is complicated, we define multiple levels of support which an
# implementation may target. For typical applications, level 1 support will be sufficient. # implementation may target. For many applications, level 1 support will be sufficient.
# Comments in this file indicate which level requires the corresponding feature to be # Comments in this file indicate which level requires the corresponding feature to be
# implemented. # implemented.
# #
# * **Level 0:** The implementation does not support object references. `Restore` is supported # * **Level 0:** The implementation does not support object references. `Restore` is supported
# only for looking up singleton objects which exist for the lifetime of the server, and only # only for looking up singleton objects which exist for the lifetime of the server, and only
# these singleton objects can receive calls. At this level, the implementation does not support # these singleton objects can receive calls. At this level, the implementation does not support
# object-oriented protocols and is similar in complexity to JSON-RPC or Protobuf "generic # object-oriented protocols and is similar in complexity to JSON-RPC or Protobuf services. This
# services". This level should be considered only a temporary stepping-stone toward level 1 as # level should be considered only a temporary stepping-stone toward level 1 as the lack of object
# the lack of object references drastically changes how protocols are designed. Applications # references drastically changes how protocols are designed. Applications _should not_ attempt
# _should not_ attempt to design their protocols around the limitations of level 0 # to design their protocols around the limitations of level 0 implementations.
# implementations.
# #
# * **Level 1:** The implementation supports simple bilateral interaction with object references # * **Level 1:** The implementation supports simple bilateral interaction with object references
# and promise pipelining, but interactions between three or more parties are supported only via # and promise pipelining, but interactions between three or more parties are supported only via
# proxying of objects. E.g. if Alice wants to send Bob a capability pointing to Carol, Alice # proxying of objects. E.g. if Alice (in Vat A) wants to send Bob (in Vat B) a capability
# must host a local proxy of Carol and send Bob a reference to that; Bob cannot form a direct # pointing to Carol (in Vat C), Alice must create a proxy of Carol within Vat A and send Bob a
# connection to Carol. Level 1 implementations do not support "join" or "eq" across capabilities # reference to that; Bob cannot form a direct connection to Carol. Level 1 implementations do
# received from different vats, although they should be supported on capabilities received from # not support checking if two capabilities received from different vats actually point to the
# the same vat. `Restore` is supported only for looking up singleton objects as in level 0. # same object ("join"), although they should be able to do this check on capabilities received
# from the same vat. `Restore` is supported only for looking up singleton objects as in level 0.
# #
# * **Level 2:** The implementation supports saving, restoring, and deleting persistent # * **Level 2:** The implementation supports saving, restoring, and deleting persistent
# capabilities. # capabilities -- i.e. capabilities that remain valid even after disconnect, and can be restored
# on a future connection.
# #
# * **Level 3:** The implementation supports three-way interactions but does not implement "Join" # * **Level 3:** The implementation supports three-way interactions. That is, if Alice (in Vat A)
# operations. The implementation can be used effectively on networks that do not require joins, # sends Bob (in Vat B) a capability pointing to Carol (in Vat C), then Vat B will automatically
# or to implement objects that never need to be joined. # form a direct connection to Vat C rather than have requests be proxied through Vat A.
# #
# * **Level 4:** The entire protocol is implemented, including joins. # * **Level 4:** The entire protocol is implemented, including joins (checking if two capabilities
# are equivalent).
# #
# Note that an implementation must also support specific networks (transports), as described in # Note that an implementation must also support specific networks (transports), as described in
# the "Network-specific Parameters" section below. An implementation might have different levels # the "Network-specific Parameters" section below. An implementation might have different levels
...@@ -128,7 +138,7 @@ $Cxx.namespace("capnp::rpc"); ...@@ -128,7 +138,7 @@ $Cxx.namespace("capnp::rpc");
# object living in the exporter's vat, or they may be "promises", meaning the exported object is # object living in the exporter's vat, or they may be "promises", meaning the exported object is
# the as-yet-unknown result of an ongoing operation and will eventually be resolved to some other # the as-yet-unknown result of an ongoing operation and will eventually be resolved to some other
# object once that operation completes. Calls made to a promise will be forwarded to the eventual # object once that operation completes. Calls made to a promise will be forwarded to the eventual
# target once it is known. The eventual replacement object does *not* take the same ID as the # target once it is known. The eventual replacement object does *not* get the same ID as the
# promise, as it may turn out to be an object that is already exported (so already has an ID) or # promise, as it may turn out to be an object that is already exported (so already has an ID) or
# may even live in a completely different vat (and so won't get an ID on the same export table # may even live in a completely different vat (and so won't get an ID on the same export table
# at all). # at all).
...@@ -139,45 +149,64 @@ $Cxx.namespace("capnp::rpc"); ...@@ -139,45 +149,64 @@ $Cxx.namespace("capnp::rpc");
# released by both sides. # released by both sides.
# #
# When a Cap'n Proto connection is lost, everything on the four tables is lost. All questions are # When a Cap'n Proto connection is lost, everything on the four tables is lost. All questions are
# canceled and throw exceptions. All imports become broken (all methods throw exceptions). All # canceled and throw exceptions. All imports become broken (all future calls to them throw
# exports and answers are implicitly released. The only things not lost are persistent # exceptions). All exports and answers are implicitly released. The only things not lost are
# capabilities (`SturdyRef`s). The application must plan for this and should respond by # persistent capabilities (`SturdyRef`s). The application must plan for this and should respond by
# establishing a new connection and restoring from these persistent capabilities. # establishing a new connection and restoring from these persistent capabilities.
using QuestionId = UInt32; using QuestionId = UInt32;
# **(level 0)** # **(level 0)**
# #
# Identifies a question in the questions/answers table. The questioner (caller) chooses an ID # Identifies a question in the sender's question table (which corresponds to the receiver's answer
# when making a call. The ID remains valid in caller -> callee messages until a Finish # table). The questioner (caller) chooses an ID when making a call. The ID remains valid in
# message is sent, and remains valid in callee -> caller messages until a Return message is sent. # caller -> callee messages until a Finish message is sent, and remains valid in callee -> caller
# messages until a Return message is sent.
using AnswerId = QuestionId;
# **(level 0)**
#
# Identifies an answer in the sender's answer table (which corresponds to the receiver's question
# table).
#
# AnswerId is physically equivalent to QuestionId, since the question and answer tables correspond,
# but we define a separate type for documentation purposes: we always use the type representing
# the sender's point of view.
using ExportId = UInt32; using ExportId = UInt32;
# **(level 1)** # **(level 1)**
# #
# Identifies an exported capability or promise in the exports/imports table. The exporter chooses # Identifies an exported capability or promise in the sender's export table (which corresponds
# an ID before sending a capability over the wire. If the capability is already in the table, the # to the receiver's import table). The exporter chooses an ID before sending a capability over the
# exporter should reuse the same ID. If the ID is a promise (as opposed to a settled capability), # wire. If the capability is already in the table, the exporter should reuse the same ID. If the
# this must be indicated at the time the ID is introduced; in this case, the importer shall expect # ID is a promise (as opposed to a settled capability), this must be indicated at the time the ID
# a later Resolve message which replaces the promise. # is introduced (e.g. by using `senderPromise` instead of `senderHosted` in `CapDescriptor`); in
# this case, the importer shall expect a later `Resolve` message which replaces the promise.
#
# ExportId/ImportIds are subject to reference counting. Whenever an `ExportId` is sent over the
# wire (from the exporter to the importer), the export's reference count is incremented (unless
# otherwise specified). The reference count is later decremented by a `Release` message. Since
# the `Release` message can specify an arbitrary number by which to reduce the reference count, the
# importer should usually batch reference decrements and only send a `Release` when it believes the
# reference count has hit zero. Of course, it is possible that a new reference to the export is
# in-flight at the time that the `Release` message is sent, so it is necessary for the exporter to
# keep track of the reference count on its end as well to avoid race conditions.
# #
# ExportIds are subject to reference counting. When an `ExportId` is sent embedded in an # When a connection is lost, all exports are implicitly released. It is not possible to restore
# CapDescriptor, the export's reference count is incremented. The reference count is # a connection state after disconnect (although a transport layer could implement a concept of
# later decremented by a `Release` message. Since the `Release` message can specify an arbitrary # persistent connections if it is transparent to the RPC layer).
# number by which to reduce the reference count, the importer should usually batch reference
# decrements and only send a `Release` when it believes the reference count has hit zero. Of using ImportId = ExportId;
# course, it is possible that a new reference to the released object is in-flight at the time # **(level 1)**
# that the `Release` message is sent, so it is necessary for the exporter to keep track of the
# reference count on its end as well to avoid race conditions.
# #
# When an `ExportId` is received as part of a exporter -> importer message but not embedded in a # Identifies an imported capability or promise in the sender's import table (which corresponds to
# question or answer, its reference count must be incremented unless otherwise specified. # the receiver's export table).
# #
# An `ExportId` remains valid in importer -> exporter messages until its reference count reaches # ImportId is physically equivalent to ExportId, since the export and import tables correspond,
# zero and a `Release` message has been sent to release it. # but we define a separate type for documentation purposes: we always use the type representing
# the sender's point of view.
# #
# When a connection is lost, all exports are implicitly released. It is not possible to restore # An `ImportId` remains valid in importer -> exporter messages until the importer has sent
# a connection state (or, restoration should be implemented at the transport layer without the RPC # `Release` messages which (it believes) have reduced the reference count to zero.
# layer knowing that anything happened).
# ======================================================================================== # ========================================================================================
# Messages # Messages
...@@ -187,17 +216,18 @@ struct Message { ...@@ -187,17 +216,18 @@ struct Message {
union { union {
unimplemented @0 :Message; unimplemented @0 :Message;
# When a peer receives a message of a type it doesn't recognize or doesn't support, it # The sender previously received this message from the peer but didn't understand it or doesn't
# must immediately echo the message back to the sender in `unimplemented`. The sender is # yet implement the functionality that was requested. So, the sender is echoing the message
# then able to examine the message and decide how to deal with it being unimplemented. # back. In some cases, the receiver may be able to recover from this by pretending the sender
# had taken some appropriate "null" action.
# #
# For example, say `resolve` is received by a level 0 implementation (because a previous call # For example, say `resolve` is received by a level 0 implementation (because a previous call
# or return happened to contain a promise). The receiver will echo it back as `unimplemented`. # or return happened to contain a promise). The level 0 implementation will echo it back as
# The sender can then simply release the cap to which the promise had resolved, thus avoiding # `unimplemented`. The original sender can then simply release the cap to which the promise
# a leak. # had resolved, thus avoiding a leak.
# #
# For any message type that introduces a question, if the message comes back unimplemented, # For any message type that introduces a question, if the message comes back unimplemented,
# the sender may simply treat it as if the question failed with an exception. # the original sender may simply treat it as if the question failed with an exception.
# #
# In cases where there is no sensible way to react to an `unimplemented` message (without # In cases where there is no sensible way to react to an `unimplemented` message (without
# resource leaks or other serious problems), the connection may need to be aborted. This is # resource leaks or other serious problems), the connection may need to be aborted. This is
...@@ -340,8 +370,8 @@ struct Return { ...@@ -340,8 +370,8 @@ struct Return {
# #
# Message type sent from callee to caller indicating that the call has completed. # Message type sent from callee to caller indicating that the call has completed.
questionId @0 :QuestionId; answerId @0 :AnswerId;
# Question ID which is being answered, as specified in the corresponding Call. # Equal to the QuestionId of the corresponding `Call` message.
releaseParamCaps @1 :Bool = true; releaseParamCaps @1 :Bool = true;
# If true, all capabilities that were in the params should be considered released. The sender # If true, all capabilities that were in the params should be considered released. The sender
...@@ -371,7 +401,7 @@ struct Return { ...@@ -371,7 +401,7 @@ struct Return {
# This is set when returning from a `Call` which had `sendResultsTo` set to something other # This is set when returning from a `Call` which had `sendResultsTo` set to something other
# than `caller`. # than `caller`.
takeFromOtherAnswer @6 :QuestionId; takeFromOtherQuestion @6 :QuestionId;
# The sender has also sent (before this message) a `Call` with the given question ID and with # The sender has also sent (before this message) a `Call` with the given question ID and with
# `sendResultsTo.yourself` set, and the results of that other call should be used as the # `sendResultsTo.yourself` set, and the results of that other call should be used as the
# results here. # results here.
...@@ -437,7 +467,9 @@ struct Resolve { ...@@ -437,7 +467,9 @@ struct Resolve {
# The ID of the promise to be resolved. # The ID of the promise to be resolved.
# #
# Unlike all other instances of `ExportId` sent from the exporter, the `Resolve` message does # Unlike all other instances of `ExportId` sent from the exporter, the `Resolve` message does
# _not_ increase the reference count of `promiseId`. # _not_ increase the reference count of `promiseId`. In fact, it is expected that the receiver
# will release the export soon after receiving `Resolve`, and the sender will not send this
# `ExportId` again until it has been released and recycled.
# #
# When an export ID sent over the wire (e.g. in a `CapDescriptor`) is indicated to be a promise, # When an export ID sent over the wire (e.g. in a `CapDescriptor`) is indicated to be a promise,
# this indicates that the sender will follow up at some point with a `Resolve` message. If the # this indicates that the sender will follow up at some point with a `Resolve` message. If the
...@@ -447,11 +479,11 @@ struct Resolve { ...@@ -447,11 +479,11 @@ struct Resolve {
# this later promise does _not_ correspond to the earlier `Resolve`. # this later promise does _not_ correspond to the earlier `Resolve`.
# #
# If a promise ID's reference count reaches zero before a `Resolve` is sent, the `Resolve` # If a promise ID's reference count reaches zero before a `Resolve` is sent, the `Resolve`
# message may or may not still be sent (in particular, the `Resolve` may have already been # message may or may not still be sent (the `Resolve` may have already been in-flight when
# in-flight when `Release` was sent). Thus a `Resolve` may be received for a promise of which # `Release` was sent, but if the `Release` is received before `Resolve` then there is no longer
# any reason to send a `Resolve`). Thus a `Resolve` may be received for a promise of which
# the receiver has no knowledge, because it already released it earlier. In this case, the # the receiver has no knowledge, because it already released it earlier. In this case, the
# receiver should immediately release the capability to which the promise resolved, if # receiver should simply release the capability to which the promise resolved.
# applicable.
union { union {
cap @1 :CapDescriptor; cap @1 :CapDescriptor;
...@@ -477,7 +509,7 @@ struct Release { ...@@ -477,7 +509,7 @@ struct Release {
# Message type sent to indicate that the sender is done with the given capability and the receiver # Message type sent to indicate that the sender is done with the given capability and the receiver
# can free resources allocated to it. # can free resources allocated to it.
id @0 :ExportId; id @0 :ImportId;
# What to release. # What to release.
referenceCount @1 :UInt32; referenceCount @1 :UInt32;
...@@ -615,7 +647,8 @@ struct Delete { ...@@ -615,7 +647,8 @@ struct Delete {
# Message type sent to delete a previously-saved persistent capability. In other words, this # Message type sent to delete a previously-saved persistent capability. In other words, this
# means "this ref will no longer be used in the future", so that the host can potentially # means "this ref will no longer be used in the future", so that the host can potentially
# garbage collect resources associated with it. Note that if any ExportId still refers to a # garbage collect resources associated with it. Note that if any ExportId still refers to a
# capability restored from this ref, that export should still remain valid until released. # capability restored from this ref, that export should still remain valid until released -- thus
# `Delete` behaves like POSIX's `unlink()` when called on a file that is currently open.
# #
# Different applications may define different policies regarding saved capability lifetimes that # Different applications may define different policies regarding saved capability lifetimes that
# may or may not rely on `Delete`. For the purpose of implementation freedom, a receiver is # may or may not rely on `Delete`. For the purpose of implementation freedom, a receiver is
...@@ -770,7 +803,7 @@ struct Join { ...@@ -770,7 +803,7 @@ struct Join {
# it receives a `Finish` for one of the join results before the joiner successfully # it receives a `Finish` for one of the join results before the joiner successfully
# connects. It can then free any resources it had allocated as part of the join. # connects. It can then free any resources it had allocated as part of the join.
capId @1 :ExportId; target @1 :MessageTarget;
# The capability to join. # The capability to join.
keyPart @2 :JoinKeyPart; keyPart @2 :JoinKeyPart;
...@@ -791,8 +824,9 @@ struct MessageTarget { ...@@ -791,8 +824,9 @@ struct MessageTarget {
# The target of a `Call` or other messages that target a capability. # The target of a `Call` or other messages that target a capability.
union { union {
exportedCap @0 :ExportId; importedCap @0 :ImportId;
# This message is to a capability or promise previously exported by the receiver. # This message is to a capability or promise previously imported by the caller (exported by
# the receiver).
promisedAnswer @1 :PromisedAnswer; promisedAnswer @1 :PromisedAnswer;
# This message is to a capability that is expected to be returned by another call that has not # This message is to a capability that is expected to be returned by another call that has not
...@@ -846,8 +880,8 @@ struct CapDescriptor { ...@@ -846,8 +880,8 @@ struct CapDescriptor {
# them. If `senderPromise` is released before the `Resolve` is sent, the sender (of this # them. If `senderPromise` is released before the `Resolve` is sent, the sender (of this
# `CapDescriptor`) may choose not to send the `Resolve` at all. # `CapDescriptor`) may choose not to send the `Resolve` at all.
receiverHosted @3 :ExportId; receiverHosted @3 :ImportId;
# A capability (or promise) previously exported by the receiver. # A capability (or promise) previously exported by the receiver (imported by the sender).
receiverAnswer @4 :PromisedAnswer; receiverAnswer @4 :PromisedAnswer;
# A capability expected to be returned in the results of a currently-outstanding call posed # A capability expected to be returned in the results of a currently-outstanding call posed
...@@ -1082,7 +1116,7 @@ using ProvisionId = AnyPointer; ...@@ -1082,7 +1116,7 @@ using ProvisionId = AnyPointer;
# The information which must be sent in an `Accept` message to identify the object being accepted. # The information which must be sent in an `Accept` message to identify the object being accepted.
# #
# In a network where each vat has a public/private key pair, this could simply be the public key # In a network where each vat has a public/private key pair, this could simply be the public key
# fingerprint of the provider vat along with the questionId used in the `Provide` message sent from # fingerprint of the provider vat along with the question ID used in the `Provide` message sent from
# that provider. # that provider.
using RecipientId = AnyPointer; using RecipientId = AnyPointer;
......
...@@ -462,7 +462,7 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = { ...@@ -462,7 +462,7 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
209, 0, 0, 0, 90, 0, 0, 0, 209, 0, 0, 0, 74, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
208, 0, 0, 0, 2, 0, 1, 0, 208, 0, 0, 0, 2, 0, 1, 0,
216, 0, 0, 0, 2, 0, 1, 0, 216, 0, 0, 0, 2, 0, 1, 0,
...@@ -504,7 +504,7 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = { ...@@ -504,7 +504,7 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = {
6, 0, 251, 255, 2, 0, 0, 0, 6, 0, 251, 255, 2, 0, 0, 0,
0, 0, 1, 0, 6, 0, 0, 0, 0, 0, 1, 0, 6, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
237, 0, 0, 0, 162, 0, 0, 0, 237, 0, 0, 0, 178, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
240, 0, 0, 0, 2, 0, 1, 0, 240, 0, 0, 0, 2, 0, 1, 0,
248, 0, 0, 0, 2, 0, 1, 0, 248, 0, 0, 0, 2, 0, 1, 0,
...@@ -515,8 +515,8 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = { ...@@ -515,8 +515,8 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
248, 0, 0, 0, 2, 0, 1, 0, 248, 0, 0, 0, 2, 0, 1, 0,
0, 1, 0, 0, 2, 0, 1, 0, 0, 1, 0, 0, 2, 0, 1, 0,
113, 117, 101, 115, 116, 105, 111, 110, 97, 110, 115, 119, 101, 114, 73, 100,
73, 100, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
...@@ -565,8 +565,8 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = { ...@@ -565,8 +565,8 @@ static const ::capnp::_::AlignedData<139> b_9e19b28d3db3573a = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
116, 97, 107, 101, 70, 114, 111, 109, 116, 97, 107, 101, 70, 114, 111, 109,
79, 116, 104, 101, 114, 65, 110, 115, 79, 116, 104, 101, 114, 81, 117, 101,
119, 101, 114, 0, 0, 0, 0, 0, 115, 116, 105, 111, 110, 0, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
...@@ -587,7 +587,7 @@ static const ::capnp::_::RawSchema* const d_9e19b28d3db3573a[] = { ...@@ -587,7 +587,7 @@ static const ::capnp::_::RawSchema* const d_9e19b28d3db3573a[] = {
&s_9a0e61223d96743b, &s_9a0e61223d96743b,
&s_d625b7063acf691a, &s_d625b7063acf691a,
}; };
static const uint16_t m_9e19b28d3db3573a[] = {7, 4, 3, 0, 1, 2, 5, 6}; static const uint16_t m_9e19b28d3db3573a[] = {7, 0, 4, 3, 1, 2, 5, 6};
static const uint16_t i_9e19b28d3db3573a[] = {2, 3, 4, 5, 6, 7, 0, 1}; static const uint16_t i_9e19b28d3db3573a[] = {2, 3, 4, 5, 6, 7, 0, 1};
const ::capnp::_::RawSchema s_9e19b28d3db3573a = { const ::capnp::_::RawSchema s_9e19b28d3db3573a = {
0x9e19b28d3db3573a, b_9e19b28d3db3573a.words, 139, d_9e19b28d3db3573a, m_9e19b28d3db3573a, 0x9e19b28d3db3573a, b_9e19b28d3db3573a.words, 139, d_9e19b28d3db3573a, m_9e19b28d3db3573a,
...@@ -1219,7 +1219,7 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = { ...@@ -1219,7 +1219,7 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = {
175, 1, 224, 144, 4, 152, 225, 251, 175, 1, 224, 144, 4, 152, 225, 251,
0, 0, 0, 0, 1, 0, 1, 0, 0, 0, 0, 0, 1, 0, 1, 0,
80, 162, 82, 37, 27, 152, 18, 179, 80, 162, 82, 37, 27, 152, 18, 179,
1, 0, 7, 0, 0, 0, 0, 0, 2, 0, 7, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
17, 0, 0, 0, 170, 0, 0, 0, 17, 0, 0, 0, 170, 0, 0, 0,
25, 0, 0, 0, 7, 0, 0, 0, 25, 0, 0, 0, 7, 0, 0, 0,
...@@ -1238,14 +1238,14 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = { ...@@ -1238,14 +1238,14 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
68, 0, 0, 0, 2, 0, 1, 0, 68, 0, 0, 0, 2, 0, 1, 0,
76, 0, 0, 0, 2, 0, 1, 0, 76, 0, 0, 0, 2, 0, 1, 0,
1, 0, 0, 0, 1, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0,
0, 0, 1, 0, 1, 0, 0, 0, 0, 0, 1, 0, 1, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
73, 0, 0, 0, 50, 0, 0, 0, 73, 0, 0, 0, 58, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
68, 0, 0, 0, 2, 0, 1, 0, 68, 0, 0, 0, 2, 0, 1, 0,
76, 0, 0, 0, 2, 0, 1, 0, 76, 0, 0, 0, 2, 0, 1, 0,
2, 0, 0, 0, 0, 0, 0, 0, 2, 0, 0, 0, 1, 0, 0, 0,
0, 0, 1, 0, 2, 0, 0, 0, 0, 0, 1, 0, 2, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
73, 0, 0, 0, 66, 0, 0, 0, 73, 0, 0, 0, 66, 0, 0, 0,
...@@ -1260,11 +1260,11 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = { ...@@ -1260,11 +1260,11 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = {
8, 0, 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
99, 97, 112, 73, 100, 0, 0, 0, 116, 97, 114, 103, 101, 116, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 193, 251, 19, 88, 84, 20, 188, 149,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
107, 101, 121, 80, 97, 114, 116, 0, 107, 101, 121, 80, 97, 114, 116, 0,
...@@ -1275,11 +1275,14 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = { ...@@ -1275,11 +1275,14 @@ static const ::capnp::_::AlignedData<59> b_fbe1980490e001af = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, } 0, 0, 0, 0, 0, 0, 0, 0, }
}; };
static const uint16_t m_fbe1980490e001af[] = {1, 2, 0}; static const ::capnp::_::RawSchema* const d_fbe1980490e001af[] = {
&s_95bc14545813fbc1,
};
static const uint16_t m_fbe1980490e001af[] = {2, 0, 1};
static const uint16_t i_fbe1980490e001af[] = {0, 1, 2}; static const uint16_t i_fbe1980490e001af[] = {0, 1, 2};
const ::capnp::_::RawSchema s_fbe1980490e001af = { const ::capnp::_::RawSchema s_fbe1980490e001af = {
0xfbe1980490e001af, b_fbe1980490e001af.words, 59, nullptr, m_fbe1980490e001af, 0xfbe1980490e001af, b_fbe1980490e001af.words, 59, d_fbe1980490e001af, m_fbe1980490e001af,
0, 3, i_fbe1980490e001af, nullptr, nullptr 1, 3, i_fbe1980490e001af, nullptr, nullptr
}; };
static const ::capnp::_::AlignedData<47> b_95bc14545813fbc1 = { static const ::capnp::_::AlignedData<47> b_95bc14545813fbc1 = {
{ 0, 0, 0, 0, 5, 0, 5, 0, { 0, 0, 0, 0, 5, 0, 5, 0,
...@@ -1313,7 +1316,7 @@ static const ::capnp::_::AlignedData<47> b_95bc14545813fbc1 = { ...@@ -1313,7 +1316,7 @@ static const ::capnp::_::AlignedData<47> b_95bc14545813fbc1 = {
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
44, 0, 0, 0, 2, 0, 1, 0, 44, 0, 0, 0, 2, 0, 1, 0,
52, 0, 0, 0, 2, 0, 1, 0, 52, 0, 0, 0, 2, 0, 1, 0,
101, 120, 112, 111, 114, 116, 101, 100, 105, 109, 112, 111, 114, 116, 101, 100,
67, 97, 112, 0, 0, 0, 0, 0, 67, 97, 112, 0, 0, 0, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
......
...@@ -71,7 +71,7 @@ struct Return { ...@@ -71,7 +71,7 @@ struct Return {
EXCEPTION, EXCEPTION,
CANCELED, CANCELED,
RESULTS_SENT_ELSEWHERE, RESULTS_SENT_ELSEWHERE,
TAKE_FROM_OTHER_ANSWER, TAKE_FROM_OTHER_QUESTION,
ACCEPT_FROM_THIRD_PARTY, ACCEPT_FROM_THIRD_PARTY,
}; };
}; };
...@@ -182,7 +182,7 @@ struct MessageTarget { ...@@ -182,7 +182,7 @@ struct MessageTarget {
class Builder; class Builder;
class Pipeline; class Pipeline;
enum Which: uint16_t { enum Which: uint16_t {
EXPORTED_CAP, IMPORTED_CAP,
PROMISED_ANSWER, PROMISED_ANSWER,
}; };
}; };
...@@ -342,7 +342,7 @@ CAPNP_DECLARE_STRUCT( ...@@ -342,7 +342,7 @@ CAPNP_DECLARE_STRUCT(
1, 1, INLINE_COMPOSITE); 1, 1, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT( CAPNP_DECLARE_STRUCT(
::capnp::rpc::Join, fbe1980490e001af, ::capnp::rpc::Join, fbe1980490e001af,
1, 1, INLINE_COMPOSITE); 1, 2, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT( CAPNP_DECLARE_STRUCT(
::capnp::rpc::MessageTarget, 95bc14545813fbc1, ::capnp::rpc::MessageTarget, 95bc14545813fbc1,
1, 1, INLINE_COMPOSITE); 1, 1, INLINE_COMPOSITE);
...@@ -834,7 +834,7 @@ public: ...@@ -834,7 +834,7 @@ public:
} }
inline Which which() const; inline Which which() const;
inline ::uint32_t getQuestionId() const; inline ::uint32_t getAnswerId() const;
inline bool getReleaseParamCaps() const; inline bool getReleaseParamCaps() const;
...@@ -852,8 +852,8 @@ public: ...@@ -852,8 +852,8 @@ public:
inline bool isResultsSentElsewhere() const; inline bool isResultsSentElsewhere() const;
inline ::capnp::Void getResultsSentElsewhere() const; inline ::capnp::Void getResultsSentElsewhere() const;
inline bool isTakeFromOtherAnswer() const; inline bool isTakeFromOtherQuestion() const;
inline ::uint32_t getTakeFromOtherAnswer() const; inline ::uint32_t getTakeFromOtherQuestion() const;
inline bool isAcceptFromThirdParty() const; inline bool isAcceptFromThirdParty() const;
inline bool hasAcceptFromThirdParty() const; inline bool hasAcceptFromThirdParty() const;
...@@ -890,8 +890,8 @@ public: ...@@ -890,8 +890,8 @@ public:
inline ::capnp::MessageSize totalSize() const { return asReader().totalSize(); } inline ::capnp::MessageSize totalSize() const { return asReader().totalSize(); }
inline Which which(); inline Which which();
inline ::uint32_t getQuestionId(); inline ::uint32_t getAnswerId();
inline void setQuestionId( ::uint32_t value); inline void setAnswerId( ::uint32_t value);
inline bool getReleaseParamCaps(); inline bool getReleaseParamCaps();
inline void setReleaseParamCaps(bool value); inline void setReleaseParamCaps(bool value);
...@@ -920,9 +920,9 @@ public: ...@@ -920,9 +920,9 @@ public:
inline ::capnp::Void getResultsSentElsewhere(); inline ::capnp::Void getResultsSentElsewhere();
inline void setResultsSentElsewhere( ::capnp::Void value = ::capnp::VOID); inline void setResultsSentElsewhere( ::capnp::Void value = ::capnp::VOID);
inline bool isTakeFromOtherAnswer(); inline bool isTakeFromOtherQuestion();
inline ::uint32_t getTakeFromOtherAnswer(); inline ::uint32_t getTakeFromOtherQuestion();
inline void setTakeFromOtherAnswer( ::uint32_t value); inline void setTakeFromOtherQuestion( ::uint32_t value);
inline bool isAcceptFromThirdParty(); inline bool isAcceptFromThirdParty();
inline bool hasAcceptFromThirdParty(); inline bool hasAcceptFromThirdParty();
...@@ -1816,7 +1816,8 @@ public: ...@@ -1816,7 +1816,8 @@ public:
inline ::uint32_t getQuestionId() const; inline ::uint32_t getQuestionId() const;
inline ::uint32_t getCapId() const; inline bool hasTarget() const;
inline ::capnp::rpc::MessageTarget::Reader getTarget() const;
inline bool hasKeyPart() const; inline bool hasKeyPart() const;
inline ::capnp::AnyPointer::Reader getKeyPart() const; inline ::capnp::AnyPointer::Reader getKeyPart() const;
...@@ -1854,8 +1855,12 @@ public: ...@@ -1854,8 +1855,12 @@ public:
inline ::uint32_t getQuestionId(); inline ::uint32_t getQuestionId();
inline void setQuestionId( ::uint32_t value); inline void setQuestionId( ::uint32_t value);
inline ::uint32_t getCapId(); inline bool hasTarget();
inline void setCapId( ::uint32_t value); inline ::capnp::rpc::MessageTarget::Builder getTarget();
inline void setTarget( ::capnp::rpc::MessageTarget::Reader value);
inline ::capnp::rpc::MessageTarget::Builder initTarget();
inline void adoptTarget(::capnp::Orphan< ::capnp::rpc::MessageTarget>&& value);
inline ::capnp::Orphan< ::capnp::rpc::MessageTarget> disownTarget();
inline bool hasKeyPart(); inline bool hasKeyPart();
inline ::capnp::AnyPointer::Builder getKeyPart(); inline ::capnp::AnyPointer::Builder getKeyPart();
...@@ -1881,6 +1886,7 @@ public: ...@@ -1881,6 +1886,7 @@ public:
inline explicit Pipeline(::capnp::AnyPointer::Pipeline&& typeless) inline explicit Pipeline(::capnp::AnyPointer::Pipeline&& typeless)
: _typeless(kj::mv(typeless)) {} : _typeless(kj::mv(typeless)) {}
inline ::capnp::rpc::MessageTarget::Pipeline getTarget();
private: private:
::capnp::AnyPointer::Pipeline _typeless; ::capnp::AnyPointer::Pipeline _typeless;
template <typename T, ::capnp::Kind k> template <typename T, ::capnp::Kind k>
...@@ -1899,8 +1905,8 @@ public: ...@@ -1899,8 +1905,8 @@ public:
} }
inline Which which() const; inline Which which() const;
inline bool isExportedCap() const; inline bool isImportedCap() const;
inline ::uint32_t getExportedCap() const; inline ::uint32_t getImportedCap() const;
inline bool isPromisedAnswer() const; inline bool isPromisedAnswer() const;
inline bool hasPromisedAnswer() const; inline bool hasPromisedAnswer() const;
...@@ -1937,9 +1943,9 @@ public: ...@@ -1937,9 +1943,9 @@ public:
inline ::capnp::MessageSize totalSize() const { return asReader().totalSize(); } inline ::capnp::MessageSize totalSize() const { return asReader().totalSize(); }
inline Which which(); inline Which which();
inline bool isExportedCap(); inline bool isImportedCap();
inline ::uint32_t getExportedCap(); inline ::uint32_t getImportedCap();
inline void setExportedCap( ::uint32_t value); inline void setImportedCap( ::uint32_t value);
inline bool isPromisedAnswer(); inline bool isPromisedAnswer();
inline bool hasPromisedAnswer(); inline bool hasPromisedAnswer();
...@@ -3570,16 +3576,16 @@ inline Return::Which Return::Builder::which() { ...@@ -3570,16 +3576,16 @@ inline Return::Which Return::Builder::which() {
return _builder.getDataField<Which>(3 * ::capnp::ELEMENTS); return _builder.getDataField<Which>(3 * ::capnp::ELEMENTS);
} }
inline ::uint32_t Return::Reader::getQuestionId() const { inline ::uint32_t Return::Reader::getAnswerId() const {
return _reader.getDataField< ::uint32_t>( return _reader.getDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS); 0 * ::capnp::ELEMENTS);
} }
inline ::uint32_t Return::Builder::getQuestionId() { inline ::uint32_t Return::Builder::getAnswerId() {
return _builder.getDataField< ::uint32_t>( return _builder.getDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS); 0 * ::capnp::ELEMENTS);
} }
inline void Return::Builder::setQuestionId( ::uint32_t value) { inline void Return::Builder::setAnswerId( ::uint32_t value) {
_builder.setDataField< ::uint32_t>( _builder.setDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS, value); 0 * ::capnp::ELEMENTS, value);
} }
...@@ -3754,28 +3760,28 @@ inline void Return::Builder::setResultsSentElsewhere( ::capnp::Void value) { ...@@ -3754,28 +3760,28 @@ inline void Return::Builder::setResultsSentElsewhere( ::capnp::Void value) {
0 * ::capnp::ELEMENTS, value); 0 * ::capnp::ELEMENTS, value);
} }
inline bool Return::Reader::isTakeFromOtherAnswer() const { inline bool Return::Reader::isTakeFromOtherQuestion() const {
return which() == Return::TAKE_FROM_OTHER_ANSWER; return which() == Return::TAKE_FROM_OTHER_QUESTION;
} }
inline bool Return::Builder::isTakeFromOtherAnswer() { inline bool Return::Builder::isTakeFromOtherQuestion() {
return which() == Return::TAKE_FROM_OTHER_ANSWER; return which() == Return::TAKE_FROM_OTHER_QUESTION;
} }
inline ::uint32_t Return::Reader::getTakeFromOtherAnswer() const { inline ::uint32_t Return::Reader::getTakeFromOtherQuestion() const {
KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_ANSWER, KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_QUESTION,
"Must check which() before get()ing a union member."); "Must check which() before get()ing a union member.");
return _reader.getDataField< ::uint32_t>( return _reader.getDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS); 2 * ::capnp::ELEMENTS);
} }
inline ::uint32_t Return::Builder::getTakeFromOtherAnswer() { inline ::uint32_t Return::Builder::getTakeFromOtherQuestion() {
KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_ANSWER, KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_QUESTION,
"Must check which() before get()ing a union member."); "Must check which() before get()ing a union member.");
return _builder.getDataField< ::uint32_t>( return _builder.getDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS); 2 * ::capnp::ELEMENTS);
} }
inline void Return::Builder::setTakeFromOtherAnswer( ::uint32_t value) { inline void Return::Builder::setTakeFromOtherQuestion( ::uint32_t value) {
_builder.setDataField<Return::Which>( _builder.setDataField<Return::Which>(
3 * ::capnp::ELEMENTS, Return::TAKE_FROM_OTHER_ANSWER); 3 * ::capnp::ELEMENTS, Return::TAKE_FROM_OTHER_QUESTION);
_builder.setDataField< ::uint32_t>( _builder.setDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS, value); 2 * ::capnp::ELEMENTS, value);
} }
...@@ -4408,37 +4414,58 @@ inline void Join::Builder::setQuestionId( ::uint32_t value) { ...@@ -4408,37 +4414,58 @@ inline void Join::Builder::setQuestionId( ::uint32_t value) {
0 * ::capnp::ELEMENTS, value); 0 * ::capnp::ELEMENTS, value);
} }
inline ::uint32_t Join::Reader::getCapId() const { inline bool Join::Reader::hasTarget() const {
return _reader.getDataField< ::uint32_t>( return !_reader.getPointerField(0 * ::capnp::POINTERS).isNull();
1 * ::capnp::ELEMENTS);
} }
inline bool Join::Builder::hasTarget() {
inline ::uint32_t Join::Builder::getCapId() { return !_builder.getPointerField(0 * ::capnp::POINTERS).isNull();
return _builder.getDataField< ::uint32_t>(
1 * ::capnp::ELEMENTS);
} }
inline void Join::Builder::setCapId( ::uint32_t value) { inline ::capnp::rpc::MessageTarget::Reader Join::Reader::getTarget() const {
_builder.setDataField< ::uint32_t>( return ::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::get(
1 * ::capnp::ELEMENTS, value); _reader.getPointerField(0 * ::capnp::POINTERS));
}
inline ::capnp::rpc::MessageTarget::Builder Join::Builder::getTarget() {
return ::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::get(
_builder.getPointerField(0 * ::capnp::POINTERS));
}
inline ::capnp::rpc::MessageTarget::Pipeline Join::Pipeline::getTarget() {
return ::capnp::rpc::MessageTarget::Pipeline(_typeless.getPointerField(0));
}
inline void Join::Builder::setTarget( ::capnp::rpc::MessageTarget::Reader value) {
::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::set(
_builder.getPointerField(0 * ::capnp::POINTERS), value);
}
inline ::capnp::rpc::MessageTarget::Builder Join::Builder::initTarget() {
return ::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::init(
_builder.getPointerField(0 * ::capnp::POINTERS));
}
inline void Join::Builder::adoptTarget(
::capnp::Orphan< ::capnp::rpc::MessageTarget>&& value) {
::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::adopt(
_builder.getPointerField(0 * ::capnp::POINTERS), kj::mv(value));
}
inline ::capnp::Orphan< ::capnp::rpc::MessageTarget> Join::Builder::disownTarget() {
return ::capnp::_::PointerHelpers< ::capnp::rpc::MessageTarget>::disown(
_builder.getPointerField(0 * ::capnp::POINTERS));
} }
inline bool Join::Reader::hasKeyPart() const { inline bool Join::Reader::hasKeyPart() const {
return !_reader.getPointerField(0 * ::capnp::POINTERS).isNull(); return !_reader.getPointerField(1 * ::capnp::POINTERS).isNull();
} }
inline bool Join::Builder::hasKeyPart() { inline bool Join::Builder::hasKeyPart() {
return !_builder.getPointerField(0 * ::capnp::POINTERS).isNull(); return !_builder.getPointerField(1 * ::capnp::POINTERS).isNull();
} }
inline ::capnp::AnyPointer::Reader Join::Reader::getKeyPart() const { inline ::capnp::AnyPointer::Reader Join::Reader::getKeyPart() const {
return ::capnp::AnyPointer::Reader( return ::capnp::AnyPointer::Reader(
_reader.getPointerField(0 * ::capnp::POINTERS)); _reader.getPointerField(1 * ::capnp::POINTERS));
} }
inline ::capnp::AnyPointer::Builder Join::Builder::getKeyPart() { inline ::capnp::AnyPointer::Builder Join::Builder::getKeyPart() {
return ::capnp::AnyPointer::Builder( return ::capnp::AnyPointer::Builder(
_builder.getPointerField(0 * ::capnp::POINTERS)); _builder.getPointerField(1 * ::capnp::POINTERS));
} }
inline ::capnp::AnyPointer::Builder Join::Builder::initKeyPart() { inline ::capnp::AnyPointer::Builder Join::Builder::initKeyPart() {
auto result = ::capnp::AnyPointer::Builder( auto result = ::capnp::AnyPointer::Builder(
_builder.getPointerField(0 * ::capnp::POINTERS)); _builder.getPointerField(1 * ::capnp::POINTERS));
result.clear(); result.clear();
return result; return result;
} }
...@@ -4450,28 +4477,28 @@ inline MessageTarget::Which MessageTarget::Builder::which() { ...@@ -4450,28 +4477,28 @@ inline MessageTarget::Which MessageTarget::Builder::which() {
return _builder.getDataField<Which>(2 * ::capnp::ELEMENTS); return _builder.getDataField<Which>(2 * ::capnp::ELEMENTS);
} }
inline bool MessageTarget::Reader::isExportedCap() const { inline bool MessageTarget::Reader::isImportedCap() const {
return which() == MessageTarget::EXPORTED_CAP; return which() == MessageTarget::IMPORTED_CAP;
} }
inline bool MessageTarget::Builder::isExportedCap() { inline bool MessageTarget::Builder::isImportedCap() {
return which() == MessageTarget::EXPORTED_CAP; return which() == MessageTarget::IMPORTED_CAP;
} }
inline ::uint32_t MessageTarget::Reader::getExportedCap() const { inline ::uint32_t MessageTarget::Reader::getImportedCap() const {
KJ_IREQUIRE(which() == MessageTarget::EXPORTED_CAP, KJ_IREQUIRE(which() == MessageTarget::IMPORTED_CAP,
"Must check which() before get()ing a union member."); "Must check which() before get()ing a union member.");
return _reader.getDataField< ::uint32_t>( return _reader.getDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS); 0 * ::capnp::ELEMENTS);
} }
inline ::uint32_t MessageTarget::Builder::getExportedCap() { inline ::uint32_t MessageTarget::Builder::getImportedCap() {
KJ_IREQUIRE(which() == MessageTarget::EXPORTED_CAP, KJ_IREQUIRE(which() == MessageTarget::IMPORTED_CAP,
"Must check which() before get()ing a union member."); "Must check which() before get()ing a union member.");
return _builder.getDataField< ::uint32_t>( return _builder.getDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS); 0 * ::capnp::ELEMENTS);
} }
inline void MessageTarget::Builder::setExportedCap( ::uint32_t value) { inline void MessageTarget::Builder::setImportedCap( ::uint32_t value) {
_builder.setDataField<MessageTarget::Which>( _builder.setDataField<MessageTarget::Which>(
2 * ::capnp::ELEMENTS, MessageTarget::EXPORTED_CAP); 2 * ::capnp::ELEMENTS, MessageTarget::IMPORTED_CAP);
_builder.setDataField< ::uint32_t>( _builder.setDataField< ::uint32_t>(
0 * ::capnp::ELEMENTS, value); 0 * ::capnp::ELEMENTS, value);
} }
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment