Commit 1e518b92 authored by Kenton Varda's avatar Kenton Varda

Tail calls half-implemented, but I think it's time to delete all the…

Tail calls half-implemented, but I think it's time to delete all the cancellation code before continuing because it will simplify things.
parent 08143efb
......@@ -174,6 +174,10 @@ public:
ObjectPointer::Pipeline(kj::refcounted<BrokenPipeline>(exception)));
}
const void* getBrand() const {
return nullptr;
}
kj::Exception exception;
LocalMessage message;
};
......
......@@ -152,6 +152,39 @@ TEST(Capability, Pipelining) {
EXPECT_EQ(1, chainedCallCount);
}
TEST(Capability, TailCall) {
kj::SimpleEventLoop loop;
int calleeCallCount = 0;
int callerCallCount = 0;
test::TestTailCallee::Client callee(kj::heap<TestTailCalleeImpl>(calleeCallCount), loop);
test::TestTailCaller::Client caller(kj::heap<TestTailCallerImpl>(callerCallCount), loop);
auto request = caller.fooRequest();
request.setI(456);
request.setCallee(callee);
auto promise = request.send();
auto dependentCall0 = promise.getC().getCallSequenceRequest().send();
auto response = loop.wait(kj::mv(promise));
EXPECT_EQ(456, response.getI());
EXPECT_EQ(456, response.getI());
auto dependentCall1 = promise.getC().getCallSequenceRequest().send();
auto dependentCall2 = response.getC().getCallSequenceRequest().send();
EXPECT_EQ(0, loop.wait(kj::mv(dependentCall0)).getN());
EXPECT_EQ(1, loop.wait(kj::mv(dependentCall1)).getN());
EXPECT_EQ(2, loop.wait(kj::mv(dependentCall2)).getN());
EXPECT_EQ(1, calleeCallCount);
EXPECT_EQ(1, callerCallCount);
}
// =======================================================================================
TEST(Capability, DynamicClient) {
......
......@@ -75,8 +75,6 @@ kj::Promise<void> ClientHook::whenResolved() const {
// =======================================================================================
namespace {
class LocalResponse final: public ResponseHook, public kj::Refcounted {
public:
LocalResponse(uint sizeHint)
......@@ -101,10 +99,33 @@ public:
request = nullptr;
}
ObjectPointer::Builder getResults(uint firstSegmentWordSize) override {
if (!response) {
response = kj::refcounted<LocalResponse>(firstSegmentWordSize);
if (response == nullptr) {
auto localResponse = kj::refcounted<LocalResponse>(firstSegmentWordSize);
responseBuilder = localResponse->message.getRoot();
response = Response<ObjectPointer>(responseBuilder.asReader(), kj::mv(localResponse));
}
return response->message.getRoot();
return responseBuilder;
}
kj::Promise<void> tailCall(kj::Own<RequestHook> request) override {
KJ_REQUIRE(response == nullptr, "Can't call tailCall() after initializing the results struct.");
releaseParams();
auto promise = request->send();
// Link pipelines.
KJ_IF_MAYBE(f, tailCallPipelineFulfiller) {
f->get()->fulfill(kj::mv(kj::implicitCast<ObjectPointer::Pipeline&>(promise)));
}
// Wait for response.
return promise.then([this](Response<ObjectPointer>&& tailResponse) {
response = kj::mv(tailResponse);
});
}
kj::Promise<ObjectPointer::Pipeline> onTailCall() override {
auto paf = kj::newPromiseAndFulfiller<ObjectPointer::Pipeline>();
tailCallPipelineFulfiller = kj::mv(paf.fulfiller);
return kj::mv(paf.promise);
}
void allowAsyncCancellation() override {
// ignored for local calls
......@@ -117,8 +138,10 @@ public:
}
kj::Maybe<kj::Own<LocalMessage>> request;
kj::Own<LocalResponse> response;
kj::Maybe<Response<ObjectPointer>> response;
ObjectPointer::Builder responseBuilder = nullptr; // only valid if `response` is non-null
kj::Own<const ClientHook> clientRef;
kj::Maybe<kj::Own<kj::PromiseFulfiller<ObjectPointer::Pipeline>>> tailCallPipelineFulfiller;
};
class LocalRequest final: public RequestHook {
......@@ -131,6 +154,8 @@ public:
interfaceId(interfaceId), methodId(methodId), client(kj::mv(client)) {}
RemotePromise<ObjectPointer> send() override {
KJ_REQUIRE(message.get() != nullptr, "Already called send() on this request.");
// For the lambda capture.
uint64_t interfaceId = this->interfaceId;
uint16_t methodId = this->methodId;
......@@ -140,15 +165,21 @@ public:
auto promise = loop.there(kj::mv(promiseAndPipeline.promise),
kj::mvCapture(context, [=](kj::Own<LocalCallContext> context) {
// Do not inline `reader` -- kj::mv on next line may occur first.
auto reader = context->getResults(1).asReader();
return Response<ObjectPointer>(reader, kj::mv(context->response));
KJ_IF_MAYBE(r, context->response) {
return kj::mv(*r);
} else {
KJ_FAIL_ASSERT("Method implementation failed to fill in results.");
}
}));
return RemotePromise<ObjectPointer>(
kj::mv(promise), ObjectPointer::Pipeline(kj::mv(promiseAndPipeline.pipeline)));
}
const void* getBrand() const {
return nullptr;
}
kj::Own<LocalMessage> message;
private:
......@@ -398,8 +429,7 @@ public:
});
// Make sure that this client cannot be destroyed until the promise completes.
promise = promise.thenInAnyThread(kj::mvCapture(kj::addRef(*this),
[](kj::Own<const LocalClient>&& ref) {}));
promise.attach(kj::addRef(*this));
// We have to fork this promise for the pipeline to receive a copy of the answer.
auto forked = server.getEventLoop().fork(kj::mv(promise));
......@@ -410,14 +440,16 @@ public:
return kj::refcounted<LocalPipeline>(kj::mv(context));
}));
auto completionPromise = forked.addBranch().thenInAnyThread(kj::mvCapture(context,
[=](kj::Own<CallContextHook>&& context) {
// Nothing to do here. We just wanted to make sure to hold on to a reference to the
// context even if the pipeline was discarded.
//
// TODO(someday): We could probably make this less ugly if we had the ability to
// convert Promise<Tuple<T, U>> -> Tuple<Promise<T>, Promise<U>>...
}));
auto tailPipelinePromise = context->onTailCall().thenInAnyThread(
[](ObjectPointer::Pipeline&& pipeline) {
return kj::mv(pipeline.hook);
});
pipelinePromise = server.getEventLoop().exclusiveJoin(
kj::mv(pipelinePromise), kj::mv(tailPipelinePromise));
auto completionPromise = forked.addBranch();
completionPromise.attach(kj::mv(context));
return VoidPromiseAndPipeline { kj::mv(completionPromise),
kj::refcounted<QueuedPipeline>(server.getEventLoop(), kj::mv(pipelinePromise)) };
......@@ -444,8 +476,6 @@ private:
kj::EventLoopGuarded<kj::Own<Capability::Server>> server;
};
} // namespace
kj::Own<const ClientHook> Capability::Client::makeLocalClient(
kj::Own<Capability::Server>&& server, const kj::EventLoop& eventLoop) {
return kj::refcounted<LocalClient>(eventLoop, kj::mv(server));
......
......@@ -83,6 +83,8 @@ private:
friend class Capability::Client;
friend struct DynamicCapability;
template <typename, typename>
friend class CallContext;
};
template <typename Results>
......@@ -207,6 +209,20 @@ public:
// `firstSegmentWordSize` indicates the suggested size of the message's first segment. This
// is a hint only. If not specified, the system will decide on its own.
template <typename SubParams>
kj::Promise<void> tailCall(Request<SubParams, Results>&& tailRequest);
// Resolve the call by making a tail call. `tailRequest` is a request that has been filled in
// but not yet sent. The context will send the call, then fill in the results with the result
// of the call. If tailCall() is used, {get,init,set,adopt}Results (above) *must not* be called.
//
// The RPC implementation may be able to optimize a tail call to another machine such that the
// results never actually pass through this machine. Even if no such optimization is possible,
// `tailCall()` may allow pipelined calls to be forwarded optimistically to the new call site.
//
// `tailCall()` implies a call to `releaseParams()`, to simplify certain implementations.
// In general, this should be the last thing a method implementation calls, and the promise
// returned from `tailCall()` should then be returned by the method implementation.
void allowAsyncCancellation();
// Indicate that it is OK for the RPC system to discard its Promise for this call's result if
// the caller cancels the call, thereby transitively canceling any asynchronous operations the
......@@ -276,6 +292,11 @@ class RequestHook {
public:
virtual RemotePromise<ObjectPointer> send() = 0;
// Send the call and return a promise for the result.
virtual const void* getBrand() const = 0;
// Returns a void* that identifies who made this request. This can be used by an RPC adapter to
// discover when tail call is going to be sent over its own connection and therefore can be
// optimized into a remote tail call.
};
class ResponseHook {
......@@ -347,9 +368,14 @@ public:
virtual ObjectPointer::Reader getParams() = 0;
virtual void releaseParams() = 0;
virtual ObjectPointer::Builder getResults(uint firstSegmentWordSize) = 0;
virtual kj::Promise<void> tailCall(kj::Own<RequestHook> request) = 0;
virtual void allowAsyncCancellation() = 0;
virtual bool isCanceled() = 0;
virtual kj::Promise<ObjectPointer::Pipeline> onTailCall() = 0;
// If `tailCall()` is called, resolves to the PipelineHook from the tail call. An
// implementation of `ClientHook::call()` is allowed to call this at most once.
virtual kj::Own<CallContextHook> addRef() = 0;
};
......@@ -561,6 +587,12 @@ inline Orphanage CallContext<Params, Results>::getResultsOrphanage(uint firstSeg
return Orphanage::getForMessageContaining(hook->getResults(firstSegmentWordSize));
}
template <typename Params, typename Results>
template <typename SubParams>
inline kj::Promise<void> CallContext<Params, Results>::tailCall(
Request<SubParams, Results>&& tailRequest) {
return hook->tailCall(kj::mv(tailRequest.hook));
}
template <typename Params, typename Results>
inline void CallContext<Params, Results>::allowAsyncCancellation() {
hook->allowAsyncCancellation();
}
......
......@@ -247,6 +247,8 @@ struct ObjectPointer {
inline Pipeline(kj::Own<const PipelineHook>&& hook, kj::Array<PipelineOp>&& ops)
: hook(kj::mv(hook)), ops(kj::mv(ops)) {}
friend class LocalClient;
};
};
......
......@@ -68,7 +68,7 @@ public:
class ConnectionImpl final: public Connection, public kj::Refcounted {
public:
ConnectionImpl() {}
ConnectionImpl(const char* name): name(name) {}
void attach(ConnectionImpl& other) {
KJ_REQUIRE(partner == nullptr);
......@@ -100,6 +100,9 @@ public:
return message->message.getRoot<ObjectPointer>();
}
void send() override {
//kj::String msg = kj::str(connection.name, ": ", message->message.getRoot<rpc::Message>());
//KJ_DBG(msg);
KJ_IF_MAYBE(p, connection.partner) {
auto lock = p->queues.lockExclusive();
if (lock->fulfillers.empty()) {
......@@ -146,6 +149,7 @@ public:
}
private:
const char* name;
kj::Maybe<ConnectionImpl&> partner;
struct Queues {
......@@ -172,8 +176,8 @@ public:
auto iter = myLock->connections.find(&dst);
if (iter == myLock->connections.end()) {
auto local = kj::refcounted<ConnectionImpl>();
auto remote = kj::refcounted<ConnectionImpl>();
auto local = kj::refcounted<ConnectionImpl>("client");
auto remote = kj::refcounted<ConnectionImpl>("server");
local->attach(*remote);
myLock->connections[&dst] = kj::addRef(*local);
......@@ -237,6 +241,10 @@ public:
return Capability::Client(newBrokenCap("No TestExtends implemented."));
case test::TestSturdyRefObjectId::Tag::TEST_PIPELINE:
return kj::heap<TestPipelineImpl>(callCount);
case test::TestSturdyRefObjectId::Tag::TEST_TAIL_CALLEE:
return kj::heap<TestTailCalleeImpl>(callCount);
case test::TestSturdyRefObjectId::Tag::TEST_TAIL_CALLER:
return kj::heap<TestTailCallerImpl>(callCount);
}
KJ_UNREACHABLE;
}
......@@ -343,6 +351,38 @@ TEST_F(RpcTest, Pipelining) {
EXPECT_EQ(1, chainedCallCount);
}
TEST_F(RpcTest, TailCall) {
auto caller = connect(test::TestSturdyRefObjectId::Tag::TEST_TAIL_CALLER)
.castAs<test::TestTailCaller>();
int calleeCallCount = 0;
test::TestTailCallee::Client callee(kj::heap<TestTailCalleeImpl>(calleeCallCount), loop);
auto request = caller.fooRequest();
request.setI(456);
request.setCallee(callee);
auto promise = request.send();
auto dependentCall0 = promise.getC().getCallSequenceRequest().send();
auto response = loop.wait(kj::mv(promise));
EXPECT_EQ(456, response.getI());
EXPECT_EQ(456, response.getI());
auto dependentCall1 = promise.getC().getCallSequenceRequest().send();
auto dependentCall2 = response.getC().getCallSequenceRequest().send();
EXPECT_EQ(0, loop.wait(kj::mv(dependentCall0)).getN());
EXPECT_EQ(1, loop.wait(kj::mv(dependentCall1)).getN());
EXPECT_EQ(2, loop.wait(kj::mv(dependentCall2)).getN());
EXPECT_EQ(1, calleeCallCount);
EXPECT_EQ(1, restorer.callCount);
}
} // namespace
} // namespace _ (private)
} // namespace capnp
......@@ -47,6 +47,9 @@ inline constexpr uint messageSizeHint<void>() {
return 1 + sizeInWords<rpc::Message>();
}
constexpr const uint MESSAGE_TARGET_SIZE_HINT = sizeInWords<rpc::MessageTarget>() +
sizeInWords<rpc::PromisedAnswer>() + 16; // +16 for ops; hope that's enough
kj::Maybe<kj::Array<PipelineOp>> toPipelineOps(List<rpc::PromisedAnswer::Op>::Reader ops) {
auto result = kj::heapArrayBuilder<PipelineOp>(ops.size());
for (auto opReader: ops) {
......@@ -280,8 +283,7 @@ public:
message->send();
}
auto pipeline = kj::refcounted<RpcPipeline>(
*this, kj::mv(questionRef), eventLoop.fork(kj::mv(paf.promise)));
auto pipeline = kj::refcounted<RpcPipeline>(*this, kj::mv(questionRef), kj::mv(paf.promise));
return pipeline->getPipelinedCap(kj::Array<const PipelineOp>(nullptr));
}
......@@ -388,6 +390,9 @@ private:
// The local QuestionRef, set to nullptr when it is destroyed, which is also when `Finish` is
// sent.
bool isTailCall = false;
// Is this a tail call? If so, we don't expect to receive results in the `Return`.
inline bool operator==(decltype(nullptr)) const {
return paramCaps == nullptr && selfRef == nullptr;
}
......@@ -408,6 +413,9 @@ private:
kj::Maybe<const RpcCallContext&> callContext;
// The call context, if it's still active. Becomes null when the `Return` message is sent. This
// object, if non-null, is owned by `asyncOp`.
kj::Maybe<kj::Own<CapInjectorImpl>> resultCaps;
// Set when `Return` is sent, free when `Finish` is received.
};
struct Export {
......@@ -952,8 +960,7 @@ private:
// a `Disembargo` to echo through the peer.
auto message = connectionState->connection->newOutgoingMessage(
messageSizeHint<rpc::Disembargo>() + sizeInWords<rpc::MessageTarget>() +
sizeInWords<rpc::PromisedAnswer>());
messageSizeHint<rpc::Disembargo>() + MESSAGE_TARGET_SIZE_HINT);
auto disembargo = message->getBody().initAs<rpc::Message>().getDisembargo();
......@@ -1543,8 +1550,8 @@ private:
: connectionState(kj::addRef(connectionState)),
target(kj::mv(target)),
message(connectionState.connection->newOutgoingMessage(
firstSegmentWordSize == 0 ? 0 : firstSegmentWordSize + messageSizeHint<rpc::Call>() +
sizeInWords<rpc::MessageTarget>() + sizeInWords<rpc::PromisedAnswer>())),
firstSegmentWordSize == 0 ? 0 :
firstSegmentWordSize + messageSizeHint<rpc::Call>() + MESSAGE_TARGET_SIZE_HINT)),
injector(kj::heap<CapInjectorImpl>(connectionState)),
context(*injector),
callBuilder(message->getBody().getAs<rpc::Message>().initCall()),
......@@ -1558,9 +1565,7 @@ private:
}
RemotePromise<ObjectPointer> send() override {
QuestionId questionId;
kj::Own<QuestionRef> questionRef;
kj::Promise<kj::Own<const RpcResponse>> promise = nullptr;
SendInternalResult sendResult;
{
auto lock = connectionState->tables.lockExclusive();
......@@ -1592,28 +1597,15 @@ private:
replacement.set(paramsBuilder);
return replacement.send();
} else {
injector->finishDescriptors(*lock);
auto paf = kj::newPromiseAndFulfiller<kj::Own<const RpcResponse>>(
connectionState->eventLoop);
auto& question = lock->questions.next(questionId);
callBuilder.setQuestionId(questionId);
question.paramCaps = kj::mv(injector);
questionRef = kj::refcounted<QuestionRef>(
*connectionState, questionId, kj::mv(paf.fulfiller),
kj::addRef(*lock->resolutionChainTail));
question.selfRef = *questionRef;
message->send();
promise = kj::mv(paf.promise);
promise.attach(kj::addRef(*questionRef));
sendResult = sendInternal(false, *lock);
}
}
auto forkedPromise = connectionState->eventLoop.fork(kj::mv(promise));
auto forkedPromise = connectionState->eventLoop.fork(kj::mv(sendResult.promise));
// The pipeline must get notified of resolution before the app does to maintain ordering.
auto pipeline = kj::refcounted<RpcPipeline>(
*connectionState, kj::mv(sendResult.questionRef), forkedPromise.addBranch());
auto appPromise = forkedPromise.addBranch().thenInAnyThread(
[=](kj::Own<const RpcResponse>&& response) {
......@@ -1621,14 +1613,58 @@ private:
return Response<ObjectPointer>(reader, kj::mv(response));
});
auto pipeline = kj::refcounted<RpcPipeline>(
*connectionState, kj::mv(questionRef), kj::mv(forkedPromise));
return RemotePromise<ObjectPointer>(
kj::mv(appPromise),
ObjectPointer::Pipeline(kj::mv(pipeline)));
}
struct TailInfo {
QuestionId questionId;
kj::Promise<void> promise;
kj::Own<const PipelineHook> pipeline;
};
kj::Maybe<TailInfo> tailSend() {
// Send the request as a tail call.
//
// Returns null if for some reason a tail call is not possible and the caller should fall
// back to using send() and copying the response.
SendInternalResult sendResult;
{
auto lock = connectionState->tables.lockExclusive();
if (lock->networkException != nullptr) {
// Disconnected; fall back to a regular send() which will fail appropriately.
return nullptr;
}
KJ_IF_MAYBE(redirect, target->writeTarget(callBuilder.getTarget())) {
// Whoops, this capability has been redirected while we were building the request!
// Fall back to regular send().
return nullptr;
} else {
sendResult = sendInternal(true, *lock);
}
}
auto promise = sendResult.promise.thenInAnyThread([](kj::Own<const RpcResponse>&& response) {
// Response should be null if `Return` handling code is correct.
KJ_ASSERT(!response) { break; }
});
QuestionId questionId = sendResult.questionRef->getId();
auto pipeline = kj::refcounted<RpcPipeline>(*connectionState, kj::mv(sendResult.questionRef));
return TailInfo { questionId, kj::mv(promise), kj::mv(pipeline) };
}
const void* getBrand() const override {
return connectionState.get();
}
private:
kj::Own<const RpcConnectionState> connectionState;
......@@ -1638,15 +1674,51 @@ private:
CapBuilderContext context;
rpc::Call::Builder callBuilder;
ObjectPointer::Builder paramsBuilder;
struct SendInternalResult {
kj::Own<QuestionRef> questionRef;
kj::Promise<kj::Own<const RpcResponse>> promise = nullptr;
};
SendInternalResult sendInternal(bool isTailCall, Tables& lockedTables) {
injector->finishDescriptors(lockedTables);
auto paf = kj::newPromiseAndFulfiller<kj::Own<const RpcResponse>>();
QuestionId questionId;
auto& question = lockedTables.questions.next(questionId);
callBuilder.setQuestionId(questionId);
if (isTailCall) {
callBuilder.getSendResultsTo().setYourself();
}
question.paramCaps = kj::mv(injector);
question.isTailCall = isTailCall;
SendInternalResult result;
result.questionRef = kj::refcounted<QuestionRef>(
*connectionState, questionId, kj::mv(paf.fulfiller),
kj::addRef(*lockedTables.resolutionChainTail));
question.selfRef = *result.questionRef;
message->send();
result.promise = kj::mv(paf.promise);
result.promise.attach(kj::addRef(*result.questionRef));
return kj::mv(result);
}
};
class RpcPipeline final: public PipelineHook, public kj::Refcounted {
public:
RpcPipeline(const RpcConnectionState& connectionState, kj::Own<const QuestionRef> questionRef,
kj::ForkedPromise<kj::Own<const RpcResponse>>&& redirectLaterParam)
kj::Promise<kj::Own<const RpcResponse>>&& redirectLaterParam)
: connectionState(kj::addRef(connectionState)),
redirectLater(kj::mv(redirectLaterParam)),
resolveSelfPromise(connectionState.eventLoop.there(redirectLater.addBranch(),
redirectLater(connectionState.eventLoop.fork(kj::mv(redirectLaterParam))),
resolveSelfPromise(connectionState.eventLoop.there(
KJ_ASSERT_NONNULL(redirectLater).addBranch(),
[this](kj::Own<const RpcResponse>&& response) {
resolve(kj::mv(response));
}, [this](kj::Exception&& exception) {
......@@ -1658,8 +1730,12 @@ private:
state.getWithoutLock().init<Waiting>(kj::mv(questionRef));
}
kj::Promise<kj::Own<const RpcResponse>> onResponse() const {
return redirectLater.addBranch();
RpcPipeline(const RpcConnectionState& connectionState, kj::Own<const QuestionRef> questionRef)
: connectionState(kj::addRef(connectionState)),
resolveSelfPromise(nullptr) {
// Construct a new RpcPipeline that is never expected to resolve.
state.getWithoutLock().init<Waiting>(kj::mv(questionRef));
}
// implements PipelineHook ---------------------------------------
......@@ -1683,14 +1759,19 @@ private:
auto pipelineClient = kj::refcounted<PipelineClient>(
*connectionState, kj::addRef(*lock->get<Waiting>()), kj::heapArray(ops.asPtr()));
auto resolutionPromise = connectionState->eventLoop.there(redirectLater.addBranch(),
kj::mvCapture(ops,
[](kj::Array<PipelineOp> ops, kj::Own<const RpcResponse>&& response) {
return response->getResults().getPipelinedCap(ops);
}));
KJ_IF_MAYBE(r, redirectLater) {
auto resolutionPromise = connectionState->eventLoop.there(r->addBranch(),
kj::mvCapture(ops,
[](kj::Array<PipelineOp> ops, kj::Own<const RpcResponse>&& response) {
return response->getResults().getPipelinedCap(ops);
}));
return kj::refcounted<PromiseClient>(
*connectionState, kj::mv(pipelineClient), kj::mv(resolutionPromise), nullptr);
return kj::refcounted<PromiseClient>(
*connectionState, kj::mv(pipelineClient), kj::mv(resolutionPromise), nullptr);
} else {
// Oh, this pipeline will never get redirected, so just return the PipelineClient.
return kj::mv(pipelineClient);
}
} else if (lock->is<Resolved>()) {
return lock->get<Resolved>()->getResults().getPipelinedCap(ops);
} else {
......@@ -1701,7 +1782,7 @@ private:
private:
kj::Own<const RpcConnectionState> connectionState;
kj::Maybe<CapExtractorImpl&> capExtractor;
kj::ForkedPromise<kj::Own<const RpcResponse>> redirectLater;
kj::Maybe<kj::ForkedPromise<kj::Own<const RpcResponse>>> redirectLater;
typedef kj::Own<const QuestionRef> Waiting;
typedef kj::Own<const RpcResponse> Resolved;
......@@ -1761,30 +1842,24 @@ private:
RpcServerResponse(const RpcConnectionState& connectionState,
kj::Own<OutgoingRpcMessage>&& message,
ObjectPointer::Builder results)
: connectionState(connectionState),
message(kj::mv(message)),
injector(connectionState),
context(injector),
: message(kj::mv(message)),
injector(kj::heap<CapInjectorImpl>(connectionState)),
context(*injector),
builder(context.imbue(results)) {}
ObjectPointer::Builder getResults() {
return builder;
}
bool hasCaps() {
return injector.hasCaps();
}
void send() {
auto lock = connectionState.tables.lockExclusive();
injector.finishDescriptors(*lock);
kj::Own<CapInjectorImpl> send(Tables& lockedTables) {
injector->finishDescriptors(lockedTables);
message->send();
return kj::mv(injector);
}
private:
const RpcConnectionState& connectionState;
kj::Own<OutgoingRpcMessage> message;
CapInjectorImpl injector;
kj::Own<CapInjectorImpl> injector;
CapBuilderContext context;
ObjectPointer::Builder builder;
};
......@@ -1811,7 +1886,10 @@ private:
Orphanage::getForMessageContaining(returnMessage));
returnMessage.adoptRetainedCaps(kj::mv(retainedCaps.exportList));
KJ_ASSERT_NONNULL(response)->send();
kj::Own<const PipelineHook> pipelineToRelease;
auto lock = connectionState->tables.lockExclusive();
auto& tables = *lock;
cleanupAnswerTable(kj::mv(lock), KJ_ASSERT_NONNULL(response)->send(tables));
}
}
void sendErrorReturn(kj::Exception&& exception) {
......@@ -1828,6 +1906,7 @@ private:
fromException(exception, builder.initException());
message->send();
cleanupAnswerTable(connectionState->tables.lockExclusive(), nullptr);
}
}
void sendCancel() {
......@@ -1843,6 +1922,7 @@ private:
builder.setCanceled();
message->send();
cleanupAnswerTable(connectionState->tables.lockExclusive(), nullptr);
}
}
......@@ -1892,6 +1972,60 @@ private:
return results;
}
}
kj::Promise<void> tailCall(kj::Own<RequestHook> request) override {
KJ_REQUIRE(response == nullptr,
"Can't call tailCall() after initializing the results struct.");
releaseParams();
if (request->getBrand() == connectionState.get()) {
// The tail call is headed towards the peer that called us in the first place, so we can
// optimize out the return trip.
KJ_IF_MAYBE(tailInfo, kj::downcast<RpcRequest>(*request).tailSend()) {
// Link pipelines.
KJ_IF_MAYBE(f, tailCallPipelineFulfiller) {
f->get()->fulfill(ObjectPointer::Pipeline(kj::mv(tailInfo->pipeline)));
}
if (isFirstResponder()) {
auto message = connectionState->connection->newOutgoingMessage(
messageSizeHint<rpc::Return>() + requestCapExtractor.retainedListSizeHint(true));
auto builder = message->getBody().initAs<rpc::Message>().initReturn();
builder.setQuestionId(questionId);
auto retainedCaps = requestCapExtractor.finalizeRetainedCaps(
Orphanage::getForMessageContaining(builder));
builder.adoptRetainedCaps(kj::mv(retainedCaps.exportList));
builder.setTakeFromOtherAnswer(tailInfo->questionId);
message->send();
cleanupAnswerTable(connectionState->tables.lockExclusive(), nullptr);
}
return kj::mv(tailInfo->promise);
}
}
// Just forwarding to another local call.
auto promise = request->send();
// Link pipelines.
// KJ_IF_MAYBE(f, tailCallPipelineFulfiller) {
// f->get()->fulfill(kj::mv(kj::implicitCast<ObjectPointer::Pipeline&>(promise)));
// }
// Wait for response.
return promise.then([this](Response<ObjectPointer>&& tailResponse) {
// Copy the response.
// TODO(perf): It would be nice if we could somehow make the response get built in-place
// but requires some refactoring.
getResults(tailResponse.targetSizeInWords()).set(tailResponse);
});
}
kj::Promise<ObjectPointer::Pipeline> onTailCall() override {
auto paf = kj::newPromiseAndFulfiller<ObjectPointer::Pipeline>();
tailCallPipelineFulfiller = kj::mv(paf.fulfiller);
return kj::mv(paf.promise);
}
void allowAsyncCancellation() override {
if (threadAcceptingCancellation != nullptr) {
threadAcceptingCancellation = &kj::EventLoop::current();
......@@ -1927,6 +2061,7 @@ private:
kj::Maybe<kj::Own<RpcServerResponse>> response;
rpc::Return::Builder returnMessage;
bool responseSent = false;
kj::Maybe<kj::Own<kj::PromiseFulfiller<ObjectPointer::Pipeline>>> tailCallPipelineFulfiller;
// Cancellation state ----------------------------------
......@@ -1981,51 +2116,60 @@ private:
}
bool isFirstResponder() {
// The first time it is called, removes self from the answer table and returns true.
// On subsequent calls, returns false.
kj::Own<const PipelineHook> pipelineToRelease;
if (responseSent) {
return false;
} else {
responseSent = true;
return true;
}
}
// We need to remove the `callContext` pointer -- which points back to us -- from the
// answer table. Or we might even be responsible for removing the entire answer table
// entry.
auto lock = connectionState->tables.lockExclusive();
void cleanupAnswerTable(kj::Locked<Tables>&& lock,
kj::Maybe<kj::Own<CapInjectorImpl>> resultCaps) {
// We need to remove the `callContext` pointer -- which points back to us -- from the
// answer table. Or we might even be responsible for removing the entire answer table
// entry.
if (__atomic_load_n(&cancellationFlags, __ATOMIC_RELAXED) & CANCEL_REQUESTED) {
// We are responsible for deleting the answer table entry. Awkwardly, however, the
// answer table may be the only thing holding a reference to the context, and we may even
// be called from the continuation represented by answer.asyncOp. So we have to do the
// actual deletion asynchronously. But we have to remove it from the table *now*, while
// we still hold the lock, because once we send the return message the answer ID is free
// for reuse.
auto promise = connectionState->eventLoop.evalLater([]() {});
promise.attach(kj::mv(lock->answers[questionId]));
connectionState->tasks.add(kj::mv(promise));
// Erase from the table.
lock->answers.erase(questionId);
} else {
// We just have to null out callContext.
auto& answer = lock->answers[questionId];
answer.callContext = nullptr;
// If the response has no capabilities in it, then we should also delete the pipeline
// so that the context can be released sooner.
KJ_IF_MAYBE(r, response) {
if (!r->get()->hasCaps()) {
KJ_IF_MAYBE(pipeline, answer.pipeline) {
pipelineToRelease = kj::mv(*pipeline);
}
}
// TODO(cleanup): This code and the code that calls it is really ugly. Moving a lock as
// a parameter? Yuck. Maybe when cancellation and thread-safety are removed this will
// get simpler?
kj::Own<const PipelineHook> pipelineToRelease;
// Release lock later so that pipelineToRelease and resultCaps can be deleted without
// deadlock.
KJ_DEFER(lock.release());
if (__atomic_load_n(&cancellationFlags, __ATOMIC_RELAXED) & CANCEL_REQUESTED) {
// We are responsible for deleting the answer table entry. Awkwardly, however, the
// answer table may be the only thing holding a reference to the context, and we may even
// be called from the continuation represented by answer.asyncOp. So we have to do the
// actual deletion asynchronously. But we have to remove it from the table *now*, while
// we still hold the lock, because once we send the return message the answer ID is free
// for reuse.
auto promise = connectionState->eventLoop.evalLater([]() {});
promise.attach(kj::mv(lock->answers[questionId]));
connectionState->tasks.add(kj::mv(promise));
// Erase from the table.
lock->answers.erase(questionId);
} else {
// We just have to null out callContext.
auto& answer = lock->answers[questionId];
answer.callContext = nullptr;
// If the response has capabilities, we need to arrange to keep the CapInjector around
// until the `Finish` message.
// If the response has no capabilities in it, then we should also delete the pipeline
// so that the context can be released sooner.
if (!resultCaps.map([](kj::Own<CapInjectorImpl>& i) { return i->hasCaps(); })
.orDefault(false)) {
KJ_IF_MAYBE(pipeline, answer.pipeline) {
pipelineToRelease = kj::mv(*pipeline);
}
}
return true;
answer.resultCaps = kj::mv(resultCaps);
}
}
};
......@@ -2056,6 +2200,7 @@ private:
void handleMessage(kj::Own<IncomingRpcMessage> message) {
auto reader = message->getBody().getAs<rpc::Message>();
switch (reader.which()) {
case rpc::Message::UNIMPLEMENTED:
handleUnimplemented(reader.getUnimplemented());
......@@ -2270,6 +2415,9 @@ private:
switch (ret.which()) {
case rpc::Return::RESULTS:
KJ_REQUIRE(!question->isTailCall,
"Tail call `Return` must set `resultsSentElsewhere`, not `results`.");
KJ_IF_MAYBE(questionRef, question->selfRef) {
// The questionRef still exists, but could be being deleted in another thread.
KJ_IF_MAYBE(ownRef, kj::tryAddRef(*questionRef)) {
......@@ -2281,6 +2429,9 @@ private:
break;
case rpc::Return::EXCEPTION:
KJ_REQUIRE(!question->isTailCall,
"Tail call `Return` must set `resultsSentElsewhere`, not `results`.");
KJ_IF_MAYBE(questionRef, question->selfRef) {
// The questionRef still exists, but could be being deleted in another thread.
KJ_IF_MAYBE(ownRef, kj::tryAddRef(*questionRef)) {
......@@ -2290,6 +2441,9 @@ private:
break;
case rpc::Return::CANCELED:
KJ_REQUIRE(!question->isTailCall,
"Tail call `Return` must set `resultsSentElsewhere`, not `results`.");
KJ_REQUIRE(question->selfRef == nullptr,
"Return message falsely claims call was canceled.") { return; }
// We don't bother fulfilling the result. If someone is somehow still waiting on it
......@@ -2297,6 +2451,19 @@ private:
// being destroyed.
break;
case rpc::Return::RESULTS_SENT_ELSEWHERE:
KJ_REQUIRE(question->isTailCall,
"`Return` had `resultsSentElsewhere` but this was not a tail call.");
KJ_IF_MAYBE(questionRef, question->selfRef) {
// The questionRef still exists, but could be being deleted in another thread.
KJ_IF_MAYBE(ownRef, kj::tryAddRef(*questionRef)) {
// Not being deleted. Tail calls are fulfilled with a null pointer.
questionRef->fulfill(kj::Own<RpcResponse>());
}
}
break;
default:
KJ_FAIL_REQUIRE("Unknown return type (not answer, exception, or canceled).") { return; }
}
......@@ -2312,6 +2479,7 @@ private:
void handleFinish(const rpc::Finish::Reader& finish) {
kj::Own<ResolutionChain> chainToRelease;
Answer answerToRelease;
auto lock = tables.lockExclusive();
......@@ -2339,6 +2507,7 @@ private:
KJ_IF_MAYBE(context, answer->callContext) {
context->requestCancel();
} else {
answerToRelease = kj::mv(*answer);
lock->answers.erase(finish.getQuestionId());
}
} else {
......@@ -2448,8 +2617,8 @@ private:
const RpcClient& downcasted = kj::downcast<const RpcClient>(*target);
auto message = connection->newOutgoingMessage(messageSizeHint<rpc::Disembargo>() +
sizeInWords<rpc::MessageTarget>() + sizeInWords<rpc::PromisedAnswer>());
auto message = connection->newOutgoingMessage(
messageSizeHint<rpc::Disembargo>() + MESSAGE_TARGET_SIZE_HINT);
auto builder = message->getBody().initAs<rpc::Message>().initDisembargo();
{
......
......@@ -275,18 +275,31 @@ struct Call {
# The params may contain capabilities. These capabilities are automatically released when the
# call returns *unless* the Return message explicitly indicates that they are being retained.
sendReturnTo :union {
sendResultsTo :union {
# Where should the return message be sent?
caller @5 :Void;
# Send the return message back to the caller (the usual).
yourself @6 :QuestionId;
yourself @6 :Void;
# **(level 1)**
#
# This is actually an echo of a call originally made by the receiver, with the given question
# ID. The result of this call should directly resolve the original call, without ever sending
# a `Return` over the wire.
# Don't actually return the results to the sender. Instead, hold on to them and await
# instructions from the sender regarding what to do with them. In particular, the sender
# may subsequently send a `Return` for some other call (which the receiver had previously made
# to the sender) with `takeFromOtherAnswer` set. The results from this call are then used
# as the results of the other call.
#
# When `yourself` is used, the receiver must still send a `Return` for the call, but sets the
# field `resultsSentElsewhere` in that `Return` rather than including the results.
#
# This feature can be used to implement tail calls in which a call from Vat A to Vat B ends up
# returning the result of a call from Vat B back to Vat A.
#
# In particular, the most common use case for this feature is when Vat A makes a call to a
# promise in Vat B, and then that promise ends up resolving to a capability back in Vat A.
# Vat B must forward all the queued calls on that promise back to Vat A, but can set `yourself`
# in the calls so that the results need not pass back through Vat B.
#
# For example:
# - Alice, in Vat A, call foo() on Bob in Vat B.
......@@ -294,14 +307,18 @@ struct Call {
# - Later on, Bob resolves the promise from foo() to point at Carol, who lives in Vat A (next
# to Alice).
# - Vat B dutifully forwards the bar() call to Carol. Let us call this forwarded call bar'().
# - The `Call` for bar'() has `sendReturnTo` set to `yourself`, with the value being the
# Notice that bar() and bar'() are travelling in opposite directions on the same network
# link.
# - The `Call` for bar'() has `sendResultsTo` set to `yourself`, with the value being the
# question ID originally assigned to the bar() call.
# - Vat A receives bar'() and delivers it to Carol.
# - When bar'() returns, Vat A does *not* send a `Return` message to Vat B. Instead, it
# directly returns the result to Alice.
# - Vat A then sends a `Finish` message for bar().
# - Vat B, on receiving the `Finish`, sends a corresponding `Finish` for bar'().
# - Neither bar() nor bar'() ever see a `Return` message sent over the wire.
# - When bar'() returns, Vat A immediately takes the results and returns them from bar().
# - Meanwhile, Vat A sends a `Return` for bar'() to Vat B, with `resultsSentElsewhere` set in
# place of results.
# - Vat A sends a `Finish` for that call to Vat B.
# - Vat B receives the `Return` for bar'() and sends a `Return` for bar(), with
# `receivedFromYourself` set in place of the results.
# - Vat B receives the `Finish` for bar() and sends a `Finish` to bar'().
thirdParty @7 :RecipientId;
# **(level 3)**
......@@ -312,10 +329,9 @@ struct Call {
#
# This operates much like `yourself`, above, except that Carol is in a separate Vat C. `Call`
# messages are sent from Vat A -> Vat B and Vat B -> Vat C. A `Return` message is sent from
# Vat B -> Vat A that contains a `redirect` to Vat C. When Vat A sends an `Accept` to Vat C,
# it receives back a `Return` containing the call's actual result. Vat C never sends a `Return`
# to Vat B, although `Finish` messages must still be sent corresponding to every `Call` as well
# as the `Accept`.
# Vat B -> Vat A that contains `acceptFromThirdParty` in place of results. When Vat A sends
# an `Accept` to Vat C, it receives back a `Return` containing the call's actual result. Vat C
# also sends a `Return` to Vat B with `resultsSentElsewhere`.
}
}
......@@ -352,12 +368,20 @@ struct Return {
# Indicates that the call was canceled due to the caller sending a Finish message
# before the call had completed.
redirect @5 :ThirdPartyCapId;
resultsSentElsewhere @5 :Void;
# This is set when returning from a `Call` which had `sendResultsTo` set to something other
# than `caller`.
takeFromOtherAnswer @6 :QuestionId;
# 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
# results here.
acceptFromThirdParty @7 :ThirdPartyCapId;
# **(level 3)**
#
# The call has been redirected to another vat, and the result should be obtained by connecting
# to that vat directly. An `Accept` message sent to the vat will return the result. See
# `Call.sendReturnTo.thirdParty`.
# The caller should contact a third-party vat to pick up the results. An `Accept` message
# sent to the vat will return the result. This pairs with `Call.sendResultsTo.thirdParty`.
}
}
......
......@@ -246,7 +246,7 @@ const ::capnp::_::RawSchema s_91b79f1f808db032 = {
static const ::capnp::_::AlignedData<98> b_836a53ce789d4cd4 = {
{ 0, 0, 0, 0, 5, 0, 5, 0,
212, 76, 157, 120, 206, 83, 106, 131,
0, 0, 0, 0, 1, 0, 3, 0,
0, 0, 0, 0, 1, 0, 2, 0,
80, 162, 82, 37, 27, 152, 18, 179,
3, 0, 7, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
......@@ -298,7 +298,7 @@ static const ::capnp::_::AlignedData<98> b_836a53ce789d4cd4 = {
5, 0, 0, 0, 0, 0, 0, 0,
1, 0, 0, 0, 0, 0, 0, 0,
153, 95, 171, 26, 246, 176, 232, 218,
165, 0, 0, 0, 106, 0, 0, 0,
165, 0, 0, 0, 114, 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,
......@@ -340,8 +340,8 @@ static const ::capnp::_::AlignedData<98> b_836a53ce789d4cd4 = {
18, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
115, 101, 110, 100, 82, 101, 116, 117,
114, 110, 84, 111, 0, 0, 0, 0, }
115, 101, 110, 100, 82, 101, 115, 117,
108, 116, 115, 84, 111, 0, 0, 0, }
};
static const ::capnp::_::RawSchema* const d_836a53ce789d4cd4[] = {
&s_95bc14545813fbc1,
......@@ -356,11 +356,11 @@ const ::capnp::_::RawSchema s_836a53ce789d4cd4 = {
static const ::capnp::_::AlignedData<61> b_dae8b0f61aab5f99 = {
{ 0, 0, 0, 0, 5, 0, 5, 0,
153, 95, 171, 26, 246, 176, 232, 218,
21, 0, 0, 0, 1, 0, 3, 0,
21, 0, 0, 0, 1, 0, 2, 0,
212, 76, 157, 120, 206, 83, 106, 131,
3, 0, 7, 0, 1, 0, 3, 0,
3, 0, 0, 0, 0, 0, 0, 0,
17, 0, 0, 0, 18, 1, 0, 0,
17, 0, 0, 0, 26, 1, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
25, 0, 0, 0, 175, 0, 0, 0,
......@@ -368,8 +368,8 @@ static const ::capnp::_::AlignedData<61> b_dae8b0f61aab5f99 = {
99, 97, 112, 110, 112, 47, 114, 112,
99, 46, 99, 97, 112, 110, 112, 58,
67, 97, 108, 108, 46, 115, 101, 110,
100, 82, 101, 116, 117, 114, 110, 84,
111, 0, 0, 0, 0, 0, 0, 0,
100, 82, 101, 115, 117, 108, 116, 115,
84, 111, 0, 0, 0, 0, 0, 0,
12, 0, 0, 0, 3, 0, 4, 0,
0, 0, 255, 255, 0, 0, 0, 0,
0, 0, 1, 0, 5, 0, 0, 0,
......@@ -378,7 +378,7 @@ static const ::capnp::_::AlignedData<61> b_dae8b0f61aab5f99 = {
0, 0, 0, 0, 0, 0, 0, 0,
64, 0, 0, 0, 2, 0, 1, 0,
72, 0, 0, 0, 2, 0, 1, 0,
1, 0, 254, 255, 4, 0, 0, 0,
1, 0, 254, 255, 0, 0, 0, 0,
0, 0, 1, 0, 6, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
69, 0, 0, 0, 74, 0, 0, 0,
......@@ -401,10 +401,10 @@ static const ::capnp::_::AlignedData<61> b_dae8b0f61aab5f99 = {
0, 0, 0, 0, 0, 0, 0, 0,
121, 111, 117, 114, 115, 101, 108, 102,
0, 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,
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,
116, 104, 105, 114, 100, 80, 97, 114,
......@@ -425,65 +425,79 @@ const ::capnp::_::RawSchema s_dae8b0f61aab5f99 = {
0xdae8b0f61aab5f99, b_dae8b0f61aab5f99.words, 61, d_dae8b0f61aab5f99, m_dae8b0f61aab5f99,
1, 3, i_dae8b0f61aab5f99, nullptr, nullptr
};
static const ::capnp::_::AlignedData<108> b_9e19b28d3db3573a = {
static const ::capnp::_::AlignedData<141> b_9e19b28d3db3573a = {
{ 0, 0, 0, 0, 5, 0, 5, 0,
58, 87, 179, 61, 141, 178, 25, 158,
0, 0, 0, 0, 1, 0, 1, 0,
0, 0, 0, 0, 1, 0, 2, 0,
80, 162, 82, 37, 27, 152, 18, 179,
2, 0, 7, 0, 0, 0, 4, 0,
2, 0, 7, 0, 0, 0, 6, 0,
2, 0, 0, 0, 0, 0, 0, 0,
17, 0, 0, 0, 186, 0, 0, 0,
25, 0, 0, 0, 7, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
21, 0, 0, 0, 87, 1, 0, 0,
21, 0, 0, 0, 199, 1, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
99, 97, 112, 110, 112, 47, 114, 112,
99, 46, 99, 97, 112, 110, 112, 58,
82, 101, 116, 117, 114, 110, 0, 0,
0, 0, 0, 0, 1, 0, 1, 0,
24, 0, 0, 0, 3, 0, 4, 0,
32, 0, 0, 0, 3, 0, 4, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 1, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
153, 0, 0, 0, 90, 0, 0, 0,
209, 0, 0, 0, 90, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
152, 0, 0, 0, 2, 0, 1, 0,
160, 0, 0, 0, 2, 0, 1, 0,
208, 0, 0, 0, 2, 0, 1, 0,
216, 0, 0, 0, 2, 0, 1, 0,
1, 0, 0, 0, 0, 0, 0, 0,
0, 0, 1, 0, 1, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
157, 0, 0, 0, 106, 0, 0, 0,
213, 0, 0, 0, 106, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
156, 0, 0, 0, 2, 0, 1, 0,
176, 0, 0, 0, 2, 0, 1, 0,
212, 0, 0, 0, 2, 0, 1, 0,
232, 0, 0, 0, 2, 0, 1, 0,
2, 0, 255, 255, 1, 0, 0, 0,
0, 0, 1, 0, 2, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
173, 0, 0, 0, 66, 0, 0, 0,
229, 0, 0, 0, 66, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
168, 0, 0, 0, 2, 0, 1, 0,
176, 0, 0, 0, 2, 0, 1, 0,
224, 0, 0, 0, 2, 0, 1, 0,
232, 0, 0, 0, 2, 0, 1, 0,
3, 0, 254, 255, 1, 0, 0, 0,
0, 0, 1, 0, 3, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
173, 0, 0, 0, 82, 0, 0, 0,
229, 0, 0, 0, 82, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
172, 0, 0, 0, 2, 0, 1, 0,
180, 0, 0, 0, 2, 0, 1, 0,
228, 0, 0, 0, 2, 0, 1, 0,
236, 0, 0, 0, 2, 0, 1, 0,
4, 0, 253, 255, 0, 0, 0, 0,
0, 0, 1, 0, 4, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
177, 0, 0, 0, 74, 0, 0, 0,
233, 0, 0, 0, 74, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
176, 0, 0, 0, 2, 0, 1, 0,
184, 0, 0, 0, 2, 0, 1, 0,
5, 0, 252, 255, 1, 0, 0, 0,
232, 0, 0, 0, 2, 0, 1, 0,
240, 0, 0, 0, 2, 0, 1, 0,
5, 0, 252, 255, 0, 0, 0, 0,
0, 0, 1, 0, 5, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
181, 0, 0, 0, 74, 0, 0, 0,
237, 0, 0, 0, 170, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
240, 0, 0, 0, 2, 0, 1, 0,
248, 0, 0, 0, 2, 0, 1, 0,
6, 0, 251, 255, 2, 0, 0, 0,
0, 0, 1, 0, 6, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
245, 0, 0, 0, 162, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
248, 0, 0, 0, 2, 0, 1, 0,
0, 1, 0, 0, 2, 0, 1, 0,
7, 0, 250, 255, 1, 0, 0, 0,
0, 0, 1, 0, 7, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
180, 0, 0, 0, 2, 0, 1, 0,
188, 0, 0, 0, 2, 0, 1, 0,
253, 0, 0, 0, 170, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 1, 0, 0, 2, 0, 1, 0,
8, 1, 0, 0, 2, 0, 1, 0,
113, 117, 101, 115, 116, 105, 111, 110,
73, 100, 0, 0, 0, 0, 0, 0,
8, 0, 0, 0, 0, 0, 0, 0,
......@@ -526,8 +540,27 @@ static const ::capnp::_::AlignedData<108> 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,
114, 101, 100, 105, 114, 101, 99, 116,
114, 101, 115, 117, 108, 116, 115, 83,
101, 110, 116, 69, 108, 115, 101, 119,
104, 101, 114, 101, 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, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
116, 97, 107, 101, 70, 114, 111, 109,
79, 116, 104, 101, 114, 65, 110, 115,
119, 101, 114, 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,
8, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
97, 99, 99, 101, 112, 116, 70, 114,
111, 109, 84, 104, 105, 114, 100, 80,
97, 114, 116, 121, 0, 0, 0, 0,
18, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0,
......@@ -538,11 +571,11 @@ static const ::capnp::_::AlignedData<108> b_9e19b28d3db3573a = {
static const ::capnp::_::RawSchema* const d_9e19b28d3db3573a[] = {
&s_d625b7063acf691a,
};
static const uint16_t m_9e19b28d3db3573a[] = {4, 3, 0, 5, 2, 1};
static const uint16_t i_9e19b28d3db3573a[] = {2, 3, 4, 5, 0, 1};
static const uint16_t m_9e19b28d3db3573a[] = {7, 4, 3, 0, 2, 5, 1, 6};
static const uint16_t i_9e19b28d3db3573a[] = {2, 3, 4, 5, 6, 7, 0, 1};
const ::capnp::_::RawSchema s_9e19b28d3db3573a = {
0x9e19b28d3db3573a, b_9e19b28d3db3573a.words, 108, d_9e19b28d3db3573a, m_9e19b28d3db3573a,
1, 6, i_9e19b28d3db3573a, nullptr, nullptr
0x9e19b28d3db3573a, b_9e19b28d3db3573a.words, 141, d_9e19b28d3db3573a, m_9e19b28d3db3573a,
1, 8, i_9e19b28d3db3573a, nullptr, nullptr
};
static const ::capnp::_::AlignedData<49> b_d37d2eb2c2f80e63 = {
{ 0, 0, 0, 0, 5, 0, 5, 0,
......@@ -1747,7 +1780,7 @@ CAPNP_DEFINE_STRUCT(
CAPNP_DEFINE_STRUCT(
::capnp::rpc::Call);
CAPNP_DEFINE_STRUCT(
::capnp::rpc::Call::SendReturnTo);
::capnp::rpc::Call::SendResultsTo);
CAPNP_DEFINE_STRUCT(
::capnp::rpc::Return);
CAPNP_DEFINE_STRUCT(
......
......@@ -44,11 +44,11 @@ struct Call {
class Reader;
class Builder;
class Pipeline;
struct SendReturnTo;
struct SendResultsTo;
};
struct Call::SendReturnTo {
SendReturnTo() = delete;
struct Call::SendResultsTo {
SendResultsTo() = delete;
class Reader;
class Builder;
......@@ -70,7 +70,9 @@ struct Return {
RESULTS,
EXCEPTION,
CANCELED,
REDIRECT,
RESULTS_SENT_ELSEWHERE,
TAKE_FROM_OTHER_ANSWER,
ACCEPT_FROM_THIRD_PARTY,
};
};
......@@ -291,13 +293,13 @@ CAPNP_DECLARE_STRUCT(
1, 1, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT(
::capnp::rpc::Call, 836a53ce789d4cd4,
3, 3, INLINE_COMPOSITE);
2, 3, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT(
::capnp::rpc::Call::SendReturnTo, dae8b0f61aab5f99,
3, 3, INLINE_COMPOSITE);
::capnp::rpc::Call::SendResultsTo, dae8b0f61aab5f99,
2, 3, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT(
::capnp::rpc::Return, 9e19b28d3db3573a,
1, 2, INLINE_COMPOSITE);
2, 2, INLINE_COMPOSITE);
CAPNP_DECLARE_STRUCT(
::capnp::rpc::Finish, d37d2eb2c2f80e63,
1, 1, INLINE_COMPOSITE);
......@@ -623,7 +625,7 @@ public:
inline bool hasParams() const;
inline ::capnp::ObjectPointer::Reader getParams() const;
inline SendReturnTo::Reader getSendReturnTo() const;
inline SendResultsTo::Reader getSendResultsTo() const;
private:
::capnp::_::StructReader _reader;
......@@ -675,8 +677,8 @@ public:
inline ::capnp::ObjectPointer::Builder getParams();
inline ::capnp::ObjectPointer::Builder initParams();
inline SendReturnTo::Builder getSendReturnTo();
inline SendReturnTo::Builder initSendReturnTo();
inline SendResultsTo::Builder getSendResultsTo();
inline SendResultsTo::Builder initSendResultsTo();
private:
::capnp::_::StructBuilder _builder;
......@@ -699,16 +701,16 @@ public:
: _typeless(kj::mv(typeless)) {}
inline ::capnp::rpc::MessageTarget::Pipeline getTarget() const;
inline SendReturnTo::Pipeline getSendReturnTo() const;
inline SendResultsTo::Pipeline getSendResultsTo() const;
private:
::capnp::ObjectPointer::Pipeline _typeless;
template <typename T, ::capnp::Kind k>
friend struct ::capnp::ToDynamic_;
};
class Call::SendReturnTo::Reader {
class Call::SendResultsTo::Reader {
public:
typedef SendReturnTo Reads;
typedef SendResultsTo Reads;
Reader() = default;
inline explicit Reader(::capnp::_::StructReader base): _reader(base) {}
......@@ -722,7 +724,7 @@ public:
inline ::capnp::Void getCaller() const;
inline bool isYourself() const;
inline ::uint32_t getYourself() const;
inline ::capnp::Void getYourself() const;
inline bool isThirdParty() const;
inline bool hasThirdParty() const;
......@@ -738,16 +740,16 @@ private:
friend struct ::capnp::List;
friend class ::capnp::MessageBuilder;
friend class ::capnp::Orphanage;
friend ::kj::StringTree KJ_STRINGIFY(Call::SendReturnTo::Reader reader);
friend ::kj::StringTree KJ_STRINGIFY(Call::SendResultsTo::Reader reader);
};
inline ::kj::StringTree KJ_STRINGIFY(Call::SendReturnTo::Reader reader) {
return ::capnp::_::structString<Call::SendReturnTo>(reader._reader);
inline ::kj::StringTree KJ_STRINGIFY(Call::SendResultsTo::Reader reader) {
return ::capnp::_::structString<Call::SendResultsTo>(reader._reader);
}
class Call::SendReturnTo::Builder {
class Call::SendResultsTo::Builder {
public:
typedef SendReturnTo Builds;
typedef SendResultsTo Builds;
Builder() = delete; // Deleted to discourage incorrect usage.
// You can explicitly initialize to nullptr instead.
......@@ -764,8 +766,8 @@ public:
inline void setCaller( ::capnp::Void value = ::capnp::VOID);
inline bool isYourself();
inline ::uint32_t getYourself();
inline void setYourself( ::uint32_t value);
inline ::capnp::Void getYourself();
inline void setYourself( ::capnp::Void value = ::capnp::VOID);
inline bool isThirdParty();
inline bool hasThirdParty();
......@@ -777,16 +779,16 @@ private:
template <typename T, ::capnp::Kind k>
friend struct ::capnp::ToDynamic_;
friend class ::capnp::Orphanage;
friend ::kj::StringTree KJ_STRINGIFY(Call::SendReturnTo::Builder builder);
friend ::kj::StringTree KJ_STRINGIFY(Call::SendResultsTo::Builder builder);
};
inline ::kj::StringTree KJ_STRINGIFY(Call::SendReturnTo::Builder builder) {
return ::capnp::_::structString<Call::SendReturnTo>(builder._builder.asReader());
inline ::kj::StringTree KJ_STRINGIFY(Call::SendResultsTo::Builder builder) {
return ::capnp::_::structString<Call::SendResultsTo>(builder._builder.asReader());
}
class Call::SendReturnTo::Pipeline {
class Call::SendResultsTo::Pipeline {
public:
typedef SendReturnTo Pipelines;
typedef SendResultsTo Pipelines;
inline Pipeline(decltype(nullptr)): _typeless(nullptr) {}
inline explicit Pipeline(::capnp::ObjectPointer::Pipeline&& typeless)
......@@ -826,9 +828,15 @@ public:
inline bool isCanceled() const;
inline ::capnp::Void getCanceled() const;
inline bool isRedirect() const;
inline bool hasRedirect() const;
inline ::capnp::ObjectPointer::Reader getRedirect() const;
inline bool isResultsSentElsewhere() const;
inline ::capnp::Void getResultsSentElsewhere() const;
inline bool isTakeFromOtherAnswer() const;
inline ::uint32_t getTakeFromOtherAnswer() const;
inline bool isAcceptFromThirdParty() const;
inline bool hasAcceptFromThirdParty() const;
inline ::capnp::ObjectPointer::Reader getAcceptFromThirdParty() const;
private:
::capnp::_::StructReader _reader;
......@@ -889,10 +897,18 @@ public:
inline ::capnp::Void getCanceled();
inline void setCanceled( ::capnp::Void value = ::capnp::VOID);
inline bool isRedirect();
inline bool hasRedirect();
inline ::capnp::ObjectPointer::Builder getRedirect();
inline ::capnp::ObjectPointer::Builder initRedirect();
inline bool isResultsSentElsewhere();
inline ::capnp::Void getResultsSentElsewhere();
inline void setResultsSentElsewhere( ::capnp::Void value = ::capnp::VOID);
inline bool isTakeFromOtherAnswer();
inline ::uint32_t getTakeFromOtherAnswer();
inline void setTakeFromOtherAnswer( ::uint32_t value);
inline bool isAcceptFromThirdParty();
inline bool hasAcceptFromThirdParty();
inline ::capnp::ObjectPointer::Builder getAcceptFromThirdParty();
inline ::capnp::ObjectPointer::Builder initAcceptFromThirdParty();
private:
::capnp::_::StructBuilder _builder;
......@@ -3307,109 +3323,108 @@ inline ::capnp::ObjectPointer::Builder Call::Builder::initParams() {
return result;
}
inline Call::SendReturnTo::Reader Call::Reader::getSendReturnTo() const {
return Call::SendReturnTo::Reader(_reader);
inline Call::SendResultsTo::Reader Call::Reader::getSendResultsTo() const {
return Call::SendResultsTo::Reader(_reader);
}
inline Call::SendReturnTo::Builder Call::Builder::getSendReturnTo() {
return Call::SendReturnTo::Builder(_builder);
inline Call::SendResultsTo::Builder Call::Builder::getSendResultsTo() {
return Call::SendResultsTo::Builder(_builder);
}
inline Call::SendReturnTo::Pipeline Call::Pipeline::getSendReturnTo() const {
return Call::SendReturnTo::Pipeline(_typeless.noop());
inline Call::SendResultsTo::Pipeline Call::Pipeline::getSendResultsTo() const {
return Call::SendResultsTo::Pipeline(_typeless.noop());
}
inline Call::SendReturnTo::Builder Call::Builder::initSendReturnTo() {
inline Call::SendResultsTo::Builder Call::Builder::initSendResultsTo() {
_builder.setDataField< ::uint16_t>(3 * ::capnp::ELEMENTS, 0);
_builder.setDataField< ::uint32_t>(4 * ::capnp::ELEMENTS, 0);
_builder.getPointerField(2 * ::capnp::POINTERS).clear();
return Call::SendReturnTo::Builder(_builder);
return Call::SendResultsTo::Builder(_builder);
}
inline Call::SendReturnTo::Which Call::SendReturnTo::Reader::which() const {
inline Call::SendResultsTo::Which Call::SendResultsTo::Reader::which() const {
return _reader.getDataField<Which>(3 * ::capnp::ELEMENTS);
}
inline Call::SendReturnTo::Which Call::SendReturnTo::Builder::which() {
inline Call::SendResultsTo::Which Call::SendResultsTo::Builder::which() {
return _builder.getDataField<Which>(3 * ::capnp::ELEMENTS);
}
inline bool Call::SendReturnTo::Reader::isCaller() const {
return which() == Call::SendReturnTo::CALLER;
inline bool Call::SendResultsTo::Reader::isCaller() const {
return which() == Call::SendResultsTo::CALLER;
}
inline bool Call::SendReturnTo::Builder::isCaller() {
return which() == Call::SendReturnTo::CALLER;
inline bool Call::SendResultsTo::Builder::isCaller() {
return which() == Call::SendResultsTo::CALLER;
}
inline ::capnp::Void Call::SendReturnTo::Reader::getCaller() const {
KJ_IREQUIRE(which() == Call::SendReturnTo::CALLER,
inline ::capnp::Void Call::SendResultsTo::Reader::getCaller() const {
KJ_IREQUIRE(which() == Call::SendResultsTo::CALLER,
"Must check which() before get()ing a union member.");
return _reader.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline ::capnp::Void Call::SendReturnTo::Builder::getCaller() {
KJ_IREQUIRE(which() == Call::SendReturnTo::CALLER,
inline ::capnp::Void Call::SendResultsTo::Builder::getCaller() {
KJ_IREQUIRE(which() == Call::SendResultsTo::CALLER,
"Must check which() before get()ing a union member.");
return _builder.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline void Call::SendReturnTo::Builder::setCaller( ::capnp::Void value) {
_builder.setDataField<Call::SendReturnTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendReturnTo::CALLER);
inline void Call::SendResultsTo::Builder::setCaller( ::capnp::Void value) {
_builder.setDataField<Call::SendResultsTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendResultsTo::CALLER);
_builder.setDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS, value);
}
inline bool Call::SendReturnTo::Reader::isYourself() const {
return which() == Call::SendReturnTo::YOURSELF;
inline bool Call::SendResultsTo::Reader::isYourself() const {
return which() == Call::SendResultsTo::YOURSELF;
}
inline bool Call::SendReturnTo::Builder::isYourself() {
return which() == Call::SendReturnTo::YOURSELF;
inline bool Call::SendResultsTo::Builder::isYourself() {
return which() == Call::SendResultsTo::YOURSELF;
}
inline ::uint32_t Call::SendReturnTo::Reader::getYourself() const {
KJ_IREQUIRE(which() == Call::SendReturnTo::YOURSELF,
inline ::capnp::Void Call::SendResultsTo::Reader::getYourself() const {
KJ_IREQUIRE(which() == Call::SendResultsTo::YOURSELF,
"Must check which() before get()ing a union member.");
return _reader.getDataField< ::uint32_t>(
4 * ::capnp::ELEMENTS);
return _reader.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline ::uint32_t Call::SendReturnTo::Builder::getYourself() {
KJ_IREQUIRE(which() == Call::SendReturnTo::YOURSELF,
inline ::capnp::Void Call::SendResultsTo::Builder::getYourself() {
KJ_IREQUIRE(which() == Call::SendResultsTo::YOURSELF,
"Must check which() before get()ing a union member.");
return _builder.getDataField< ::uint32_t>(
4 * ::capnp::ELEMENTS);
return _builder.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline void Call::SendReturnTo::Builder::setYourself( ::uint32_t value) {
_builder.setDataField<Call::SendReturnTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendReturnTo::YOURSELF);
_builder.setDataField< ::uint32_t>(
4 * ::capnp::ELEMENTS, value);
inline void Call::SendResultsTo::Builder::setYourself( ::capnp::Void value) {
_builder.setDataField<Call::SendResultsTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendResultsTo::YOURSELF);
_builder.setDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS, value);
}
inline bool Call::SendReturnTo::Reader::isThirdParty() const {
return which() == Call::SendReturnTo::THIRD_PARTY;
inline bool Call::SendResultsTo::Reader::isThirdParty() const {
return which() == Call::SendResultsTo::THIRD_PARTY;
}
inline bool Call::SendReturnTo::Builder::isThirdParty() {
return which() == Call::SendReturnTo::THIRD_PARTY;
inline bool Call::SendResultsTo::Builder::isThirdParty() {
return which() == Call::SendResultsTo::THIRD_PARTY;
}
inline bool Call::SendReturnTo::Reader::hasThirdParty() const {
if (which() != Call::SendReturnTo::THIRD_PARTY) return false;
inline bool Call::SendResultsTo::Reader::hasThirdParty() const {
if (which() != Call::SendResultsTo::THIRD_PARTY) return false;
return !_reader.getPointerField(2 * ::capnp::POINTERS).isNull();
}
inline bool Call::SendReturnTo::Builder::hasThirdParty() {
if (which() != Call::SendReturnTo::THIRD_PARTY) return false;
inline bool Call::SendResultsTo::Builder::hasThirdParty() {
if (which() != Call::SendResultsTo::THIRD_PARTY) return false;
return !_builder.getPointerField(2 * ::capnp::POINTERS).isNull();
}
inline ::capnp::ObjectPointer::Reader Call::SendReturnTo::Reader::getThirdParty() const {
KJ_IREQUIRE(which() == Call::SendReturnTo::THIRD_PARTY,
inline ::capnp::ObjectPointer::Reader Call::SendResultsTo::Reader::getThirdParty() const {
KJ_IREQUIRE(which() == Call::SendResultsTo::THIRD_PARTY,
"Must check which() before get()ing a union member.");
return ::capnp::ObjectPointer::Reader(
_reader.getPointerField(2 * ::capnp::POINTERS));
}
inline ::capnp::ObjectPointer::Builder Call::SendReturnTo::Builder::getThirdParty() {
KJ_IREQUIRE(which() == Call::SendReturnTo::THIRD_PARTY,
inline ::capnp::ObjectPointer::Builder Call::SendResultsTo::Builder::getThirdParty() {
KJ_IREQUIRE(which() == Call::SendResultsTo::THIRD_PARTY,
"Must check which() before get()ing a union member.");
return ::capnp::ObjectPointer::Builder(
_builder.getPointerField(2 * ::capnp::POINTERS));
}
inline ::capnp::ObjectPointer::Builder Call::SendReturnTo::Builder::initThirdParty() {
_builder.setDataField<Call::SendReturnTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendReturnTo::THIRD_PARTY);
inline ::capnp::ObjectPointer::Builder Call::SendResultsTo::Builder::initThirdParty() {
_builder.setDataField<Call::SendResultsTo::Which>(
3 * ::capnp::ELEMENTS, Call::SendResultsTo::THIRD_PARTY);
auto result = ::capnp::ObjectPointer::Builder(
_builder.getPointerField(2 * ::capnp::POINTERS));
result.clear();
......@@ -3586,35 +3601,87 @@ inline void Return::Builder::setCanceled( ::capnp::Void value) {
0 * ::capnp::ELEMENTS, value);
}
inline bool Return::Reader::isRedirect() const {
return which() == Return::REDIRECT;
inline bool Return::Reader::isResultsSentElsewhere() const {
return which() == Return::RESULTS_SENT_ELSEWHERE;
}
inline bool Return::Builder::isResultsSentElsewhere() {
return which() == Return::RESULTS_SENT_ELSEWHERE;
}
inline ::capnp::Void Return::Reader::getResultsSentElsewhere() const {
KJ_IREQUIRE(which() == Return::RESULTS_SENT_ELSEWHERE,
"Must check which() before get()ing a union member.");
return _reader.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline ::capnp::Void Return::Builder::getResultsSentElsewhere() {
KJ_IREQUIRE(which() == Return::RESULTS_SENT_ELSEWHERE,
"Must check which() before get()ing a union member.");
return _builder.getDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS);
}
inline void Return::Builder::setResultsSentElsewhere( ::capnp::Void value) {
_builder.setDataField<Return::Which>(
2 * ::capnp::ELEMENTS, Return::RESULTS_SENT_ELSEWHERE);
_builder.setDataField< ::capnp::Void>(
0 * ::capnp::ELEMENTS, value);
}
inline bool Return::Reader::isTakeFromOtherAnswer() const {
return which() == Return::TAKE_FROM_OTHER_ANSWER;
}
inline bool Return::Builder::isTakeFromOtherAnswer() {
return which() == Return::TAKE_FROM_OTHER_ANSWER;
}
inline ::uint32_t Return::Reader::getTakeFromOtherAnswer() const {
KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_ANSWER,
"Must check which() before get()ing a union member.");
return _reader.getDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS);
}
inline ::uint32_t Return::Builder::getTakeFromOtherAnswer() {
KJ_IREQUIRE(which() == Return::TAKE_FROM_OTHER_ANSWER,
"Must check which() before get()ing a union member.");
return _builder.getDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS);
}
inline void Return::Builder::setTakeFromOtherAnswer( ::uint32_t value) {
_builder.setDataField<Return::Which>(
2 * ::capnp::ELEMENTS, Return::TAKE_FROM_OTHER_ANSWER);
_builder.setDataField< ::uint32_t>(
2 * ::capnp::ELEMENTS, value);
}
inline bool Return::Reader::isAcceptFromThirdParty() const {
return which() == Return::ACCEPT_FROM_THIRD_PARTY;
}
inline bool Return::Builder::isRedirect() {
return which() == Return::REDIRECT;
inline bool Return::Builder::isAcceptFromThirdParty() {
return which() == Return::ACCEPT_FROM_THIRD_PARTY;
}
inline bool Return::Reader::hasRedirect() const {
if (which() != Return::REDIRECT) return false;
inline bool Return::Reader::hasAcceptFromThirdParty() const {
if (which() != Return::ACCEPT_FROM_THIRD_PARTY) return false;
return !_reader.getPointerField(1 * ::capnp::POINTERS).isNull();
}
inline bool Return::Builder::hasRedirect() {
if (which() != Return::REDIRECT) return false;
inline bool Return::Builder::hasAcceptFromThirdParty() {
if (which() != Return::ACCEPT_FROM_THIRD_PARTY) return false;
return !_builder.getPointerField(1 * ::capnp::POINTERS).isNull();
}
inline ::capnp::ObjectPointer::Reader Return::Reader::getRedirect() const {
KJ_IREQUIRE(which() == Return::REDIRECT,
inline ::capnp::ObjectPointer::Reader Return::Reader::getAcceptFromThirdParty() const {
KJ_IREQUIRE(which() == Return::ACCEPT_FROM_THIRD_PARTY,
"Must check which() before get()ing a union member.");
return ::capnp::ObjectPointer::Reader(
_reader.getPointerField(1 * ::capnp::POINTERS));
}
inline ::capnp::ObjectPointer::Builder Return::Builder::getRedirect() {
KJ_IREQUIRE(which() == Return::REDIRECT,
inline ::capnp::ObjectPointer::Builder Return::Builder::getAcceptFromThirdParty() {
KJ_IREQUIRE(which() == Return::ACCEPT_FROM_THIRD_PARTY,
"Must check which() before get()ing a union member.");
return ::capnp::ObjectPointer::Builder(
_builder.getPointerField(1 * ::capnp::POINTERS));
}
inline ::capnp::ObjectPointer::Builder Return::Builder::initRedirect() {
inline ::capnp::ObjectPointer::Builder Return::Builder::initAcceptFromThirdParty() {
_builder.setDataField<Return::Which>(
2 * ::capnp::ELEMENTS, Return::REDIRECT);
2 * ::capnp::ELEMENTS, Return::ACCEPT_FROM_THIRD_PARTY);
auto result = ::capnp::ObjectPointer::Builder(
_builder.getPointerField(1 * ::capnp::POINTERS));
result.clear();
......
......@@ -22,6 +22,7 @@
// SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#include "test-util.h"
#include <kj/debug.h>
#include <gtest/gtest.h>
namespace capnp {
......@@ -932,5 +933,43 @@ TestPipelineImpl::TestPipelineImpl(int& callCount): callCount(callCount) {}
});
}
kj::Promise<void> TestCallOrderImpl::getCallSequence(
test::TestCallOrder::GetCallSequenceParams::Reader params,
test::TestCallOrder::GetCallSequenceResults::Builder result) {
result.setN(count++);
return kj::READY_NOW;
}
TestTailCallerImpl::TestTailCallerImpl(int& callCount): callCount(callCount) {}
kj::Promise<void> TestTailCallerImpl::fooAdvanced(
capnp::CallContext<test::TestTailCaller::FooParams,
test::TestTailCallee::TailResult> context) {
++callCount;
auto params = context.getParams();
auto tailRequest = params.getCallee().fooRequest();
tailRequest.setI(params.getI());
tailRequest.setT("from TestTailCaller");
return context.tailCall(kj::mv(tailRequest));
}
TestTailCalleeImpl::TestTailCalleeImpl(int& callCount): callCount(callCount) {}
kj::Promise<void> TestTailCalleeImpl::fooAdvanced(
capnp::CallContext<test::TestTailCallee::FooParams,
test::TestTailCallee::TailResult> context) {
++callCount;
auto params = context.getParams();
auto results = context.getResults();
results.setI(params.getI());
results.setT(params.getT());
results.setC(kj::heap<TestCallOrderImpl>());
return kj::READY_NOW;
}
} // namespace _ (private)
} // namespace capnp
......@@ -187,6 +187,40 @@ private:
int& callCount;
};
class TestCallOrderImpl final: public test::TestCallOrder::Server {
public:
kj::Promise<void> getCallSequence(
test::TestCallOrder::GetCallSequenceParams::Reader params,
test::TestCallOrder::GetCallSequenceResults::Builder result) override;
private:
uint count = 0;
};
class TestTailCallerImpl final: public test::TestTailCaller::Server {
public:
TestTailCallerImpl(int& callCount);
kj::Promise<void> fooAdvanced(
capnp::CallContext<test::TestTailCaller::FooParams,
test::TestTailCallee::TailResult> context) override;
private:
int& callCount;
};
class TestTailCalleeImpl final: public test::TestTailCallee::Server {
public:
TestTailCalleeImpl(int& callCount);
kj::Promise<void> fooAdvanced(
capnp::CallContext<test::TestTailCallee::FooParams,
test::TestTailCallee::TailResult> context) override;
private:
int& callCount;
};
} // namespace _ (private)
} // namespace capnp
......
......@@ -610,6 +610,25 @@ interface TestPipeline {
}
}
interface TestCallOrder {
getCallSequence @0 () -> (n: UInt32);
# First call returns 0, next returns 1, ...
}
interface TestTailCallee {
struct TailResult {
i @0 :UInt32;
t @1 :Text;
c @2 :TestCallOrder;
}
foo @0 (i :Int32, t :Text) -> TailResult;
}
interface TestTailCaller {
foo @0 (i :Int32, callee :TestTailCallee) -> TestTailCallee.TailResult;
}
struct TestSturdyRefHostId {
host @0 :Text;
}
......@@ -620,6 +639,8 @@ struct TestSturdyRefObjectId {
testInterface @0;
testExtends @1;
testPipeline @2;
testTailCallee @3;
testTailCaller @4;
}
}
......
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