From 876689e763ad560a58c0d1620f31489436d10e86 Mon Sep 17 00:00:00 2001 From: Brian Warner Date: Tue, 1 Mar 2022 20:42:02 -0800 Subject: [PATCH 1/2] refactor(swingset): improve vat deliveries This moves some logic out of `deliverAndLogToVal` and into a higher-level dispatch function, where it can react to all failure modes in a single place. We pull "run-queue events" off the run-queue (currently named the "acceptance queue". Many of these events are aimed at a specific vat, including 'notify', but the primary one is 'send' and is aimed at a *target* kref, which is either a kernel object or a kernel promise. For these we use `routeSendEvent` to either queue the event on a promise queue, reject it if it can't be delivered, or deliver it to a specific vat. This is the part that will change with #3465: in that world, each vat-input-queue exists for a specific vat, so the routing step moves earlier (into a "routing crank" that pulls events from a vat-output-queue and possibly adds them to a vat-input-queue). Some kinds of deliveries care about metering (or don't), and some have opinions about what should happen if a crank gets unwound. These are now reported as additional properties in the DeliveryStatus object that is returned by this delivery path. `processDeliveryMessage` looks at these properties, plus the delivery status (if any) to decide what to do at the end of the crank. I think most of the behavior should be the same as before. One change is that the `runPolicy` will probably get more information about non-'send' cranks than before (e.g. `create-vat` might report metering). This refactoring should make it easier to implement #1848 vat-upgrade, as well as #3465 queueing changes. closes #4687 --- packages/SwingSet/src/kernel/kernel.js | 637 +++++++++++------- packages/SwingSet/src/kernel/kernelSyscall.js | 4 +- .../SwingSet/src/kernel/state/kernelKeeper.js | 33 +- .../SwingSet/src/kernel/state/vatKeeper.js | 4 +- packages/SwingSet/src/types-exported.js | 23 + packages/SwingSet/src/types-internal.js | 17 + 6 files changed, 457 insertions(+), 261 deletions(-) create mode 100644 packages/SwingSet/src/types-internal.js diff --git a/packages/SwingSet/src/kernel/kernel.js b/packages/SwingSet/src/kernel/kernel.js index 39db361a38c..9a0a6d0d796 100644 --- a/packages/SwingSet/src/kernel/kernel.js +++ b/packages/SwingSet/src/kernel/kernel.js @@ -266,18 +266,6 @@ export default function buildKernel( } } - let terminationTrigger; - let postAbortActions; - - function resetDeliveryTriggers() { - terminationTrigger = undefined; - postAbortActions = { - meterDeductions: [], // list of { meterID, compute } - discardFailedDelivery: false, - }; - } - resetDeliveryTriggers(); - function notifyMeterThreshold(meterID) { // tell vatAdmin that a meter has dropped below its notifyThreshold const { remaining } = kernelKeeper.getMeter(meterID); @@ -286,35 +274,35 @@ export default function buildKernel( queueToKref(vatAdminRootKref, 'meterCrossedThreshold', args, 'logFailure'); } - function deductMeter(meterID, compute, firstTime) { - assert.typeof(compute, 'bigint'); - const res = kernelKeeper.deductMeter(meterID, compute); + // TODO: instead of using a kernel-wide flag here, consider making each + // VatManager responsible for remembering if/when a KernelSyscallResult + // reports a non-'ok' status and therefore the vat is toast. Then the + // delivery handler could ask the manager (or vat-warehouse) afterwards for + // the sticky-fatal state. If we did that, we wouldn't need + // `vatFatalSyscall`. We'd still need a way for `requestTermination` to + // work, though. - // We record the deductMeter() in postAbortActions.meterDeductions. If - // the delivery is rewound for any reason (syscall error, res.underflow), - // then deliverAndLogToVat will repeat the deductMeter (which will repeat - // the notifyMeterThreshold), so their side-effects will survive the - // abortCrank(). But we don't record it (again) during the repeat, to - // make sure exactly one copy of the changes will be committed. + let terminationTrigger; - if (firstTime) { - postAbortActions.meterDeductions.push({ meterID, compute }); - } - if (res.notify) { - notifyMeterThreshold(meterID); + // this is called for syscall.exit, which allows the crank to complete + // before terminating the vat + function requestTermination(vatID, reject, info) { + insistCapData(info); + // if vatFatalSyscall was here already, don't override: bad syscalls win + if (!terminationTrigger) { + terminationTrigger = { vatID, abortCrank: false, reject, info }; } - return res.underflow; } - // this is called for syscall.exit (shouldAbortCrank=false), and for any - // vat-fatal errors (shouldAbortCrank=true) - function setTerminationTrigger(vatID, shouldAbortCrank, shouldReject, info) { - if (shouldAbortCrank) { - assert(shouldReject); - } - if (!terminationTrigger || shouldAbortCrank) { - terminationTrigger = { vatID, shouldAbortCrank, shouldReject, info }; - } + // this is called for vat-fatal syscall errors, which aborts the crank and + // then terminates the vat + function vatFatalSyscall(vatID, problem) { + terminationTrigger = { + vatID, + abortCrank: true, + reject: true, + info: makeError(problem), + }; } const kernelSyscallHandler = makeKernelSyscallHandler({ @@ -323,101 +311,87 @@ export default function buildKernel( doSend, doSubscribe, doResolve, - setTerminationTrigger, + requestTermination, }); + /** + * + * @typedef { { compute: number } } MeterConsumption + * + * Any delivery crank (send, notify, start-vat.. anything which is allowed + * to make vat delivery) emits one of these status events if a delivery + * actually happened. + * + * @typedef { { + * vatID?: VatID, // vat to which the delivery was made + * metering?: MeterConsumption | null, // delivery metering results + * useMeter?: boolean, // this delivery should count against the vat's meter + * decrementReapCount?: boolean, // the reap counter should decrement + * discardFailedDelivery?: boolean, // crank abort should not repeat the delivery + * terminate?: string | null, // vat should be terminated + * } } DeliveryStatus + * + */ + /** * Perform one delivery to a vat. * - * @param {string} vatID - * @param {*} kd + * @param {VatID} vatID + * @param {KernelDeliveryObject} kd * @param {VatDeliveryObject} vd - * @param {boolean} useMeter - * @returns {Promise} + * @returns {Promise} */ - async function deliverAndLogToVat(vatID, kd, vd, useMeter) { - /** @type {PolicyInputCrankComplete} */ - let policyInput = ['crank', {}]; + async function deliverAndLogToVat(vatID, kd, vd) { // eslint-disable-next-line no-use-before-define assert(vatWarehouse.lookup(vatID)); - const vatKeeper = kernelKeeper.provideVatKeeper(vatID); - const { meterID } = vatKeeper.getOptions(); // Ensure that the vatSlogger is available before clist translation. const vs = kernelSlog.provideVatSlogger(vatID).vatSlog; try { + /** @type { VatDeliveryResult } */ // eslint-disable-next-line no-use-before-define const deliveryResult = await vatWarehouse.deliverToVat(vatID, kd, vd, vs); - insistVatDeliveryResult(deliveryResult); - if (vd[0] !== 'bringOutYourDead') { - if (vatKeeper.countdownToReap()) { - kernelKeeper.scheduleReap(vatID); - } - } - const [status, problem] = deliveryResult; - if (status !== 'ok') { - // probably a metering fault, or a bug in the vat's dispatch() - console.log(`delivery problem, terminating vat ${vatID}`, problem); - setTerminationTrigger(vatID, true, true, makeError(problem)); - return harden(['crank-failed', {}]); - } + insistVatDeliveryResult(deliveryResult); + // const [ ok, problem, usage ] = deliveryResult; if (deliveryResult[0] === 'ok') { - let used; - const metering = deliveryResult[2]; - if (metering) { - // if the result has metering, we report it to the runPolicy - const consumed = metering.compute; - assert.typeof(consumed, 'number'); - used = BigInt(consumed); - policyInput = ['crank', { computrons: used }]; - if (useMeter && meterID) { - // If we have a Meter and we want to use it, do so. - const underflow = deductMeter(meterID, used, true); - if (underflow) { - console.log( - `meter ${meterID} underflow, terminating vat ${vatID}`, - ); - const err = makeError('meter underflow, vat terminated'); - setTerminationTrigger(vatID, true, true, err); - return harden(['crank-failed', {}]); - } - } - } + return { metering: deliveryResult[2] }; + } else { + // probably a hard metering fault, or a bug in the vat's dispatch() + return { terminate: deliveryResult[1] }; // might be dead } } catch (e) { // log so we get a stack trace console.error(`error in kernel.deliver:`, e); throw e; } - return harden(policyInput); } /** * Deliver one message to a vat. * - * @param { string } vatID + * @param { VatID } vatID * @param { string } target - * @param { * } msg - * @returns { Promise } + * @param { Message } msg + * @returns { Promise } */ - async function deliverToVat(vatID, target, msg) { + async function processSend(vatID, target, msg) { insistMessage(msg); - /** @type { PolicyInput } */ - let policyInput = ['none']; kernelKeeper.incStat('dispatches'); kernelKeeper.incStat('dispatchDeliver'); // eslint-disable-next-line no-use-before-define if (!vatWarehouse.lookup(vatID)) { + // splat if (msg.result) { resolveToError(msg.result, VAT_TERMINATION_ERROR); } - } else { - const kd = harden(['message', target, msg]); - // eslint-disable-next-line no-use-before-define - const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); - policyInput = await deliverAndLogToVat(vatID, kd, vd, true); + return null; } - return harden(policyInput); + + /** @type { KernelDeliveryMessage } */ + const kd = harden(['message', target, msg]); + // eslint-disable-next-line no-use-before-define + const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); + return deliverAndLogToVat(vatID, kd, vd); } function extractPresenceIfPresent(data) { @@ -441,65 +415,66 @@ export default function buildKernel( /** * - * @param { * } message - * @returns { Promise } + * @param { RunQueueEventNotify } message + * @returns { Promise } */ async function processNotify(message) { const { vatID, kpid } = message; - /** @type { PolicyInput } */ - let policyInput = ['none']; insistVatID(vatID); insistKernelType('promise', kpid); kernelKeeper.incStat('dispatches'); // eslint-disable-next-line no-use-before-define if (!vatWarehouse.lookup(vatID)) { kdebug(`dropping notify of ${kpid} to ${vatID} because vat is dead`); - } else { - const p = kernelKeeper.getKernelPromise(kpid); - kernelKeeper.incStat('dispatchNotify'); - const vatKeeper = kernelKeeper.provideVatKeeper(vatID); + return null; + } - assert(p.state !== 'unresolved', X`spurious notification ${kpid}`); - const resolutions = []; - if (!vatKeeper.hasCListEntry(kpid)) { - kdebug(`vat ${vatID} has no c-list entry for ${kpid}`); - kdebug(`skipping notify of ${kpid} because it's already been done`); - return harden(policyInput); - } - const targets = getKpidsToRetire(kernelKeeper, kpid, p.data); - if (targets.length === 0) { - kdebug(`no kpids to retire`); - kdebug(`skipping notify of ${kpid} because it's already been done`); - return harden(policyInput); - } - for (const toResolve of targets) { - resolutions.push([toResolve, kernelKeeper.getKernelPromise(toResolve)]); - } - const kd = harden(['notify', resolutions]); - // eslint-disable-next-line no-use-before-define - const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); - vatKeeper.deleteCListEntriesForKernelSlots(targets); - policyInput = await deliverAndLogToVat(vatID, kd, vd, true); + const p = kernelKeeper.getKernelPromise(kpid); + kernelKeeper.incStat('dispatchNotify'); + const vatKeeper = kernelKeeper.provideVatKeeper(vatID); + + assert(p.state !== 'unresolved', X`spurious notification ${kpid}`); + /** @type { KernelDeliveryOneNotify[] } */ + const resolutions = []; + if (!vatKeeper.hasCListEntry(kpid)) { + kdebug(`vat ${vatID} has no c-list entry for ${kpid}`); + kdebug(`skipping notify of ${kpid} because it's already been done`); + return null; } - return harden(policyInput); + const targets = getKpidsToRetire(kernelKeeper, kpid, p.data); + if (targets.length === 0) { + kdebug(`no kpids to retire`); + kdebug(`skipping notify of ${kpid} because it's already been done`); + return null; + } + for (const toResolve of targets) { + const { state, data } = kernelKeeper.getKernelPromise(toResolve); + resolutions.push([toResolve, { state, data }]); + } + /** @type { KernelDeliveryNotify } */ + const kd = harden(['notify', resolutions]); + // eslint-disable-next-line no-use-before-define + const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); + vatKeeper.deleteCListEntriesForKernelSlots(targets); + + return deliverAndLogToVat(vatID, kd, vd); } /** * - * @param { * } message - * @returns { Promise } + * @param { RunQueueEventDropExports | RunQueueEventRetireImports | RunQueueEventRetireExports } message + * @returns { Promise } */ async function processGCMessage(message) { // used for dropExports, retireExports, and retireImports - /** @type { PolicyInput } */ - let policyInput = ['none']; const { type, vatID, krefs } = message; // console.log(`-- processGCMessage(${vatID} ${type} ${krefs.join(',')})`); insistVatID(vatID); // eslint-disable-next-line no-use-before-define if (!vatWarehouse.lookup(vatID)) { - return harden(policyInput); // can't collect from the dead + return null; // can't collect from the dead } + /** @type { KernelDeliveryDropExports | KernelDeliveryRetireExports | KernelDeliveryRetireImports } */ const kd = harden([type, krefs]); if (type === 'retireExports') { for (const kref of krefs) { @@ -511,65 +486,66 @@ export default function buildKernel( } // eslint-disable-next-line no-use-before-define const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); - policyInput = await deliverAndLogToVat(vatID, kd, vd, false); - return harden(policyInput); + return deliverAndLogToVat(vatID, kd, vd); } /** * - * @param { * } message - * @returns { Promise } + * @param { RunQueueEventBringOutYourDead } message + * @returns { Promise } */ async function processBringOutYourDead(message) { - /** @type { PolicyInput } */ - let policyInput = ['none']; const { type, vatID } = message; // console.log(`-- processBringOutYourDead(${vatID})`); insistVatID(vatID); // eslint-disable-next-line no-use-before-define if (!vatWarehouse.lookup(vatID)) { - return harden(policyInput); // can't collect from the dead + return null; // can't collect from the dead } + /** @type { KernelDeliveryBringOutYourDead } */ const kd = harden([type]); // eslint-disable-next-line no-use-before-define const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); - policyInput = await deliverAndLogToVat(vatID, kd, vd, false); - return harden(policyInput); + return deliverAndLogToVat(vatID, kd, vd); } - // The 'startVat' event is queued by `initializeKernel` for all static vats, - // so that we execute their bundle imports and call their `buildRootObject` - // functions in a transcript context. The consequence of this is that if - // there are N static vats, N 'startVat' events will be the first N events on - // the initial run queue. For dynamic vats, the handler of the 'create-vat' - // event, `processCreateVat`, calls `processStartVat` directly, rather than - // enqueing 'startVat', so that vat startup happens promptly after creation - // and so that there are no intervening events in the run queue between vat - // creation and vat startup (it would probably not be a problem if there were, - // but doing it this way simply guarantees there won't be such a problem - // without requiring any further analysis to be sure). + /** + * The 'startVat' event is queued by `initializeKernel` for all static vats, + * so that we execute their bundle imports and call their `buildRootObject` + * functions in a transcript context. The consequence of this is that if + * there are N static vats, N 'startVat' events will be the first N events on + * the initial run queue. For dynamic vats, the handler of the 'create-vat' + * event, `processCreateVat`, calls `processStartVat` directly, rather than + * enqueing 'startVat', so that vat startup happens promptly after creation + * and so that there are no intervening events in the run queue between vat + * creation and vat startup (it would probably not be a problem if there were, + * but doing it this way simply guarantees there won't be such a problem + * without requiring any further analysis to be sure). + * + * @param { RunQueueEventStartVat } message + * @returns { Promise } + */ async function processStartVat(message) { - postAbortActions.discardFailedDelivery = true; const { type, vatID } = message; // console.log(`-- processStartVat(${vatID})`); insistVatID(vatID); // eslint-disable-next-line no-use-before-define assert(vatWarehouse.lookup(vatID)); + /** @type { KernelDeliveryStartVat } */ const kd = harden([type]); // TODO(4381) add vatParameters here // eslint-disable-next-line no-use-before-define const vd = vatWarehouse.kernelDeliveryToVatDelivery(vatID, kd); // TODO: can we provide a computron count to the run policy? - const policyInput = await deliverAndLogToVat(vatID, kd, vd, false); - return harden(policyInput); + const status = await deliverAndLogToVat(vatID, kd, vd); + return { ...status, discardFailedDelivery: true }; } /** * - * @param { * } message - * @returns { Promise } + * @param { RunQueueEventCreateVat } message + * @returns { Promise } */ async function processCreateVat(message) { - postAbortActions.discardFailedDelivery = true; assert(vatAdminRootKref, `initializeKernel did not set vatAdminRootKref`); const { vatID, source, dynamicOptions } = message; kernelKeeper.addDynamicVatID(vatID); @@ -585,65 +561,65 @@ export default function buildKernel( vatKeeper.initializeReapCountdown(options.reapInterval); const { enableSetup } = options; - function makeSuccessResponse() { + function sendNewVatCallback(args) { + // @ts-ignore see assert(...) above + queueToKref(vatAdminRootKref, 'newVatCallback', args, 'logFailure'); + } + + function makeSuccessResponse(status) { // build success message, giving admin vat access to the new vat's root // object const kernelRootObjSlot = exportRootObject(kernelKeeper, vatID); - return { + const args = { body: JSON.stringify([ vatID, { rootObject: { '@qclass': 'slot', index: 0 } }, ]), slots: [kernelRootObjSlot], }; + sendNewVatCallback(args); + return { ...status, discardFailedDelivery: true }; } function makeErrorResponse(error) { // delete partial vat state kernelKeeper.cleanupAfterTerminatedVat(vatID); - return { + const args = { body: JSON.stringify([vatID, { error: `${error}` }]), slots: [], }; + sendNewVatCallback(args); + // ?? will this cause double-termination? or just get unwound? + return { terminate: error, discardFailedDelivery: true }; } - function sendResponse(args) { - // @ts-ignore see assert(...) above - queueToKref(vatAdminRootKref, 'newVatCallback', args, 'logFailure'); - } - - /** @type { PolicyInput } */ - const policyInput = harden(['create-vat', {}]); - // TODO: combine this with the return value from processStartVat + // TODO warner think through failure paths return ( // eslint-disable-next-line no-use-before-define vatWarehouse .createDynamicVat(vatID) // if createDynamicVat fails, go directly to makeErrorResponse - .then(_ => + .then(_vatinfo => enableSetup ? null : processStartVat({ type: 'startVat', vatID }), ) // TODO(4381) add vatParameters here - // Like any other run queue event handler, if processStartVat fails it - // returns a PolicyInput object indicating the problem rather than - // throwing directly. Consequently, if it fails, either during module - // initialization or during the call to `buildRootObject`, the result - // promise we are waiting on here will resolve successfully and get - // handled by the makeSuccessResponse-generated resolve handler rather - // than by the makeErrorResponse-generated rejection handler -- that is, - // it's a "success", but the value of the success indicates an - // underlying problem. However, `deliverAndLogToVat` will also have set - // the vat's termination trigger, resulting in (1) this crank being - // terminated in an error state, (2) the (incorrectly initialized) vat - // being terminated and expunged, and (3) the vatAdmin vat being - // notified of the termination. + // If processStartVat/deliverAndLogToVat observes a worker error, it + // will return status={ terminate: problem } rather than throw an + // error, so makeSuccessResponse will sendNewVatCallback. But the + // status is passed through, so processDeliveryMessage() will + // terminate the half-started vat and abort the crank, undoing + // sendNewVatCallback. processDeliveryMessage() is responsible for + // notifying vat-admin of the termination after doing abortCrank(). .then(makeSuccessResponse, makeErrorResponse) - .then(sendResponse) .catch(err => console.error(`error in vat creation`, err)) - .then(() => policyInput) ); } + /** + * + * @param { RunQueueEventUpgradeVat } message + * @returns { Promise } + */ async function processUpgradeVat(message) { assert(vatAdminRootKref, `initializeKernel did not set vatAdminRootKref`); // const { upgradeID, bundleID, vatParameters } = message; @@ -656,9 +632,7 @@ export default function buildKernel( slots: [], }; queueToKref(vatAdminRootKref, 'vatUpgradeCallback', args, 'logFailure'); - /** @type { PolicyInput } */ - const policyInput = ['none']; - return policyInput; + return null; // no delivery made (yet) } function legibilizeMessage(message) { @@ -686,7 +660,7 @@ export default function buildKernel( } } - /* + /** * routeSend(message) figures out where a 'send' event should go. If the * message needs to be queued (it is sent to an unresolved promise without * a pipelining decider), this queues it, and returns null. If the message @@ -697,8 +671,10 @@ export default function buildKernel( * this returns that settled object. * * This does not decrement any refcounts. The caller should do that. + * + * @param { RunQueueEventSend } message + * @returns { { vatID: VatID, targetObject: string } | null } */ - function routeSendEvent(message) { const { target, msg } = message; const { type } = parseKernelSlot(target); @@ -786,110 +762,261 @@ export default function buildKernel( const gcMessages = ['dropExports', 'retireExports', 'retireImports']; - async function deliverRunQueueEvent(message) { - /** @type { PolicyInput } */ - let policyInput = ['none']; + /** + * @typedef { import('../types-internal.js').VatID } VatID + * @typedef { import('../types-internal.js').InternalDynamicVatOptions } InternalDynamicVatOptions + * + * @typedef { { type: 'notify', vatID: VatID, kpid: string } } RunQueueEventNotify + * @typedef { { type: 'send', target: string, msg: Message }} RunQueueEventSend + * @typedef { { type: 'create-vat', vatID: VatID, + * source: { bundle: Bundle } | { bundleID: BundleID }, + * dynamicOptions: InternalDynamicVatOptions } + * } RunQueueEventCreateVat + * @typedef { { type: 'upgrade-vat', vatID: VatID, upgradeID: string, + * bundleID: BundleID, vatParameters: SwingSetCapData } } RunQueueEventUpgradeVat + * @typedef { { type: 'startVat', vatID: VatID } } RunQueueEventStartVat + * @typedef { { type: 'dropExports', vatID: VatID, krefs: string[] } } RunQueueEventDropExports + * @typedef { { type: 'retireExports', vatID: VatID, krefs: string[] } } RunQueueEventRetireExports + * @typedef { { type: 'retireImports', vatID: VatID, krefs: string[] } } RunQueueEventRetireImports + * @typedef { { type: 'bringOutYourDead', vatID: VatID } } RunQueueEventBringOutYourDead + * @typedef { RunQueueEventNotify | RunQueueEventSend | RunQueueEventCreateVat | + * RunQueueEventUpgradeVat | RunQueueEventStartVat | + * RunQueueEventDropExports | RunQueueEventRetireExports | RunQueueEventRetireImports | + * RunQueueEventBringOutYourDead + * } RunQueueEvent + */ + /** + * + * Dispatch one delivery event. Eventually, this will be called in a + * "delivery crank" for a DeliveryEvent, after the scheduler chooses a + * vat with a non-empty vat-input-queue, and we'll know the target vat + * ahead of time. For now, this is called for each run-queue event, so + * 'send' does not yet know which vat will be involved (if any). + * + * @param { RunQueueEvent } message + * @returns { Promise } + */ + async function deliverRunQueueEvent(message) { // Decref everything in the message, under the assumption that most of // the time we're delivering to a vat or answering the result promise // with an error. If we wind up queueing it on a promise, we'll // re-increment everything there. + // .vatID is present on all RunQueueEvents except 'send', which gets it + // from routeSendEvent. Eventually, every DeliveryEvent will have a + // specific vatID and this will be provided as an argument + let vatID; + if (message.type !== 'send') { + vatID = message.vatID; + } + let useMeter = false; + let deliverP = null; + if (message.type === 'send') { + useMeter = true; const route = routeSendEvent(message); decrementSendEventRefCount(message); if (route) { - const { vatID, targetObject } = route; - policyInput = await deliverToVat(vatID, targetObject, message.msg); + vatID = route.vatID; + deliverP = processSend(vatID, route.targetObject, message.msg); } } else if (message.type === 'notify') { + useMeter = true; decrementNotifyEventRefCount(message); - policyInput = await processNotify(message); + deliverP = processNotify(message); } else if (message.type === 'create-vat') { // creating a new dynamic vat will immediately do start-vat - policyInput = await processCreateVat(message); + deliverP = processCreateVat(message); } else if (message.type === 'startVat') { - policyInput = await processStartVat(message); + deliverP = processStartVat(message); } else if (message.type === 'upgrade-vat') { - policyInput = await processUpgradeVat(message); + deliverP = processUpgradeVat(message); } else if (message.type === 'bringOutYourDead') { - policyInput = await processBringOutYourDead(message); + deliverP = processBringOutYourDead(message); } else if (gcMessages.includes(message.type)) { - policyInput = await processGCMessage(message); + deliverP = processGCMessage(message); } else { assert.fail(X`unable to process message.type ${message.type}`); } - return policyInput; + + let status = await deliverP; + + // status will be set if we made a delivery, else undefined + if (status) { + const decrementReapCount = message.type !== 'bringOutYourDead'; + // the caller needs to be told the vatID that received the delivery, + // but eventually they'll tell us, and 'vatID' should be removed from + // DeliveryStatus + assert(vatID, 'DeliveryStatus.vatID missing'); + status = { vatID, useMeter, decrementReapCount, ...status }; + } + return status; } - let processQueueRunning; async function processDeliveryMessage(message) { kdebug(`processQ ${JSON.stringify(message)}`); kdebug(legibilizeMessage(message)); - if (processQueueRunning) { - console.error(`We're currently already running at`, processQueueRunning); - assert.fail(X`Kernel reentrancy is forbidden`); - } kernelSlog.write({ type: 'crank-start', message }); /** @type { PolicyInput } */ let policyInput = ['none']; - try { - processQueueRunning = Error('here'); - resetDeliveryTriggers(); - - policyInput = await deliverRunQueueEvent(message); - - let didAbort = false; - if (terminationTrigger) { - // the vat is doomed, either voluntarily or from meter/syscall fault - const { vatID, shouldReject, info } = terminationTrigger; - if (terminationTrigger.shouldAbortCrank) { - // errors unwind any changes the vat made - kernelKeeper.abortCrank(); - didAbort = true; - // but metering deductions and underflow notifications must survive - const { meterDeductions, discardFailedDelivery } = postAbortActions; - for (const { meterID, compute } of meterDeductions) { - deductMeter(meterID, compute, false); - // that will re-push any notifications + if (message.type === 'create-vat') { + policyInput = ['create-vat', {}]; + } + + // terminationTrigger can be set by syscall.exit or a vat-fatal syscall + terminationTrigger = null; // reset terminationTrigger before delivery + + // 'deduction' remembers any meter deduction we performed, in case we + // unwind state and have to apply it again + let deduction; + let vatID; + let discardFailedDelivery; + + // The DeliveryStatus tells us what happened to the delivery (success or + // worker error). It will be null if the delivery got cancelled, like a + // 'notify' or 'retireExports' that was superceded somehow. + + const status = await deliverRunQueueEvent(message); + + if (status) { + policyInput = ['crank', {}]; + vatID = status.vatID; + const vatKeeper = kernelKeeper.provideVatKeeper(vatID); + + // deliveries cause garbage, garbage needs collection + const { decrementReapCount } = status; + if (decrementReapCount && vatKeeper.countdownToReap()) { + kernelKeeper.scheduleReap(vatID); + } + + // deliveries cause metering, metering needs deducting + const meterID = vatKeeper.getOptions().meterID; + const { metering, useMeter } = status; + if (metering) { + // if the result has metering, we report it to the runPolicy + const consumed = metering.compute; + assert.typeof(consumed, 'number'); + const computrons = BigInt(consumed); + policyInput = ['crank', { computrons }]; + + // and if both vat and delivery are metered, deduct from the Meter + if (useMeter && meterID) { + deduction = { meterID, computrons }; // in case we must rededuct + const { notify, underflow } = kernelKeeper.deductMeter( + meterID, + computrons, + ); + if (notify) { + notifyMeterThreshold(meterID); } - if (discardFailedDelivery) { - // kernelKeeper.abortCrank removed all evidence that the crank ever - // happened, including, notably, the removal of the delivery itself - // from the head of the run queue, which will result in it being - // delivered again on the next crank. If we don't want that, then - // we need to remove it again. - // eslint-disable-next-line no-use-before-define - getNextDeliveryMessage(); + + // deducting too much causes termination + if (underflow) { + console.log(`meter ${meterID} underflow, terminating vat ${vatID}`); + policyInput = ['crank-failed', {}]; + const err = makeError('meter underflow, vat terminated'); + terminationTrigger = { + vatID, + abortCrank: true, + reject: true, + info: err, + }; } } - // state changes reflecting the termination must also survive, so - // these happen after a possible abortCrank() - terminateVat(vatID, shouldReject, info); - kernelSlog.terminateVat(vatID, shouldReject, info); - kdebug(`vat terminated: ${JSON.stringify(info)}`); } - if (!didAbort) { - // eslint-disable-next-line no-use-before-define - await vatWarehouse.maybeSaveSnapshot(); + + // Deliveries cause syscalls, syscalls might cause errors, errors cause + // termination. Those are reported by the syscall handlers setting + // terminationTrigger. + + // worker errors also terminate the vat + const { terminate } = status; + if (terminate) { + console.log(`delivery problem, terminating vat ${vatID}`, terminate); + policyInput = ['crank-failed', {}]; + const info = makeError(terminate); + terminationTrigger = { vatID, abortCrank: true, reject: true, info }; } - kernelKeeper.processRefcounts(); - kernelKeeper.saveStats(); - const crankNum = kernelKeeper.getCrankNumber(); - kernelKeeper.incrementCrankNumber(); - const { crankhash, activityhash } = kernelKeeper.commitCrank(); - kernelSlog.write({ - type: 'crank-finish', - crankNum, - crankhash, - activityhash, - }); - } finally { - processQueueRunning = undefined; + + // some deliveries should be consumed when they fail + discardFailedDelivery = status.discardFailedDelivery; + } else { + // no status: the delivery got dropped, so no metering or termination + assert(!terminationTrigger, 'hey, no delivery means no termination'); + } + + // terminate upon fatal syscalls, sys.exit requests, and worker problems + let didAbort = false; + if (terminationTrigger) { + assert(vatID, `terminationTrigger but not vatID`); + const ttvid = terminationTrigger.vatID; + assert.equal(ttvid, vatID, `wrong vat got terminated`); + const { abortCrank, reject, info } = terminationTrigger; + if (abortCrank) { + // errors unwind any changes the vat made + kernelKeeper.abortCrank(); + didAbort = true; + // but metering deductions and underflow notifications must survive + if (deduction) { + const { meterID, computrons } = deduction; // re-deduct metering + const { notify } = kernelKeeper.deductMeter(meterID, computrons); + if (notify) { + notifyMeterThreshold(meterID); // re-queue notification + } + } + // some deliveries should be consumed when they fail + if (discardFailedDelivery) { + // kernelKeeper.abortCrank removed all evidence that the crank ever + // happened, including, notably, the removal of the delivery itself + // from the head of the run queue, which will result in it being + // delivered again on the next crank. If we don't want that, then + // we need to remove it again. + + // eslint-disable-next-line no-use-before-define + getNextDeliveryMessage(); + } + // other deliveries should be re-attempted on the next crank, so they + // get the right error: we leave those on the queue + } + + // state changes reflecting the termination must also survive, so these + // happen after a possible abortCrank() + terminateVat(vatID, reject, info); + kernelSlog.terminateVat(vatID, reject, info); + kdebug(`vat terminated: ${JSON.stringify(info)}`); } + + if (!didAbort) { + // eslint-disable-next-line no-use-before-define + await vatWarehouse.maybeSaveSnapshot(); + } + kernelKeeper.processRefcounts(); + kernelKeeper.saveStats(); + const crankNum = kernelKeeper.getCrankNumber(); + kernelKeeper.incrementCrankNumber(); + const { crankhash, activityhash } = kernelKeeper.commitCrank(); + kernelSlog.write({ + type: 'crank-finish', + crankNum, + crankhash, + activityhash, + }); return harden(policyInput); } + let processQueueRunning; + async function tryProcessDeliveryMessage(message) { + if (processQueueRunning) { + console.error(`We're currently already running at`, processQueueRunning); + assert.fail(X`Kernel reentrancy is forbidden`); + } + processQueueRunning = Error('here'); + return processDeliveryMessage(message).finally(() => { + processQueueRunning = undefined; + }); + } + async function processAcceptanceMessage(message) { kdebug(`processAcceptanceQ ${JSON.stringify(message)}`); kdebug(legibilizeMessage(message)); @@ -961,7 +1088,7 @@ export default function buildKernel( // vatManager is somehow confused. console.error(`vatSyscallHandler invoked on dead vat ${vatID}`); const problem = 'vat is dead'; - setTerminationTrigger(vatID, true, true, makeError(problem)); + vatFatalSyscall(vatID, problem); return harden(['error', problem]); } /** @type { KernelSyscallObject | undefined } */ @@ -981,7 +1108,7 @@ export default function buildKernel( kdebug(`vat ${vatID} terminated: error during translation: ${vaterr} ${JSON.stringify(vatSyscallObject)}`); console.log(`error during syscall translation:`, vaterr); const problem = 'syscall translation error: prepare to die'; - setTerminationTrigger(vatID, true, true, makeError(problem)); + vatFatalSyscall(vatID, problem); kres = harden(['error', problem]); vres = harden(['error', problem]); // we leave this catch() with ksc=undefined, so no doKernelSyscall() @@ -1016,7 +1143,7 @@ export default function buildKernel( // the kernel is now in a shutdown state, but it may take a while to // grind to a halt const problem = 'you killed my kernel. prepare to die'; - setTerminationTrigger(vatID, true, true, makeError(problem)); + vatFatalSyscall(vatID, problem); vres = harden(['error', problem]); } } @@ -1241,7 +1368,7 @@ export default function buildKernel( } else { message = getNextDeliveryMessage(); if (message) { - resultPromise = processDeliveryMessage(message); + resultPromise = tryProcessDeliveryMessage(message); } } return { resultPromise }; diff --git a/packages/SwingSet/src/kernel/kernelSyscall.js b/packages/SwingSet/src/kernel/kernelSyscall.js index 3cbe5aed659..2aa1383072d 100644 --- a/packages/SwingSet/src/kernel/kernelSyscall.js +++ b/packages/SwingSet/src/kernel/kernelSyscall.js @@ -15,7 +15,7 @@ export function makeKernelSyscallHandler(tools) { doSend, doSubscribe, doResolve, - setTerminationTrigger, + requestTermination, } = tools; const { kvStore } = kernelKeeper; @@ -30,7 +30,7 @@ export function makeKernelSyscallHandler(tools) { function exit(vatID, isFailure, info) { kernelKeeper.incStat('syscalls'); kernelKeeper.incStat('syscallExit'); - setTerminationTrigger(vatID, false, !!isFailure, info); + requestTermination(vatID, !!isFailure, info); return OKNULL; } diff --git a/packages/SwingSet/src/kernel/state/kernelKeeper.js b/packages/SwingSet/src/kernel/state/kernelKeeper.js index 642fd3aceea..d14f43e9688 100644 --- a/packages/SwingSet/src/kernel/state/kernelKeeper.js +++ b/packages/SwingSet/src/kernel/state/kernelKeeper.js @@ -307,12 +307,39 @@ export default function makeKernelKeeper( kvStore.set('kernel.defaultReapInterval', `${defaultReapInterval}`); } + /** + * + * @param {string} mt + * @returns { asserts mt is ManagerType } + */ + function insistManagerType(mt) { + assert( + [ + 'local', + 'nodeWorker', + 'node-subprocess', + 'xs-worker', + 'xs-worker-no-gc', + ].includes(mt), + ); + return undefined; // hush JSDoc + } + function getDefaultManagerType() { - return getRequired('kernel.defaultManagerType'); + const mt = getRequired('kernel.defaultManagerType'); + insistManagerType(mt); + return mt; } + /** + * + * @returns { number | 'never' } + */ function getDefaultReapInterval() { - return getRequired('kernel.defaultReapInterval'); + const r = getRequired('kernel.defaultReapInterval'); + const ri = r === 'never' ? r : Number.parseInt(r, 10); + assert(ri === 'never' || typeof ri === 'number', `k.dri is '${ri}'`); + return ri; } const bundleIDRE = new RegExp('^b1-[0-9a-f]{128}$'); @@ -552,7 +579,7 @@ export default function makeKernelKeeper( function getKernelPromise(kernelSlot) { insistKernelType('promise', kernelSlot); - const p = { state: kvStore.get(`${kernelSlot}.state`) }; + const p = { state: getRequired(`${kernelSlot}.state`) }; switch (p.state) { case undefined: assert.fail(X`unknown kernelPromise '${kernelSlot}'`); diff --git a/packages/SwingSet/src/kernel/state/vatKeeper.js b/packages/SwingSet/src/kernel/state/vatKeeper.js index d7d59b331f1..e6a115c2c51 100644 --- a/packages/SwingSet/src/kernel/state/vatKeeper.js +++ b/packages/SwingSet/src/kernel/state/vatKeeper.js @@ -13,6 +13,8 @@ import { buildReachableAndVatSlot, } from './reachable.js'; +/** @typedef {import('../../types-internal.js').RecordedVatOptions} RecordedVatOptions */ + // makeVatKeeper is a pure function: all state is kept in the argument object // TODO: tests rely on these numbers and haven't been updated to use names. @@ -93,7 +95,7 @@ export function makeVatKeeper( /** * @param {SourceOfBundle} source - * @param {ManagerOptions} options + * @param {RecordedVatOptions} options */ function setSourceAndOptions(source, options) { // take care with API change diff --git a/packages/SwingSet/src/types-exported.js b/packages/SwingSet/src/types-exported.js index 4db3e22de9b..a6c49f0df52 100644 --- a/packages/SwingSet/src/types-exported.js +++ b/packages/SwingSet/src/types-exported.js @@ -11,4 +11,27 @@ export {}; * @typedef { string } BundleID * @typedef {*} BundleCap * @typedef { { moduleFormat: 'endoZipBase64', endoZipBase64: string } } EndoZipBase64Bundle + * + * @typedef { unknown } Meter + * + * E(vatAdminService).createVat(bundle, options: DynamicVatOptions) + * + * @typedef { { description?: string, + * meter?: Meter, + * managerType?: ManagerType, + * vatParameters?: {*}, + * enableSetup?: boolean, + * enablePipelining?: boolean + * enableVatstore?: boolean, + * virtualObjectCacheSize?: number, + * useTranscript?: boolean, + * reapInterval? : number | 'never', + * }} DynamicVatOptionsWithoutMeter + * @typedef { { meter?: Meter } } HasMeter + * @typedef { DynamicVatOptionsWithoutMeter & HasMeter } DynamicVatOptions + * + * config.vats[name].creationOptions: StaticVatOptions + * + * @typedef { { enableDisavow?: boolean } } HasEnableDisavow + * @typedef { DynamicVatOptions & HasEnableDisavow } StaticVatOptions */ diff --git a/packages/SwingSet/src/types-internal.js b/packages/SwingSet/src/types-internal.js new file mode 100644 index 00000000000..d076ec69079 --- /dev/null +++ b/packages/SwingSet/src/types-internal.js @@ -0,0 +1,17 @@ +import './types.js'; +import './types-exported.js'; + +export {}; + +/** + * @typedef { string } VatID + * @typedef { string } MeterID + * @typedef { { meterID?: MeterID } } HasMeterID + * @typedef { import('./types-exported.js').DynamicVatOptionsWithoutMeter } DynamicVatOptionsWithoutMeter + * + * // used by vatKeeper.setSourceAndOptions(source, RecordedVatOptions) + * + * @typedef { DynamicVatOptionsWithoutMeter & HasMeterID } InternalDynamicVatOptions + * @typedef { StaticVatOptions | { InternalDynamicVatOptions & HasMeterID } } RecordedVatOptions + * + */ From f62335aff97183554af43900cdc1a54a7ab9a53c Mon Sep 17 00:00:00 2001 From: Mathieu Hofman Date: Thu, 3 Mar 2022 19:38:35 +0000 Subject: [PATCH 2/2] refactor(SwingSet): Define SwingSet types as module exported Add "re-exporting" ambient file for backwards compatibility --- .../src/controller/initializeSwingset.js | 2 +- .../src/devices/vat-admin/device-vat-admin.js | 2 +- packages/SwingSet/src/index.js | 2 +- packages/SwingSet/src/kernel/deviceManager.js | 2 +- .../src/kernel/vat-loader/manager-helper.js | 2 +- .../vat-loader/manager-subprocess-xsnap.js | 2 +- .../nodeworker/supervisor-nodeworker.js | 2 +- .../supervisor-subprocess-xsnap.js | 2 +- .../src/supervisors/supervisor-helper.js | 2 +- packages/SwingSet/src/types-ambient.js | 167 ++++++++++++++++++ packages/SwingSet/src/types-exported.js | 37 ---- .../src/{types.js => types-external.js} | 40 ++++- packages/SwingSet/src/types-internal.js | 7 +- packages/zoe/src/contractFacet/zcfZygote.js | 2 +- 14 files changed, 215 insertions(+), 56 deletions(-) create mode 100644 packages/SwingSet/src/types-ambient.js delete mode 100644 packages/SwingSet/src/types-exported.js rename packages/SwingSet/src/{types.js => types-external.js} (92%) diff --git a/packages/SwingSet/src/controller/initializeSwingset.js b/packages/SwingSet/src/controller/initializeSwingset.js index da74d999a33..6e97b34f2c6 100644 --- a/packages/SwingSet/src/controller/initializeSwingset.js +++ b/packages/SwingSet/src/controller/initializeSwingset.js @@ -7,7 +7,7 @@ import { resolve as resolveModuleSpecifier } from 'import-meta-resolve'; import { assert, details as X } from '@agoric/assert'; import bundleSource from '@endo/bundle-source'; -import '../types.js'; +import '../types-ambient.js'; import { insistStorageAPI } from '../lib/storageAPI.js'; import { initializeKernel } from './initializeKernel.js'; import { kdebugEnable } from '../lib/kdebug.js'; diff --git a/packages/SwingSet/src/devices/vat-admin/device-vat-admin.js b/packages/SwingSet/src/devices/vat-admin/device-vat-admin.js index f2d1459202d..86a6fd22f5e 100644 --- a/packages/SwingSet/src/devices/vat-admin/device-vat-admin.js +++ b/packages/SwingSet/src/devices/vat-admin/device-vat-admin.js @@ -71,7 +71,7 @@ bundleID before submitting to the kernel), or (temporarily) a full bundle. /** * if you're into types, this might loosely describe devices.vatAdmin * - * @typedef { import('../../types-exported.js').BundleID } BundleID + * @typedef { import('../../types-external.js').BundleID } BundleID * * @typedef { string } MeterID * @typedef { string } VatID diff --git a/packages/SwingSet/src/index.js b/packages/SwingSet/src/index.js index 2406cc3c5c4..e2aa4b88615 100644 --- a/packages/SwingSet/src/index.js +++ b/packages/SwingSet/src/index.js @@ -20,4 +20,4 @@ export { default as buildCommand } from './devices/command/command.js'; export { buildPlugin } from './devices/plugin/plugin.js'; // eslint-disable-next-line import/export -export * from './types-exported.js'; +export * from './types-external.js'; diff --git a/packages/SwingSet/src/kernel/deviceManager.js b/packages/SwingSet/src/kernel/deviceManager.js index 59643069869..18196778891 100644 --- a/packages/SwingSet/src/kernel/deviceManager.js +++ b/packages/SwingSet/src/kernel/deviceManager.js @@ -3,7 +3,7 @@ import { assert } from '@agoric/assert'; import { makeDeviceSlots } from './deviceSlots.js'; import { insistCapData } from '../lib/capdata.js'; -import '../types.js'; +import '../types-ambient.js'; /* The DeviceManager is much simpler than the VatManager, because the feature * set is smaller: diff --git a/packages/SwingSet/src/kernel/vat-loader/manager-helper.js b/packages/SwingSet/src/kernel/vat-loader/manager-helper.js index c4c0dc7e0a8..a6379287857 100644 --- a/packages/SwingSet/src/kernel/vat-loader/manager-helper.js +++ b/packages/SwingSet/src/kernel/vat-loader/manager-helper.js @@ -1,6 +1,6 @@ // @ts-check import { assert } from '@agoric/assert'; -import '../../types.js'; +import '../../types-ambient.js'; import { insistVatDeliveryResult } from '../../lib/message.js'; import { makeTranscriptManager } from './transcript.js'; diff --git a/packages/SwingSet/src/kernel/vat-loader/manager-subprocess-xsnap.js b/packages/SwingSet/src/kernel/vat-loader/manager-subprocess-xsnap.js index 5ca574dad5f..a9847c23fe2 100644 --- a/packages/SwingSet/src/kernel/vat-loader/manager-subprocess-xsnap.js +++ b/packages/SwingSet/src/kernel/vat-loader/manager-subprocess-xsnap.js @@ -7,7 +7,7 @@ import { insistVatSyscallObject, insistVatDeliveryResult, } from '../../lib/message.js'; -import '../../types.js'; +import '../../types-ambient.js'; import './types.js'; // eslint-disable-next-line no-unused-vars diff --git a/packages/SwingSet/src/supervisors/nodeworker/supervisor-nodeworker.js b/packages/SwingSet/src/supervisors/nodeworker/supervisor-nodeworker.js index 95df6265289..21491556620 100644 --- a/packages/SwingSet/src/supervisors/nodeworker/supervisor-nodeworker.js +++ b/packages/SwingSet/src/supervisors/nodeworker/supervisor-nodeworker.js @@ -6,7 +6,7 @@ import '@endo/init'; import { parentPort } from 'worker_threads'; import anylogger from 'anylogger'; -import '../../types.js'; +import '../../types-ambient.js'; import { assert, details as X } from '@agoric/assert'; import { importBundle } from '@endo/import-bundle'; import { makeMarshal } from '@endo/marshal'; diff --git a/packages/SwingSet/src/supervisors/subprocess-xsnap/supervisor-subprocess-xsnap.js b/packages/SwingSet/src/supervisors/subprocess-xsnap/supervisor-subprocess-xsnap.js index 93ddace7443..7ab5a9b95a9 100644 --- a/packages/SwingSet/src/supervisors/subprocess-xsnap/supervisor-subprocess-xsnap.js +++ b/packages/SwingSet/src/supervisors/subprocess-xsnap/supervisor-subprocess-xsnap.js @@ -3,7 +3,7 @@ import { assert, details as X } from '@agoric/assert'; import { importBundle } from '@endo/import-bundle'; import { makeMarshal } from '@endo/marshal'; -import '../../types.js'; +import '../../types-ambient.js'; // grumble... waitUntilQuiescent is exported and closes over ambient authority import { waitUntilQuiescent } from '../../lib-nodejs/waitUntilQuiescent.js'; import { makeGcAndFinalize } from '../../lib-nodejs/gc-and-finalize.js'; diff --git a/packages/SwingSet/src/supervisors/supervisor-helper.js b/packages/SwingSet/src/supervisors/supervisor-helper.js index 49d347e4dfe..99dcfd2a8c0 100644 --- a/packages/SwingSet/src/supervisors/supervisor-helper.js +++ b/packages/SwingSet/src/supervisors/supervisor-helper.js @@ -4,7 +4,7 @@ import { insistVatSyscallObject, insistVatSyscallResult, } from '../lib/message.js'; -import '../types.js'; +import '../types-ambient.js'; /** * @typedef { (delivery: VatDeliveryObject) => (VatDeliveryResult | Promise) } VatDispatcherSyncAsync diff --git a/packages/SwingSet/src/types-ambient.js b/packages/SwingSet/src/types-ambient.js new file mode 100644 index 00000000000..16419008cfe --- /dev/null +++ b/packages/SwingSet/src/types-ambient.js @@ -0,0 +1,167 @@ +// @ts-check + +/** + * @typedef { import('./types-external.js').BundleFormat } BundleFormat + */ + +/** + * @typedef { import('@endo/marshal').CapData } SwingSetCapData + */ + +/** @typedef { import('./types-external.js').BundleID } BundleID */ +/** @typedef { import('./types-external.js').BundleCap } BundleCap */ +/** @typedef { import('./types-external.js').EndoZipBase64Bundle } EndoZipBase64Bundle */ + +/** + * @typedef { import('./types-external.js').GetExportBundle } GetExportBundle + * @typedef { import('./types-external.js').NestedEvaluateBundle } NestedEvaluateBundle + * @typedef { import('./types-external.js').Bundle } Bundle + * + * @typedef { import('./types-external.js').HasBundle } HasBundle + * @typedef { import('./types-external.js').HasSetup } HasSetup + * @typedef { import('./types-external.js').ManagerType } ManagerType + * @typedef { import('./types-external.js').ManagerOptions } ManagerOptions + */ + +/** + * See ../docs/static-vats.md#vatpowers + * + * @typedef { import('./types-external.js').VatPowers } VatPowers + * @typedef { import('./types-external.js').StaticVatPowers } StaticVatPowers + * @typedef { import('./types-external.js').MarshallingVatPowers } MarshallingVatPowers + * @typedef { import('./types-external.js').MeteringVatPowers } MeteringVatPowers + * + * @typedef { import('./types-external.js').TerminationVatPowers } TerminationVatPowers + */ + +/** + * @typedef { import('./types-external.js').Message } Message + * + * @typedef { import('./types-external.js').ResolutionPolicy } ResolutionPolicy + * + * @typedef { import('./types-external.js').VatDeliveryMessage } VatDeliveryMessage + * @typedef { import('./types-external.js').VatOneResolution } VatOneResolution + * @typedef { import('./types-external.js').VatDeliveryNotify } VatDeliveryNotify + * @typedef { import('./types-external.js').VatDeliveryDropExports } VatDeliveryDropExports + * @typedef { import('./types-external.js').VatDeliveryRetireExports } VatDeliveryRetireExports + * @typedef { import('./types-external.js').VatDeliveryRetireImports } VatDeliveryRetireImports + * @typedef { import('./types-external.js').VatDeliveryStartVat } VatDeliveryStartVat + * @typedef { import('./types-external.js').VatDeliveryBringOutYourDead } VatDeliveryBringOutYourDead + * @typedef { import('./types-external.js').VatDeliveryObject } VatDeliveryObject + * @typedef { import('./types-external.js').VatDeliveryResult } VatDeliveryResult + * + * @typedef { import('./types-external.js').VatSyscallSend } VatSyscallSend + * @typedef { import('./types-external.js').VatSyscallCallNow } VatSyscallCallNow + * @typedef { import('./types-external.js').VatSyscallSubscribe } VatSyscallSubscribe + * @typedef { import('./types-external.js').VatSyscallResolve } VatSyscallResolve + * @typedef { import('./types-external.js').VatSyscallExit } VatSyscallExit + * @typedef { import('./types-external.js').VatSyscallVatstoreGet } VatSyscallVatstoreGet + * @typedef { import('./types-external.js').VatSyscallVatstoreGetAfter } VatSyscallVatstoreGetAfter + * @typedef { import('./types-external.js').VatSyscallVatstoreSet } VatSyscallVatstoreSet + * @typedef { import('./types-external.js').VatSyscallVatstoreDelete } VatSyscallVatstoreDelete + * @typedef { import('./types-external.js').VatSyscallDropImports } VatSyscallDropImports + * @typedef { import('./types-external.js').VatSyscallRetireImports } VatSyscallRetireImports + * @typedef { import('./types-external.js').VatSyscallRetireExports } VatSyscallRetireExports + * + * @typedef { import('./types-external.js').VatSyscallObject } VatSyscallObject + * + * @typedef { import('./types-external.js').VatSyscallResultOk } VatSyscallResultOk + * @typedef { import('./types-external.js').VatSyscallResultError } VatSyscallResultError + * @typedef { import('./types-external.js').VatSyscallResult } VatSyscallResult + * @typedef { import('./types-external.js').VatSyscaller } VatSyscaller + * + * @typedef { import('./types-external.js').KernelDeliveryMessage } KernelDeliveryMessage + * @typedef { import('./types-external.js').KernelDeliveryOneNotify } KernelDeliveryOneNotify + * @typedef { import('./types-external.js').KernelDeliveryNotify } KernelDeliveryNotify + * @typedef { import('./types-external.js').KernelDeliveryDropExports } KernelDeliveryDropExports + * @typedef { import('./types-external.js').KernelDeliveryRetireExports } KernelDeliveryRetireExports + * @typedef { import('./types-external.js').KernelDeliveryRetireImports } KernelDeliveryRetireImports + * @typedef { import('./types-external.js').KernelDeliveryStartVat } KernelDeliveryStartVat + * @typedef { import('./types-external.js').KernelDeliveryBringOutYourDead } KernelDeliveryBringOutYourDead + * @typedef { import('./types-external.js').KernelDeliveryObject } KernelDeliveryObject + * @typedef { import('./types-external.js').KernelSyscallSend } KernelSyscallSend + * @typedef { import('./types-external.js').KernelSyscallInvoke } KernelSyscallInvoke + * @typedef { import('./types-external.js').KernelSyscallSubscribe } KernelSyscallSubscribe + * @typedef { import('./types-external.js').KernelOneResolution } KernelOneResolution + * @typedef { import('./types-external.js').KernelSyscallResolve } KernelSyscallResolve + * @typedef { import('./types-external.js').KernelSyscallExit } KernelSyscallExit + * @typedef { import('./types-external.js').KernelSyscallVatstoreGet } KernelSyscallVatstoreGet + * @typedef { import('./types-external.js').KernelSyscallVatstoreGetAfter } KernelSyscallVatstoreGetAfter + * @typedef { import('./types-external.js').KernelSyscallVatstoreSet } KernelSyscallVatstoreSet + * @typedef { import('./types-external.js').KernelSyscallVatstoreDelete } KernelSyscallVatstoreDelete + * @typedef { import('./types-external.js').KernelSyscallDropImports } KernelSyscallDropImports + * @typedef { import('./types-external.js').KernelSyscallRetireImports } KernelSyscallRetireImports + * @typedef { import('./types-external.js').KernelSyscallRetireExports } KernelSyscallRetireExports + * + * @typedef { import('./types-external.js').KernelSyscallObject } KernelSyscallObject + * @typedef { import('./types-external.js').KernelSyscallResultOk } KernelSyscallResultOk + * @typedef { import('./types-external.js').KernelSyscallResultError } KernelSyscallResultError + * @typedef { import('./types-external.js').KernelSyscallResult } KernelSyscallResult + * + * @typedef { import('./types-external.js').DeviceInvocation } DeviceInvocation + * @typedef { import('./types-external.js').DeviceInvocationResultOk } DeviceInvocationResultOk + * @typedef { import('./types-external.js').DeviceInvocationResultError } DeviceInvocationResultError + * @typedef { import('./types-external.js').DeviceInvocationResult } DeviceInvocationResult + * + * @typedef { import('./types-external.js').TranscriptEntry } TranscriptEntry + * @typedef { import('./types-external.js').VatStats } VatStats + * @typedef { import('./types-external.js').VatKeeper } VatKeeper + * @typedef { import('./types-external.js').KernelKeeper } KernelKeeper + * @typedef { import('./types-external.js').XSnap } XSnap + * @typedef { import('./types-external.js').SlogFinishDelivery } SlogFinishDelivery + * @typedef { import('./types-external.js').SlogFinishSyscall } SlogFinishSyscall + * @typedef { import('./types-external.js').KernelSlog } KernelSlog + * @typedef { import('./types-external.js').VatSlog } VatSlog + * + * @typedef { import('./types-external.js').VatManagerFactory } VatManagerFactory + * @typedef { import('./types-external.js').VatManager } VatManager + * @typedef { import('./types-external.js').SnapStore } SnapStore + * @typedef { import('./types-external.js').WaitUntilQuiescent } WaitUntilQuiescent + */ + +/** + * @typedef { import('./types-external.js').SourceSpec } SourceSpec + * @typedef { import('./types-external.js').BundleSpec } BundleSpec + * @typedef { import('./types-external.js').BundleRef } BundleRef + * @typedef { import('./types-external.js').SwingSetConfigProperties } SwingSetConfigProperties + */ + +/** + * @typedef { import('./types-external.js').SwingSetConfigDescriptor } SwingSetConfigDescriptor + */ + +/** + * @typedef { import('./types-external.js').SwingSetConfig } SwingSetConfig + */ + +/** + * @typedef { import('./types-external.js').SwingSetKernelConfig } SwingSetKernelConfig + */ + +/** + * @typedef { import('./types-external.js').SourceOfBundle } SourceOfBundle + */ +/** + * @typedef { import('@agoric/swing-store').KVStore } KVStore + * @typedef { import('@agoric/swing-store').StreamStore } StreamStore + * @typedef { import('@agoric/swing-store').StreamPosition } StreamPosition + * @typedef { import('@agoric/swing-store').SwingStore } SwingStore + * + * @typedef { import('./types-external.js').HostStore } HostStore + * + * @typedef { import('./types-external.js').KVStorePlus } KVStorePlus + */ + +/** + * @typedef { import('./types-external.js').PolicyInputNone } PolicyInputNone + * @typedef { import('./types-external.js').PolicyInputCreateVat } PolicyInputCreateVat + * @typedef { import('./types-external.js').PolicyInputCrankComplete } PolicyInputCrankComplete + * @typedef { import('./types-external.js').PolicyInputCrankFailed } PolicyInputCrankFailed + * @typedef { import('./types-external.js').PolicyInput } PolicyInput + * @typedef { import('./types-external.js').PolicyOutput } PolicyOutput + * @typedef { import('./types-external.js').RunPolicy } RunPolicy + */ + +/** + * @typedef { import('./types-external.js').MeterControl } MeterControl + */ diff --git a/packages/SwingSet/src/types-exported.js b/packages/SwingSet/src/types-exported.js deleted file mode 100644 index a6c49f0df52..00000000000 --- a/packages/SwingSet/src/types-exported.js +++ /dev/null @@ -1,37 +0,0 @@ -import './types.js'; - -export {}; - -/* This file defines types that part of the external API of swingset. That - * includes standard services which user-provided vat code might interact - * with, like VatAdminService. */ - -/** - * - * @typedef { string } BundleID - * @typedef {*} BundleCap - * @typedef { { moduleFormat: 'endoZipBase64', endoZipBase64: string } } EndoZipBase64Bundle - * - * @typedef { unknown } Meter - * - * E(vatAdminService).createVat(bundle, options: DynamicVatOptions) - * - * @typedef { { description?: string, - * meter?: Meter, - * managerType?: ManagerType, - * vatParameters?: {*}, - * enableSetup?: boolean, - * enablePipelining?: boolean - * enableVatstore?: boolean, - * virtualObjectCacheSize?: number, - * useTranscript?: boolean, - * reapInterval? : number | 'never', - * }} DynamicVatOptionsWithoutMeter - * @typedef { { meter?: Meter } } HasMeter - * @typedef { DynamicVatOptionsWithoutMeter & HasMeter } DynamicVatOptions - * - * config.vats[name].creationOptions: StaticVatOptions - * - * @typedef { { enableDisavow?: boolean } } HasEnableDisavow - * @typedef { DynamicVatOptions & HasEnableDisavow } StaticVatOptions - */ diff --git a/packages/SwingSet/src/types.js b/packages/SwingSet/src/types-external.js similarity index 92% rename from packages/SwingSet/src/types.js rename to packages/SwingSet/src/types-external.js index 208fd8986ee..8c6cd0417aa 100644 --- a/packages/SwingSet/src/types.js +++ b/packages/SwingSet/src/types-external.js @@ -1,6 +1,10 @@ // @ts-check -// import '@endo/marshal/src/types.js'; +export {}; + +/* This file defines types that part of the external API of swingset. That + * includes standard services which user-provided vat code might interact + * with, like VatAdminService. */ /** * @typedef {'getExport' | 'nestedEvaluate' | 'endoZipBase64'} BundleFormat @@ -10,10 +14,6 @@ * @typedef {import('@endo/marshal').CapData} SwingSetCapData */ -/** @typedef { import('./types-exported.js').BundleID } BundleID */ -/** @typedef { import('./types-exported.js').BundleCap } BundleCap */ -/** @typedef { import('./types-exported.js').EndoZipBase64Bundle } EndoZipBase64Bundle */ - /** * @typedef { { moduleFormat: 'getExport', source: string, sourceMap: string? } } GetExportBundle * @typedef { { moduleFormat: 'nestedEvaluate', source: string, sourceMap: string? } } NestedEvaluateBundle @@ -317,3 +317,33 @@ * @property {*} unmetered Wrap a callback with runWithoutMetering * */ + +/** + * + * @typedef { string } BundleID + * @typedef {*} BundleCap + * @typedef { { moduleFormat: 'endoZipBase64', endoZipBase64: string } } EndoZipBase64Bundle + * + * @typedef { unknown } Meter + * + * E(vatAdminService).createVat(bundle, options: DynamicVatOptions) + * + * @typedef { { description?: string, + * meter?: Meter, + * managerType?: ManagerType, + * vatParameters?: *, + * enableSetup?: boolean, + * enablePipelining?: boolean + * enableVatstore?: boolean, + * virtualObjectCacheSize?: number, + * useTranscript?: boolean, + * reapInterval? : number | 'never', + * }} DynamicVatOptionsWithoutMeter + * @typedef { { meter?: Meter } } HasMeter + * @typedef { DynamicVatOptionsWithoutMeter & HasMeter } DynamicVatOptions + * + * config.vats[name].creationOptions: StaticVatOptions + * + * @typedef { { enableDisavow?: boolean } } HasEnableDisavow + * @typedef { DynamicVatOptions & HasEnableDisavow } StaticVatOptions + */ diff --git a/packages/SwingSet/src/types-internal.js b/packages/SwingSet/src/types-internal.js index d076ec69079..c4f55bd0860 100644 --- a/packages/SwingSet/src/types-internal.js +++ b/packages/SwingSet/src/types-internal.js @@ -1,5 +1,4 @@ -import './types.js'; -import './types-exported.js'; +// @ts-check export {}; @@ -7,11 +6,11 @@ export {}; * @typedef { string } VatID * @typedef { string } MeterID * @typedef { { meterID?: MeterID } } HasMeterID - * @typedef { import('./types-exported.js').DynamicVatOptionsWithoutMeter } DynamicVatOptionsWithoutMeter + * @typedef { import('./types-external.js').DynamicVatOptionsWithoutMeter } DynamicVatOptionsWithoutMeter * * // used by vatKeeper.setSourceAndOptions(source, RecordedVatOptions) * * @typedef { DynamicVatOptionsWithoutMeter & HasMeterID } InternalDynamicVatOptions - * @typedef { StaticVatOptions | { InternalDynamicVatOptions & HasMeterID } } RecordedVatOptions + * @typedef { import('./types-external.js').StaticVatOptions | ( InternalDynamicVatOptions & HasMeterID ) } RecordedVatOptions * */ diff --git a/packages/zoe/src/contractFacet/zcfZygote.js b/packages/zoe/src/contractFacet/zcfZygote.js index 32540606142..e305e6b987c 100644 --- a/packages/zoe/src/contractFacet/zcfZygote.js +++ b/packages/zoe/src/contractFacet/zcfZygote.js @@ -22,7 +22,7 @@ import { addToAllocation, subtractFromAllocation } from './allocationMath.js'; import '../../exported.js'; import '../internal-types.js'; -import '@agoric/swingset-vat/src/types.js'; +import '@agoric/swingset-vat/src/types-ambient.js'; /** @type {MakeZCFZygote} */ export const makeZCFZygote = (