(core) add GVISOR_LIMIT_MEMORY to cap memory available in sandbox

Summary:
This allows limiting the memory available to documents in the sandbox when gvisor is used. If memory limit is exceeded, we offer to open doc in recovery mode. Recovery mode is tweaked to open docs with tables in "ondemand" mode, which will generally take less memory and allow for deleting rows.

The limit is on the size of the virtual address space available to the sandbox (`RLIMIT_AS`), which in practice appears to function as one would want, and is the only practical option. There is a documented `RLIMIT_RSS` limit to `specifies the limit (in bytes) of the process's resident set (the number of virtual pages resident in RAM)` but this is no longer enforced by the kernel (neither the host nor gvisor).

When the sandbox runs out of memory, there are many ways it can fail. This diff catches all the ones I saw, but there could be more.

Test Plan: added tests

Reviewers: alexmojaki

Reviewed By: alexmojaki

Subscribers: alexmojaki

Differential Revision: https://phab.getgrist.com/D3398
pull/171/head
Paul Fitzpatrick 2 years ago
parent 2fd8a34ff8
commit cf23a2d1ee

@ -110,7 +110,7 @@ import {Events as BackboneEvents} from 'backbone';
* @property {Number} data - An array of unread invites (see app/common/sharing).
*/
const ValidEvent = StringUnion('docListAction', 'docUserAction', 'docShutdown',
const ValidEvent = StringUnion('docListAction', 'docUserAction', 'docShutdown', 'docError',
'clientConnect', 'clientLogout',
'profileFetch', 'userSettings', 'receiveInvites');
type ValidEvent = typeof ValidEvent.type;
@ -213,8 +213,10 @@ export class Comm extends dispose.Disposable implements GristServerAPI, DocListA
private _connections: Map<string|null, GristWSConnection> = new Map();
private _collectedUserActions: UserAction[] | null;
private _singleWorkerMode: boolean = getInitialDocAssignment() === null; // is this classic Grist?
private _reportError?: (err: Error) => void; // optional callback for errors
public create() {
public create(reportError?: (err: Error) => void) {
this._reportError = reportError;
this.autoDisposeCallback(() => {
for (const connection of this._connections.values()) { connection.dispose(); }
this._connections.clear();
@ -469,6 +471,7 @@ export class Comm extends dispose.Disposable implements GristServerAPI, DocListA
err.shouldFork = message.shouldFork;
console.log(`Comm response #${reqId} ${r.methodName} ERROR:${code} ${message.error}`
+ (message.shouldFork ? ` (should fork)` : ''));
this._reportError?.(err);
r.reject(err);
} else {
console.log(`Comm response #${reqId} ${r.methodName} OK`);

@ -72,6 +72,10 @@ export interface DocPageModel {
updateCurrentDoc(urlId: string, openMode: OpenDocMode): Promise<Document>;
refreshCurrentDoc(doc: DocInfo): Promise<Document>;
updateCurrentDocUsage(docUsage: FilteredDocUsageSummary): void;
// Offer to open document in recovery mode, if user is owner, and report
// the error that prompted the offer. If user is not owner, just flag that
// document needs attention of an owner.
offerRecovery(err: Error): void;
}
export interface ImportSource {
@ -204,26 +208,21 @@ export class DocPageModelImpl extends Disposable implements DocPageModel {
return urlState().pushUrl(nextState, {avoidReload: true, ...options});
}
private _onOpenError(err: Error) {
if (err instanceof CancelledError) {
// This means that we started loading a new doc before the previous one finished loading.
console.log("DocPageModel _openDoc cancelled");
return;
}
// Expected errors (e.g. Access Denied) produce a separate error page. For unexpected errors,
// show a modal, and include a toast for the sake of the "Report error" link.
reportError(err);
public offerRecovery(err: Error) {
const isDenied = (err as any).code === 'ACL_DENY';
const isOwner = this.currentDoc.get()?.access === 'owners';
confirmModal(
"Error opening document",
"Error accessing document",
"Reload",
async () => window.location.reload(true),
isOwner ? `You can try reloading the document, or using recovery mode. ` +
`Recovery mode opens the document to be fully accessible to owners, and ` +
`inaccessible to others. ` +
`[${err.message}]` : err.message,
`inaccessible to others. It also disables formulas. ` +
`[${err.message}]` :
isDenied ? `Sorry, access to this document has been denied. [${err.message}]` :
`Document owners can attempt to recover the document. [${err.message}]`,
{ hideCancel: true,
extraButtons: isOwner ? bigBasicButton('Enter recovery mode', dom.on('click', async () => {
extraButtons: (isOwner && !isDenied) ? bigBasicButton('Enter recovery mode', dom.on('click', async () => {
await this._api.getDocAPI(this.currentDocId.get()!).recover(true);
window.location.reload(true);
}), testId('modal-recovery-mode')) : null,
@ -231,6 +230,18 @@ export class DocPageModelImpl extends Disposable implements DocPageModel {
);
}
private _onOpenError(err: Error) {
if (err instanceof CancelledError) {
// This means that we started loading a new doc before the previous one finished loading.
console.log("DocPageModel _openDoc cancelled");
return;
}
// Expected errors (e.g. Access Denied) produce a separate error page. For unexpected errors,
// show a modal, and include a toast for the sake of the "Report error" link.
reportError(err);
this.offerRecovery(err);
}
private async _openDoc(flow: AsyncFlow, urlId: string, urlOpenMode: OpenDocMode | undefined,
comparisonUrlId: string | undefined,
linkParameters: Record<string, string> | undefined): Promise<void> {

@ -9,6 +9,7 @@ import {isDesktop} from 'app/client/lib/browserInfo';
import {FocusLayer} from 'app/client/lib/FocusLayer';
import * as koUtil from 'app/client/lib/koUtil';
import {reportError, TopAppModel, TopAppModelImpl} from 'app/client/models/AppModel';
import {DocPageModel} from 'app/client/models/DocPageModel';
import {setUpErrorHandling} from 'app/client/models/errors';
import {createAppUI} from 'app/client/ui/AppUI';
import {addViewportTag} from 'app/client/ui/viewport';
@ -31,7 +32,7 @@ export class App extends DisposableWithEvents {
// Used by #newui code to avoid a dependency on commands.js, and by tests to issue commands.
public allCommands = commands.allCommands;
public comm = this.autoDispose(Comm.create());
public comm = this.autoDispose(Comm.create(this._checkError.bind(this)));
public clientScope: ClientScope;
public features: ko.Computed<ISupportedFeatures>;
public topAppModel: TopAppModel; // Exposed because used by test/nbrowser/gristUtils.
@ -42,6 +43,9 @@ export class App extends DisposableWithEvents {
// we can choose to refresh the client also.
private _serverVersion: string|null = null;
// Track the most recently created DocPageModel, for some error handling.
private _mostRecentDocPageModel?: DocPageModel;
constructor() {
super();
@ -154,6 +158,10 @@ export class App extends DisposableWithEvents {
setTimeout(() => this.reloadPane(), 0);
});
this.listenTo(this.comm, 'docError', (msg) => {
this._checkError(new Error(msg.data.message));
});
// When the document is unloaded, dispose the app, allowing it to do any needed
// cleanup (e.g. Document on disposal triggers closeDoc message to the server). It needs to be
// in 'beforeunload' rather than 'unload', since websocket is closed by the time of 'unload'.
@ -202,6 +210,10 @@ export class App extends DisposableWithEvents {
return true;
}
public setDocPageModel(pageModel: DocPageModel) {
this._mostRecentDocPageModel = pageModel;
}
// Get the user profile for testing purposes
public async testGetProfile(): Promise<any> {
const resp = await fetchFromHome('/api/profile/user', {credentials: 'include'});
@ -211,4 +223,16 @@ export class App extends DisposableWithEvents {
public testNumPendingApiRequests(): number {
return BaseAPI.numPendingRequests();
}
private _checkError(err: Error) {
const message = String(err);
// Take special action on any error that suggests a memory problem.
if (message.match(/MemoryError|unmarshallable object/)) {
if (err.message.length > 30) {
// TLDR
err.message = 'Memory Error';
}
this._mostRecentDocPageModel?.offerRecovery(err);
}
}
}

@ -110,6 +110,7 @@ function pagePanelsDoc(owner: IDisposableOwner, appModel: AppModel, appObj: App)
// To simplify manual inspection in the common case, keep the most recently created
// DocPageModel available as a global variable.
(window as any).gristDocPageModel = pageModel;
appObj.setDocPageModel(pageModel);
const leftPanelOpen = createSessionObs<boolean>(owner, "leftPanelOpen", true, isBoolean);
const rightPanelOpen = createSessionObs<boolean>(owner, "rightPanelOpen", false, isBoolean);
const leftPanelWidth = createSessionObs<number>(owner, "leftPanelWidth", 240, isNumber);

@ -584,7 +584,13 @@ export class ActiveDoc extends EventEmitter {
this._startLoadingTables(docSession, desiredTableNames);
const pendingTableNames = tableNames.filter(name => !name.startsWith('_grist_'));
await this._initDoc(docSession);
this._initializationPromise = this._finishInitialization(docSession, pendingTableNames, onDemandNames, startTime);
this._initializationPromise = this._finishInitialization(docSession, pendingTableNames,
onDemandNames, startTime).catch(async (err) => {
await this.docClients.broadcastDocMessage(null, 'docError', {
when: 'initialization',
message: String(err),
});
});
} catch (err) {
await this.shutdown();
throw err;
@ -613,7 +619,8 @@ export class ActiveDoc extends EventEmitter {
public async _initDoc(docSession: OptDocSession): Promise<void> {
const metaTableData = await this._tableMetadataLoader.fetchTablesAsActions();
this.docData = new DocData(tableId => this.fetchTable(makeExceptionalDocSession('system'), tableId), metaTableData);
this._onDemandActions = new OnDemandActions(this.docStorage, this.docData);
this._onDemandActions = new OnDemandActions(this.docStorage, this.docData,
this._recoveryMode);
await this._actionHistory.initialize();
this._granularAccess = new GranularAccess(this.docData, this.docClients, (query) => {
@ -1579,7 +1586,8 @@ export class ActiveDoc extends EventEmitter {
// Figure out which tables are on-demand.
const onDemandMap = zipObject(tablesParsed.tableId as string[], tablesParsed.onDemand);
const onDemandNames = remove(tableNames, (t) => onDemandMap[t]);
const onDemandNames = remove(tableNames, (t) => (onDemandMap[t] ||
(this._recoveryMode && !t.startsWith('_grist_'))));
this._log.debug(docSession, "Loading %s normal tables, skipping %s on-demand tables",
tableNames.length, onDemandNames.length);

@ -495,6 +495,7 @@ export class DocWorkerApi {
const recoveryModeRaw = req.body.recoveryMode;
const recoveryMode = (typeof recoveryModeRaw === 'boolean') ? recoveryModeRaw : undefined;
if (!await this._isOwner(req)) { throw new Error('Only owners can control recovery mode'); }
this._docManager.setRecovery(getDocId(req), recoveryMode ?? true);
const activeDoc = await this._docManager.fetchDoc(docSessionFromRequest(req), getDocId(req), recoveryMode);
res.json({
recoveryMode: activeDoc.recoveryMode

@ -6,7 +6,7 @@ import {EventEmitter} from 'events';
import * as path from 'path';
import {ApiError} from 'app/common/ApiError';
import {mapSetOrClear} from 'app/common/AsyncCreate';
import {mapSetOrClear, MapWithTTL} from 'app/common/AsyncCreate';
import {BrowserSettings} from 'app/common/BrowserSettings';
import {DocCreationInfo, DocEntry, DocListAPI, OpenDocMode, OpenLocalDocResult} from 'app/common/DocListAPI';
import {FilteredDocUsageSummary} from 'app/common/DocUsage';
@ -37,6 +37,10 @@ import noop = require('lodash/noop');
// but is a bit of a burden under heavy traffic.
export const DEFAULT_CACHE_TTL = 10000;
// How long to remember that a document has been explicitly set in a
// recovery mode.
export const RECOVERY_CACHE_TTL = 30000;
/**
* DocManager keeps track of "active" Grist documents, i.e. those loaded
* in-memory, with clients connected to them.
@ -45,6 +49,8 @@ export class DocManager extends EventEmitter {
// Maps docName to promise for ActiveDoc object. Most of the time the promise
// will be long since resolved, with the resulting document cached.
private _activeDocs: Map<string, Promise<ActiveDoc>> = new Map();
// Remember recovery mode of documents.
private _inRecovery = new MapWithTTL<string, boolean>(RECOVERY_CACHE_TTL);
constructor(
public readonly storageManager: IDocStorageManager,
@ -55,6 +61,10 @@ export class DocManager extends EventEmitter {
super();
}
public setRecovery(docId: string, recovery: boolean) {
this._inRecovery.set(docId, recovery);
}
// attach a home database to the DocManager. During some tests, it
// is awkward to have this set up at the point of construction.
public testSetHomeDbManager(dbManager: HomeDBManager) {
@ -459,7 +469,7 @@ export class DocManager extends EventEmitter {
if (!this._activeDocs.has(docName)) {
activeDoc = await mapSetOrClear(
this._activeDocs, docName,
this._createActiveDoc(docSession, docName, wantRecoveryMode)
this._createActiveDoc(docSession, docName, wantRecoveryMode ?? this._inRecovery.get(docName))
.then(newDoc => {
// Propagate backupMade events from newly opened activeDocs (consolidate all to DocMan)
newDoc.on('backupMade', (bakPath: string) => {

@ -1381,7 +1381,7 @@ export class GranularAccess implements GranularAccessForBundle {
// If aclAsUserId/aclAsUser is set, then override user for acl purposes.
if (linkParameters.aclAsUserId || linkParameters.aclAsUser) {
if (access !== 'owners') { throw new Error('only an owner can override user'); }
if (access !== 'owners') { throw new ErrorWithCode('ACL_DENY', 'only an owner can override user'); }
if (attrs.override) {
// Used cached properties.
access = attrs.override.access;

@ -1,6 +1,7 @@
/**
* JS controller for the pypy sandbox.
*/
import {arrayToString} from 'app/common/arrayToString';
import * as marshal from 'app/common/marshal';
import {ISandbox, ISandboxCreationOptions, ISandboxCreator} from 'app/server/lib/ISandbox';
import * as log from 'app/server/lib/log';
@ -94,6 +95,7 @@ export class NSandbox implements ISandbox {
private _logMeta: log.ILogMeta;
private _streamToSandbox: Writable;
private _streamFromSandbox: Stream;
private _lastStderr: Uint8Array; // Record last error line seen.
// Create a unique subdirectory for each sandbox process so they can be replayed separately
private _recordBuffersDir = recordBuffersRoot ? path.resolve(recordBuffersRoot, new Date().toISOString()) : null;
@ -131,7 +133,11 @@ export class NSandbox implements ISandbox {
this._streamFromSandbox = (this.childProc.stdio as Stream[])[4];
this.childProc.stdout.on('data', sandboxUtil.makeLinePrefixer('Sandbox stdout: ', this._logMeta));
}
this.childProc.stderr.on('data', sandboxUtil.makeLinePrefixer('Sandbox stderr: ', this._logMeta));
const sandboxStderrLogger = sandboxUtil.makeLinePrefixer('Sandbox stderr: ', this._logMeta);
this.childProc.stderr.on('data', data => {
this._lastStderr = data;
sandboxStderrLogger(data);
});
this.childProc.on('close', this._onExit.bind(this));
this.childProc.on('error', this._onError.bind(this));
@ -254,7 +260,7 @@ export class NSandbox implements ISandbox {
*/
private _sendData(msgCode: MsgCode, data: any) {
if (this._isReadClosed) {
throw new sandboxUtil.SandboxError("PipeToSandbox is closed");
throw this._sandboxClosedError('PipeToSandbox');
}
this._marshaller.marshal(msgCode);
this._marshaller.marshal(data);
@ -287,12 +293,24 @@ export class NSandbox implements ISandbox {
this._control.prepareToClose();
this._isReadClosed = true;
// Clear out all reads pending on PipeFromSandbox, rejecting them with the given error.
const err = new sandboxUtil.SandboxError("PipeFromSandbox is closed");
const err = this._sandboxClosedError('PipeFromSandbox');
this._pendingReads.forEach(resolvePair => resolvePair[1](err));
this._pendingReads = [];
}
/**
* Generate an error message for a pipe to the sandbox. Include the
* last stderr line seen from the sandbox - more reliable than
* error results send via the standard protocol.
*/
private _sandboxClosedError(label: string) {
const parts = [`${label} is closed`];
if (this._lastStderr) {
parts.push(arrayToString(this._lastStderr));
}
return new sandboxUtil.SandboxError(parts.join(': '));
}
/**
* Process a parsed message from the sandboxed process.

@ -25,11 +25,13 @@ export class OnDemandActions {
private _tablesMeta: TableData = this._docData.getMetaTable('_grist_Tables');
private _columnsMeta: TableData = this._docData.getMetaTable('_grist_Tables_column');
constructor(private _storage: OnDemandStorage, private _docData: DocData) {}
constructor(private _storage: OnDemandStorage, private _docData: DocData,
private _forceOnDemand: boolean = false) {}
// TODO: Ideally a faster data structure like an index by tableId would be used to decide whether
// the table is onDemand.
public isOnDemand(tableId: string): boolean {
if (this._forceOnDemand) { return true; }
const tableRef = this._tablesMeta.findRow('tableId', tableId);
// OnDemand tables must have a record in the _grist_Tables metadata table.
return tableRef ? Boolean(this._tablesMeta.getValue(tableRef, 'onDemand')) : false;

@ -866,6 +866,9 @@ class Engine(object):
raise self._cell_required_error # pylint: disable=raising-bad-type
self.formula_tracer(col, record)
return result
except MemoryError:
# Don't try to wrap memory errors.
raise
except: # pylint: disable=bare-except
# Since col.method runs untrusted user code, we use a bare except to catch all
# exceptions (even those not derived from BaseException).

@ -44,6 +44,9 @@ if os.environ.get('CHECK_FOR_TERMINAL') == '1':
args = parser.parse_args(main_args)
sys.stderr.write('run.py: ' + ' '.join(sys.argv) + "\n")
sys.stderr.flush()
include_bash = args.command == 'bash'
include_python2 = args.command == 'python2'
include_python3 = args.command == 'python3'
@ -84,7 +87,7 @@ settings = {
},
"args": cmd_args,
"env": env,
"cwd": "/",
"cwd": "/"
},
"root": {
"path": "/", # The fork of gvisor we use shares paths with host.
@ -113,6 +116,15 @@ settings = {
]
}
}
memory_limit = os.environ.get('GVISOR_LIMIT_MEMORY')
if memory_limit:
settings['process']['rlimits'] = [
{
"type": "RLIMIT_AS",
"hard": int(memory_limit),
"soft": int(memory_limit)
}
]
# Helper for preparing a mount.
def preserve(*locations, short_failure=False):
@ -184,8 +196,10 @@ if args.mount:
for directory in os.listdir('/'):
if directory not in exceptions and ("/" + directory) not in preserved:
mounts.insert(0, {
"destination": "/" + directory,
"type": "tmpfs" # This places an empty directory at this destination.
# This places an empty directory at this destination.
# Follow any symlinks since otherwise there is an error.
"destination": os.path.realpath("/" + directory),
"type": "tmpfs"
})
# Set up faketime inside the sandbox if requested. Can't be set up outside the sandbox,
@ -245,12 +259,21 @@ with tempfile.TemporaryDirectory() as root: # pylint: disable=no-member
# Start up the sandbox, and wait for it to emit a message on stderr ('Ready').
command = make_command(root, ["run"])
process = subprocess.Popen(command, cwd=root, stderr=subprocess.PIPE)
ready_line = process.stderr.readline() # wait for ready
sys.stderr.write('Ready message: ' + ready_line.decode('utf-8'))
sys.stderr.flush()
text = process.stderr.readline().decode('utf-8') # wait for ready
if 'Ready' in text:
sys.stderr.write('Ready message: ' + text)
sys.stderr.flush()
else:
# Something unexpected has happened, echo the full error and hang.
while True:
sys.stderr.write('Problem: ' + text)
sys.stderr.flush()
text = process.stderr.readline().decode('utf-8')
# Remove existing checkpoint if present.
if os.path.exists(os.path.join(args.checkpoint, 'checkpoint.img')):
os.remove(os.path.join(args.checkpoint, 'checkpoint.img'))
if os.path.exists(os.path.join(args.checkpoint, 'checkpoint.json')):
os.remove(os.path.join(args.checkpoint, 'checkpoint.json'))
# Make the directory, so we will later have the right to delete the checkpoint if
# we wish to replace it. Otherwise there is a muddle around permissions.
if not os.path.exists(args.checkpoint):
@ -261,4 +284,9 @@ with tempfile.TemporaryDirectory() as root: # pylint: disable=no-member
result = subprocess.run(command, cwd=root) # pylint: disable=no-member
if result.returncode != 0:
raise Exception('gvisor runsc checkpointing problem: ' + json.dumps(command))
# Save the configuration of the checkpoint for easy reference.
with open(config_filename, 'r', encoding='utf-8') as fin:
with open(os.path.join(args.checkpoint, 'checkpoint.json'), 'w', encoding='utf-8') as fout:
spec = json.load(fin)
json.dump(spec, fout, indent=2)
# We are done!

@ -14,7 +14,7 @@
# Note for mac users: the checkpoint will be made in the docker
# container running runsc.
set -ex
set -e
SCRIPT_DIR=$( cd -- "$( dirname -- "${BASH_SOURCE[0]}" )" &> /dev/null && pwd )

Loading…
Cancel
Save