(core) revamp snapshot inventory

Summary:
Deliberate changes:
 * save snapshots to s3 prior to migrations.
 * label migration snapshots in s3 metadata.
 * avoid pruning migration snapshots for a month.

Opportunistic changes:
 * Associate document timezone with snapshots, so pruning can respect timezones.
 * Associate actionHash/Num with snapshots.
 * Record time of last change in snapshots (rather than just s3 upload time, which could be a while later).

This ended up being a biggish change, because there was nowhere ideal to put tags (list of possibilities in diff).

Test Plan: added tests

Reviewers: dsagal

Reviewed By: dsagal

Differential Revision: https://phab.getgrist.com/D2646
pull/3/head
Paul Fitzpatrick 4 years ago
parent ce824aad34
commit 71519d9e5c

@ -0,0 +1,37 @@
/**
* Core metadata about a single document version.
*/
export interface ObjSnapshot {
lastModified: string;
snapshotId: string;
}
/**
* Extended Grist metadata about a single document version. Names of fields are kept
* short since there is a tight limit on total metadata size in S3.
*/
export interface ObjMetadata {
t?: string; // timestamp
tz?: string; // timezone
h?: string; // actionHash
n?: number; // actionNum
label?: string;
}
export interface ObjSnapshotWithMetadata extends ObjSnapshot {
metadata?: ObjMetadata;
}
/**
* Information about a single document snapshot in S3, including a Grist docId.
*/
export interface DocSnapshot extends ObjSnapshotWithMetadata {
docId: string;
}
/**
* A collection of document snapshots. Most recent snapshots first.
*/
export interface DocSnapshots {
snapshots: DocSnapshot[];
}

@ -0,0 +1,46 @@
import { Mutex, MutexInterface } from 'async-mutex';
/**
* A per-key mutex. It has the same interface as Mutex, but with an extra key supplied.
* Maintains an independent mutex for each key on need.
*/
export class KeyedMutex {
private _mutexes = new Map<string, Mutex>();
public async acquire(key: string): Promise<MutexInterface.Releaser> {
// Create a new mutex if we need one.
if (!this._mutexes.has(key)) {
this._mutexes.set(key, new Mutex());
}
const mutex = this._mutexes.get(key)!
const unlock = await mutex.acquire();
return () => {
unlock();
// After unlocking, clean-up the mutex if it is no longer needed.
// unlock() leaves the mutex locked if anyone has been waiting for it.
if (!mutex.isLocked()) {
this._mutexes.delete(key);
}
};
}
public async runExclusive<T>(key: string, callback: MutexInterface.Worker<T>): Promise<T> {
const unlock = await this.acquire(key);
try {
return callback();
} finally {
unlock();
}
}
public isLocked(key: string): boolean {
const mutex = this._mutexes.get(key);
if (!mutex) { return false; }
return mutex.isLocked();
}
// Check how many mutexes are in use.
public get size(): number {
return this._mutexes.size;
}
}

@ -102,6 +102,14 @@ class DummyDocWorkerMap implements IDocWorkerMap {
this._elections.delete(name);
}
}
public async updateChecksum(family: string, key: string, checksum: string) {
// nothing to do
}
public async getChecksum(family: string, key: string) {
return null;
}
}
/**
@ -367,7 +375,16 @@ export class DocWorkerMap implements IDocWorkerMap {
}
public async updateDocStatus(docId: string, checksum: string): Promise<void> {
await this._client.setexAsync(`doc-${docId}-checksum`, CHECKSUM_TTL_MSEC / 1000.0, checksum);
this.updateChecksum('doc', docId, checksum);
}
public async updateChecksum(family: string, key: string, checksum: string) {
await this._client.setexAsync(`${family}-${key}-checksum`, CHECKSUM_TTL_MSEC / 1000.0, checksum);
}
public async getChecksum(family: string, key: string) {
const checksum = await this._client.getAsync(`${family}-${key}-checksum`);
return checksum === 'null' ? null : checksum;
}
public async setPermit(permit: Permit): Promise<string> {

@ -4,7 +4,7 @@
// TypeORM Sqlite driver does not support using transactions in async code, if it is possible
// for two transactions to get called (one of the whole point of transactions). This
// patch adds support for that, based on a monkey patch published in:
// https://gist.github.com/keenondrums/556f8c61d752eff730841170cd2bc3f1
// https://gist.github.com/aigoncharov/556f8c61d752eff730841170cd2bc3f1
// Explanation at https://github.com/typeorm/typeorm/issues/1884#issuecomment-380767213
// Patch 2:
@ -13,6 +13,7 @@
// changed during construction of a query.
import * as sqlite3 from '@gristlabs/sqlite3';
import {Mutex, MutexInterface} from 'async-mutex';
import isEqual = require('lodash/isEqual');
import {EntityManager, QueryRunner} from 'typeorm';
import {SqliteDriver} from 'typeorm/driver/sqlite/SqliteDriver';
@ -27,73 +28,11 @@ import {QueryBuilder} from 'typeorm/query-builder/QueryBuilder';
* Patch 1
**********************/
type Releaser = () => void;
type Worker<T> = () => Promise<T>|T;
interface MutexInterface {
acquire(): Promise<Releaser>;
runExclusive<T>(callback: Worker<T>): Promise<T>;
isLocked(): boolean;
}
class Mutex implements MutexInterface {
private _queue: Array<(release: Releaser) => void> = [];
private _pending = false;
public isLocked(): boolean {
return this._pending;
}
public acquire(): Promise<Releaser> {
const ticket = new Promise<Releaser>(resolve => this._queue.push(resolve));
if (!this._pending) {
this._dispatchNext();
}
return ticket;
}
public runExclusive<T>(callback: Worker<T>): Promise<T> {
return this
.acquire()
.then(release => {
let result: T|Promise<T>;
try {
result = callback();
} catch (e) {
release();
throw(e);
}
return Promise
.resolve(result)
.then(
(x: T) => (release(), x),
e => {
release();
throw e;
}
);
}
);
}
private _dispatchNext(): void {
if (this._queue.length > 0) {
this._pending = true;
this._queue.shift()!(this._dispatchNext.bind(this));
} else {
this._pending = false;
}
}
}
// A singleton mutex for all sqlite transactions.
const mutex = new Mutex();
class SqliteQueryRunnerPatched extends SqliteQueryRunner {
private _releaseMutex: Releaser | null;
private _releaseMutex: MutexInterface.Releaser | null;
public async startTransaction(level?: any): Promise<void> {
this._releaseMutex = await mutex.acquire();

@ -26,6 +26,7 @@ import {mapGetOrSet, MapWithTTL} from 'app/common/AsyncCreate';
import {BulkColValues, CellValue, DocAction, RowRecord, TableDataAction, UserAction} from 'app/common/DocActions';
import {toTableDataAction} from 'app/common/DocActions';
import {DocData} from 'app/common/DocData';
import {DocSnapshots} from 'app/common/DocSnapshot';
import {EncActionBundleFromHub} from 'app/common/EncActionBundle';
import {byteString} from 'app/common/gutil';
import {InactivityTimer} from 'app/common/InactivityTimer';
@ -40,7 +41,6 @@ import {Authorizer} from 'app/server/lib/Authorizer';
import {checksumFile} from 'app/server/lib/checksumFile';
import {Client} from 'app/server/lib/Client';
import {DEFAULT_CACHE_TTL, DocManager} from 'app/server/lib/DocManager';
import {DocSnapshots} from 'app/server/lib/DocSnapshots';
import {makeForkIds} from 'app/server/lib/idUtils';
import {ISandbox} from 'app/server/lib/ISandbox';
import * as log from 'app/server/lib/log';
@ -109,6 +109,7 @@ export class ActiveDoc extends EventEmitter {
private _granularAccess: GranularAccess;
private _muted: boolean = false; // If set, changes to this document should not propagate
// to outside world
private _migrating: number = 0; // If positive, a migration is in progress
private _initializationPromise: Promise<boolean>|null = null;
// If set, wait on this to be sure the ActiveDoc is fully
// initialized. True on success.
@ -175,6 +176,10 @@ export class ActiveDoc extends EventEmitter {
return this._muted;
}
public isMigrating() {
return this._migrating;
}
// Note that this method is only used in tests, and should be avoided in production (see note
// in ActionHistory about getRecentActions).
public getRecentActionsDirect(maxActions?: number): Promise<LocalActionBundle[]> {
@ -311,7 +316,14 @@ export class ActiveDoc extends EventEmitter {
await this.createDoc(docSession);
await this.addInitialTable(docSession);
} else {
await this.docStorage.openFile();
await this.docStorage.openFile({
beforeMigration: async (currentVersion, newVersion) => {
return this._beforeMigration(docSession, 'storage', currentVersion, newVersion);
},
afterMigration: async (newVersion, success) => {
return this._afterMigration(docSession, 'storage', newVersion, success);
},
});
const tableNames = await this._loadOpenDoc(docSession);
const desiredTableNames = tableNames.filter(name => name.startsWith('_grist_'));
await this._loadTables(docSession, desiredTableNames);
@ -972,7 +984,14 @@ export class ActiveDoc extends EventEmitter {
const docSchemaVersion = (versionCol && versionCol.length === 1 ? versionCol[0] : 0);
if (docSchemaVersion < schemaVersion) {
this.logInfo(docSession, "Doc needs migration from v%s to v%s", docSchemaVersion, schemaVersion);
await this._migrate(docSession);
await this._beforeMigration(docSession, 'schema', docSchemaVersion, schemaVersion);
let success: boolean = false;
try {
await this._migrate(docSession);
success = true;
} finally {
await this._afterMigration(docSession, 'schema', schemaVersion, success);
}
} else if (docSchemaVersion > schemaVersion) {
// We do NOT attempt to down-migrate in this case. Migration code cannot down-migrate
// directly (since it doesn't know anything about newer documents). We could revert the
@ -1054,8 +1073,7 @@ export class ActiveDoc extends EventEmitter {
if (result.isModification) {
this._fetchCache.clear(); // This could be more nuanced.
this._docManager.markAsChanged(this);
this._docManager.markAsEdited(this);
this._docManager.markAsChanged(this, 'edit');
}
return result;
}
@ -1105,10 +1123,6 @@ export class ActiveDoc extends EventEmitter {
* collaborators.
*/
private async _migrate(docSession: OptDocSession): Promise<void> {
// TODO: makeBackup should possibly be in docManager directly.
const backupPath = await this._docManager.storageManager.makeBackup(this._docName, "migrate");
this.logInfo(docSession, "_migrate: backup made at %s", backupPath);
this.emit("backupMade", backupPath);
const allTables = await this.docStorage.fetchAllTables();
const docActions: DocAction[] = await this._dataEngine.pyCall('create_migrations', allTables);
this.logInfo(docSession, "_migrate: applying %d migration actions", docActions.length);
@ -1215,6 +1229,29 @@ export class ActiveDoc extends EventEmitter {
if (result.docActions.length === 0) { return null; }
return result;
}
/**
* Called before a migration. Makes sure a back-up is made.
*/
private async _beforeMigration(docSession: OptDocSession, versionType: 'storage' | 'schema',
currentVersion: number, newVersion: number) {
this._migrating++;
const label = `migrate-${versionType}-last-v${currentVersion}-before-v${newVersion}`;
this._docManager.markAsChanged(this); // Give backup current time.
const location = await this._docManager.makeBackup(this, label);
this.logInfo(docSession, "_beforeMigration: backup made with label %s at %s", label, location);
this.emit("backupMade", location);
}
/**
* Called after a migration.
*/
private async _afterMigration(docSession: OptDocSession, versionType: 'storage' | 'schema',
newVersion: number, success: boolean) {
this._migrating--;
// Mark as changed even if migration is not successful, out of caution.
if (!this._migrating) { this._docManager.markAsChanged(this); }
}
}
// Helper to initialize a sandbox action bundle with no values.

@ -337,16 +337,16 @@ export class DocManager extends EventEmitter {
}
}
public markAsChanged(activeDoc: ActiveDoc) {
if (!activeDoc.muted) {
this.storageManager.markAsChanged(activeDoc.docName);
public markAsChanged(activeDoc: ActiveDoc, reason?: 'edit') {
// Ignore changes if document is muted or in the middle of a migration.
if (!activeDoc.muted && !activeDoc.isMigrating()) {
this.storageManager.markAsChanged(activeDoc.docName, reason);
}
}
public markAsEdited(activeDoc: ActiveDoc) {
if (!activeDoc.muted) {
this.storageManager.markAsEdited(activeDoc.docName);
}
public async makeBackup(activeDoc: ActiveDoc, name: string): Promise<string> {
if (activeDoc.muted) { throw new Error('Document is disabled'); }
return this.storageManager.makeBackup(activeDoc.docName, name);
}
/**

@ -1,30 +1,17 @@
import { ObjSnapshotWithMetadata } from 'app/common/DocSnapshot';
import { KeyedOps } from 'app/common/KeyedOps';
import { KeyedMutex } from 'app/common/KeyedMutex';
import { ExternalStorage } from 'app/server/lib/ExternalStorage';
import * as log from 'app/server/lib/log';
import * as fse from 'fs-extra';
import * as moment from 'moment';
/**
* Metadata about a single document version.
* A subset of the ExternalStorage interface, focusing on maintaining a list of versions.
*/
export interface ObjSnapshot {
lastModified: Date;
snapshotId: string;
}
/**
* Information about a single document snapshot in S3, including a Grist docId.
* Similar to a type in app/common/UserAPI, but with lastModified as a Date
* rather than a string.
*/
export interface DocSnapshot extends ObjSnapshot {
docId: string;
}
/**
* A collection of document snapshots. Most recent snapshots first.
*/
export interface DocSnapshots {
snapshots: DocSnapshot[];
export interface IInventory {
versions(key: string): Promise<ObjSnapshotWithMetadata[]>;
remove(key: string, snapshotIds: string[]): Promise<void>;
}
/**
@ -35,7 +22,7 @@ export class DocSnapshotPruner {
private _prunes: KeyedOps;
// Specify store to be pruned, and delay before pruning.
constructor(private _ext: ExternalStorage, _options: {
constructor(private _ext: IInventory, _options: {
delayBeforeOperationMs?: number,
minDelayBetweenOperationsMs?: number
} = {}) {
@ -60,16 +47,18 @@ export class DocSnapshotPruner {
}
// Note that a document has changed, and should be pruned (or repruned). Pruning operation
// done as a background operation.
public requestPrune(key: string) {
// done as a background operation. Returns true if a pruning operation has been scheduled.
public requestPrune(key: string): boolean {
// If closing down, do not accept any prune requests.
if (this._closing) { return; }
// Mark the key as needing work.
this._prunes.addOperation(key);
if (!this._closing) {
// Mark the key as needing work.
this._prunes.addOperation(key);
}
return this._prunes.hasPendingOperation(key);
}
// Get all snapshots for a document, and whether they should be kept or pruned.
public async classify(key: string): Promise<Array<{snapshot: ObjSnapshot, keep: boolean}>> {
public async classify(key: string): Promise<Array<{snapshot: ObjSnapshotWithMetadata, keep: boolean}>> {
const versions = await this._ext.versions(key);
return shouldKeepSnapshots(versions).map((keep, index) => ({keep, snapshot: versions[index]}));
}
@ -83,24 +72,217 @@ export class DocSnapshotPruner {
}
}
/**
* Maintain a list of document versions, with metadata, so we can query versions and
* make sensible pruning decisions without needing to HEAD each version (in the
* steady state).
*
* The list of versions (with metadata) for a document is itself stored in S3. This isn't
* ideal since we cannnot simply append a new version to the list without rewriting it in full.
* But the alternatives have more serious problems, and this way folds quite well into the
* existing pruning setup.
* - Storing in db would mean we'd need sharding sooner than otherwise
* - Storing in redis would similarly make this the dominant load driving redis
* - Storing in dynamodb would create more operational work
* - Using S3 metadata alone would be too slow
* - Using S3 tags could do some of what we want, but tags have serious limits
*
* Operations related to a particular document are serialized for clarity.
*
* The inventory is cached on the local file system, since we reuse the ExternalStorage
* interface which is file based.
*/
export class DocSnapshotInventory implements IInventory {
private _needFlush = new Set<string>();
private _mutex = new KeyedMutex();
/**
* Expects to be given the store for documents, a store for metadata, and a method
* for naming cache files on the local filesystem. The stores should be consistent.
*/
constructor(private _doc: ExternalStorage, private _meta: ExternalStorage,
private _getFilename: (key: string) => Promise<string>) {}
/**
* Add a new snapshot of a document to the existing inventory. A prevSnapshotId may
* be supplied as a cross-check. It will be matched against the most recent
* snapshotId in the inventory, and if it doesn't match the inventory will be
* recreated.
*
* The inventory is not automatically flushed to S3. Call flush() to do that,
* or ask DocSnapshotPrune.requestPrune() to prune the document - it will flush
* after pruning.
*
* The snapshot supplied will be modified in place to a normalized form.
*/
public async add(key: string, snapshot: ObjSnapshotWithMetadata, prevSnapshotId: string|null) {
await this._mutex.runExclusive(key, async() => {
const snapshots = await this._getSnapshots(key, prevSnapshotId);
// Could be already added if reconstruction happened.
if (snapshots[0].snapshotId === snapshot.snapshotId) { return; }
this._normalizeMetadata(snapshot);
snapshots.unshift(snapshot);
const fname = await this._getFilename(key);
await this._saveToFile(fname, snapshots);
// We don't write to s3 yet, but do mark the list as dirty.
this._needFlush.add(key);
});
}
/**
* Make sure the latest state of the inventory is stored in S3.
*/
public async flush(key: string) {
await this._mutex.runExclusive(key, async() => {
await this._flush(key);
});
}
/**
* Remove a set of snapshots from the inventory, and then flush to S3.
*/
public async remove(key: string, snapshotIds: string[]) {
await this._mutex.runExclusive(key, async() => {
const current = await this._getSnapshots(key, null);
const oldIds = new Set(snapshotIds);
if (oldIds.size > 0) {
const results = current.filter(v => !oldIds.has(v.snapshotId));
const fname = await this._getFilename(key);
await this._saveToFile(fname, results);
this._needFlush.add(key);
}
await this._flush(key);
});
}
/**
* Read the cached version of the inventory if available, otherwise fetch
* it from S3. If expectSnapshotId is set, the cached version is ignored if
* the most recent version listed is not the expected one.
*/
public async versions(key: string, expectSnapshotId?: string|null): Promise<ObjSnapshotWithMetadata[]> {
return this._mutex.runExclusive(key, async() => {
return await this._getSnapshots(key, expectSnapshotId || null);
});
}
// Do whatever it takes to get an inventory of versions.
// Most recent versions returned first.
private async _getSnapshots(key: string, expectSnapshotId: string|null): Promise<ObjSnapshotWithMetadata[]> {
// Check if we have something useful cached on the local filesystem.
const fname = await this._getFilename(key);
let data = await this._loadFromFile(fname);
if (data && expectSnapshotId && data[0]?.snapshotId !== expectSnapshotId) {
data = null;
}
// If nothing yet, check if we have something useful in s3.
if (!data && await this._meta.exists(key)) {
await fse.remove(fname);
await this._meta.download(key, fname);
data = await this._loadFromFile(fname);
if (data && expectSnapshotId && data[0]?.snapshotId !== expectSnapshotId) {
data = null;
}
}
if (!data) {
// No joy, all we can do is reconstruct from individual s3 version HEAD metadata.
data = await this._reconstruct(key);
if (data) {
if (expectSnapshotId && data[0]?.snapshotId !== expectSnapshotId) {
// Surprising, since S3 ExternalInterface should have its own consistency
// checks. Not much we can do about it other than accept it.
log.error(`Surprise in getSnapshots, expected ${expectSnapshotId} for ${key} ` +
`but got ${data[0]?.snapshotId}`);
}
// Reconstructed data is precious. Save it to S3 and local cache.
await this._saveToFile(fname, data);
await this._meta.upload(key, fname);
}
}
return data;
}
// Load inventory from local file system, if available.
private async _loadFromFile(fname: string): Promise<ObjSnapshotWithMetadata[]|null> {
try {
if (await fse.pathExists(fname)) {
return JSON.parse(await fse.readFile(fname, 'utf8'));
}
return null;
} catch (e) {
return null;
}
}
// Save inventory to local file system.
private async _saveToFile(fname: string, data: ObjSnapshotWithMetadata[]) {
await fse.outputFile(fname, JSON.stringify(data, null, 2), 'utf8');
}
// This is a relatively expensive operation, calling the S3 api for every stored
// version of a document. In the steady state, we should rarely need to do this.
private async _reconstruct(key: string): Promise<ObjSnapshotWithMetadata[]> {
const snapshots = await this._doc.versions(key);
if (snapshots.length > 1) {
log.info(`Reconstructing history of ${key} (${snapshots.length} versions)`);
}
const results: ObjSnapshotWithMetadata[] = [];
for (const snapshot of snapshots) {
const head = await this._doc.head(key, snapshot.snapshotId);
if (head) {
this._normalizeMetadata(head);
results.push(head);
} else {
log.debug(`When reconstructing history of ${key}, did not find ${snapshot.snapshotId}`);
}
}
return results;
}
// Flush inventory to S3.
private async _flush(key: string) {
if (this._needFlush.has(key)) {
const fname = await this._getFilename(key);
await this._meta.upload(key, fname);
this._needFlush.delete(key);
}
}
// Normalize metadata. We store a timestamp that is distinct from the S3 timestamp,
// recording when the file was changed by Grist.
// TODO: deal with possibility of this creating trouble with pruning if the local time is
// sufficiently wrong.
private _normalizeMetadata(snapshot: ObjSnapshotWithMetadata) {
if (snapshot?.metadata?.t) {
snapshot.lastModified = snapshot.metadata.t;
delete snapshot.metadata.t;
}
}
}
/**
* Calculate which snapshots to keep. Expects most recent snapshots to be first.
* We keep:
* - The five most recent versions (including the current version)
* - The most recent version in every hour, for up to 25 hours before the current version
* - The most recent version in every day, for up to 32 days before the current version
* - The most recent version in every week, for up to 12 weeks before the current version
* - The most recent version in every month, for up to 36 months before the current version
* - The most recent version in every year, for up to 1000 years before the current version
* - The most recent version in every hour, for up to 25 distinct hours
* - The most recent version in every day, for up to 32 distinct days
* - The most recent version in every week, for up to 12 distinct weeks
* - The most recent version in every month, for up to 36 distinct months
* - The most recent version in every year, for up to 1000 distinct years
* - Anything with a label, for up to 32 days before the current version.
* Calculations done in UTC, Gregorian calendar, ISO weeks (week starts with Monday).
*/
export function shouldKeepSnapshots(snapshots: ObjSnapshot[]): boolean[] {
export function shouldKeepSnapshots(snapshots: ObjSnapshotWithMetadata[]): boolean[] {
// Get current version
const current = snapshots[0];
if (!current) { return []; }
const tz = current.metadata?.tz || 'UTC';
// Get time of current version
const start = moment.utc(current.lastModified);
const start = moment.tz(current.lastModified, tz);
// Track saved version per hour, day, week, month, year, and number of times a version
// has been saved based on a corresponding rule.
@ -115,10 +297,14 @@ export function shouldKeepSnapshots(snapshots: ObjSnapshot[]): boolean[] {
// it with the last saved snapshot based on hour, day, week, month, year
return snapshots.map((snapshot, index) => {
let keep = index < 5; // Keep 5 most recent versions
const date = moment.utc(snapshot.lastModified);
const date = moment.tz(snapshot.lastModified, tz);
for (const bucket of buckets) {
if (updateAndCheckRange(date, bucket)) { keep = true; }
}
// Preserve recent labelled snapshots in a naive and limited way. No doubt this will
// be elaborated on if we make this a user-facing feature.
if (snapshot.metadata?.label &&
start.diff(moment.tz(snapshot.lastModified, tz), 'days') < 32) { keep = true; }
return keep;
});
}

@ -25,7 +25,7 @@ import groupBy = require('lodash/groupBy');
import * as _ from 'underscore';
import * as util from 'util';
import * as uuidv4 from "uuid/v4";
import {ISQLiteDB, OpenMode, quoteIdent, ResultRow, SchemaInfo, SQLiteDB} from './SQLiteDB';
import { ISQLiteDB, MigrationHooks, OpenMode, quoteIdent, ResultRow, SchemaInfo, SQLiteDB} from './SQLiteDB';
// Run with environment variable NODE_DEBUG=db (may include additional comma-separated sections)
@ -458,10 +458,10 @@ export class DocStorage implements ISQLiteDB {
/**
* Opens an existing SQLite database and prepares it for use.
*/
public openFile(): Promise<void> {
public openFile(hooks: MigrationHooks = {}): Promise<void> {
// It turns out to be important to return a bluebird promise, a lot of code outside
// of DocStorage ultimately depends on this.
return bluebird.Promise.resolve(this._openFile(OpenMode.OPEN_EXISTING))
return bluebird.Promise.resolve(this._openFile(OpenMode.OPEN_EXISTING, hooks))
.then(() => this._initDB())
.then(() => this._updateMetadata());
}
@ -473,7 +473,7 @@ export class DocStorage implements ISQLiteDB {
public createFile(): Promise<void> {
// It turns out to be important to return a bluebird promise, a lot of code outside
// of DocStorage ultimately depends on this.
return bluebird.Promise.resolve(this._openFile(OpenMode.CREATE_EXCL))
return bluebird.Promise.resolve(this._openFile(OpenMode.CREATE_EXCL, {}))
.then(() => this._initDB());
// Note that we don't call _updateMetadata() as there are no metadata tables yet anyway.
}
@ -1193,9 +1193,9 @@ export class DocStorage implements ISQLiteDB {
* Creates a new or opens an existing SQLite database, depending on mode.
* @return {Promise<number>} Promise for user_version stored in the database.
*/
private async _openFile(mode: number): Promise<number> {
private async _openFile(mode: number, hooks: MigrationHooks): Promise<number> {
try {
this._db = await SQLiteDB.openDB(this.docPath, DocStorage.docStorageSchema, mode);
this._db = await SQLiteDB.openDB(this.docPath, DocStorage.docStorageSchema, mode, hooks);
log.debug("DB %s open successfully", this.docName);
return this._db.getMigrationVersion();
} catch (err) {

@ -5,10 +5,10 @@ import * as moment from 'moment';
import * as path from 'path';
import {DocEntry, DocEntryTag} from 'app/common/DocListAPI';
import {DocSnapshots} from 'app/common/DocSnapshot';
import * as gutil from 'app/common/gutil';
import * as Comm from 'app/server/lib/Comm';
import {OptDocSession} from 'app/server/lib/DocSession';
import {DocSnapshots} from 'app/server/lib/DocSnapshots';
import * as docUtils from 'app/server/lib/docUtils';
import {GristServer} from 'app/server/lib/GristServer';
import {IDocStorageManager} from 'app/server/lib/IDocStorageManager';

@ -3,6 +3,7 @@
* TODO This is not yet implemented, there is only a hard-coded stub.
*/
import { IChecksumStore } from 'app/server/lib/IChecksumStore';
import { IElectionStore } from 'app/server/lib/IElectionStore';
import { IPermitStore } from 'app/server/lib/Permit';
@ -32,7 +33,7 @@ export interface DocStatus {
/**
* Assignment of documents to workers, and other storage related to distributed work.
*/
export interface IDocWorkerMap extends IPermitStore, IElectionStore {
export interface IDocWorkerMap extends IPermitStore, IElectionStore, IChecksumStore {
// Looks up which DocWorker is responsible for this docId.
getDocWorker(docId: string): Promise<DocStatus|null>;

@ -1,4 +1,4 @@
import {ObjSnapshot} from 'app/server/lib/DocSnapshots';
import {ObjMetadata, ObjSnapshot, ObjSnapshotWithMetadata} from 'app/common/DocSnapshot';
import * as log from 'app/server/lib/log';
import {createTmpDir} from 'app/server/lib/uploads';
import {delay} from 'bluebird';
@ -20,10 +20,13 @@ export const DELETED_TOKEN = '*DELETED*';
*/
export interface ExternalStorage {
// Check if content exists in the store for a given key.
exists(key: string): Promise<boolean>;
exists(key: string, snapshotId?: string): Promise<boolean>;
// Get side information for content, if content exists in the store.
head(key: string, snapshotId?: string): Promise<ObjSnapshotWithMetadata|null>;
// Upload content from file to the given key. Returns a snapshotId if store supports that.
upload(key: string, fname: string): Promise<string|null>;
upload(key: string, fname: string, metadata?: ObjMetadata): Promise<string|null>;
// Download content from key to given file. Can download a specific version of the key
// if store supports that (should throw a fatal exception if not).
@ -59,12 +62,16 @@ export class KeyMappedExternalStorage implements ExternalStorage {
private _map: (key: string) => string) {
}
public exists(key: string): Promise<boolean> {
return this._ext.exists(this._map(key));
public exists(key: string, snapshotId?: string): Promise<boolean> {
return this._ext.exists(this._map(key), snapshotId);
}
public head(key: string, snapshotId?: string) {
return this._ext.head(this._map(key), snapshotId);
}
public upload(key: string, fname: string) {
return this._ext.upload(this._map(key), fname);
public upload(key: string, fname: string, metadata?: ObjMetadata) {
return this._ext.upload(this._map(key), fname, metadata);
}
public download(key: string, fname: string, snapshotId?: string) {
@ -135,31 +142,26 @@ export class ChecksummedExternalStorage implements ExternalStorage {
}) {
}
public async exists(key: string): Promise<boolean> {
return this._retry('exists', async () => {
const hash = await this._options.sharedHash.load(key);
const expected = hash !== null && hash !== DELETED_TOKEN;
const reported = await this._ext.exists(key);
// If we expect an object but store doesn't seem to have it, retry.
if (expected && !reported) { return undefined; }
// If store says there is an object but that is not what we expected (if we
// expected anything), retry.
if (hash && !expected && reported) { return undefined; }
// If expectations are matched, or we don't have expectations, return.
return reported;
});
public async exists(key: string, snapshotId?: string): Promise<boolean> {
return this._retryWithExistenceCheck('exists', key, snapshotId,
this._ext.exists.bind(this._ext));
}
public async upload(key: string, fname: string) {
public async head(key: string, snapshotId?: string) {
return this._retryWithExistenceCheck('head', key, snapshotId,
this._ext.head.bind(this._ext));
}
public async upload(key: string, fname: string, metadata?: ObjMetadata) {
try {
const checksum = await this._options.computeFileHash(fname);
const prevChecksum = await this._options.localHash.load(key);
if (prevChecksum && prevChecksum === checksum) {
if (prevChecksum && prevChecksum === checksum && !metadata?.label) {
// nothing to do, checksums match
log.info("ext upload: %s unchanged, not sending", key);
return this._options.latestVersion.load(key);
}
const snapshotId = await this._ext.upload(key, fname);
const snapshotId = await this._ext.upload(key, fname, metadata);
log.info("ext upload: %s checksum %s", this._ext.url(key), checksum);
if (snapshotId) { await this._options.latestVersion.save(key, snapshotId); }
await this._options.localHash.save(key, checksum);
@ -183,6 +185,11 @@ export class ChecksummedExternalStorage implements ExternalStorage {
if (!snapshotIds) {
await this._options.latestVersion.save(key, DELETED_TOKEN);
await this._options.sharedHash.save(key, DELETED_TOKEN);
} else for (const snapshotId of snapshotIds) {
// Removing snapshots breaks their partial immutability, so we mark them
// as deleted in redis so that we don't get stale info from S3 if we check
// for their existence. Nothing currently depends on this in practice.
await this._options.sharedHash.save(this._keyWithSnapshot(key, snapshotId), DELETED_TOKEN);
}
} catch (err) {
log.error("ext delete: %s failure to remove, error %s", key, err.message);
@ -203,7 +210,7 @@ export class ChecksummedExternalStorage implements ExternalStorage {
public async downloadTo(fromKey: string, toKey: string, fname: string, snapshotId?: string) {
await this._retry('download', async () => {
const {tmpDir, cleanupCallback} = await createTmpDir({});
const tmpPath = path.join(tmpDir, `${toKey}.grist-tmp`); // NOTE: assumes key is file-system safe.
const tmpPath = path.join(tmpDir, `${toKey}-tmp`); // NOTE: assumes key is file-system safe.
try {
await this._ext.download(fromKey, tmpPath, snapshotId);
@ -309,6 +316,35 @@ export class ChecksummedExternalStorage implements ExternalStorage {
log.error(`operation failed to become consistent: ${name} - ${problems}`);
throw new Error(`operation failed to become consistent: ${name} - ${problems}`);
}
/**
* Retry an operation which will fail if content does not exist, until it is consistent
* with our expectation of the content's existence.
*/
private async _retryWithExistenceCheck<T>(label: string, key: string, snapshotId: string|undefined,
op: (key: string, snapshotId?: string) => Promise<T>): Promise<T> {
return this._retry(label, async () => {
const hash = await this._options.sharedHash.load(this._keyWithSnapshot(key, snapshotId));
const expected = hash !== null && hash !== DELETED_TOKEN;
const reported = await op(key, snapshotId);
// If we expect an object but store doesn't seem to have it, retry.
if (expected && !reported) { return undefined; }
// If store says there is an object but that is not what we expected (if we
// expected anything), retry.
if (hash && !expected && reported) { return undefined; }
// If expectations are matched, or we don't have expectations, return.
return reported;
});
}
/**
* Generate a key to use with Redis for a document. Add in snapshot information
* if that is present (snapshots are immutable, except that they can be deleted,
* so we only set checksums for them in Redis when they are deleted).
*/
private _keyWithSnapshot(key: string, snapshotId?: string|null) {
return snapshotId ? `${key}--${snapshotId}` : key;
}
}
/**

@ -829,15 +829,20 @@ export class FlexServer implements GristServer {
this.addComm();
if (!isSingleUserMode()) {
const s3Bucket = this._disableS3 ? '' : (process.env.GRIST_DOCS_S3_BUCKET || '');
const s3Prefix = process.env.GRIST_DOCS_S3_PREFIX || "docs/";
this.info.push(['s3Bucket', `${s3Bucket}/${s3Prefix}`]);
if (!process.env.GRIST_DOCS_S3_BUCKET || process.env.GRIST_DISABLE_S3 === 'true') {
this._disableS3 = true;
}
for (const [key, val] of Object.entries(this.create.configurationOptions())) {
this.info.push([key, val]);
}
if (this._disableS3) {
this.info.push(['s3', 'disabled']);
}
const workers = this._docWorkerMap;
const docWorkerId = await this._addSelfAsWorker(workers);
const storageManager = new HostedStorageManager(this.docsRoot, docWorkerId, s3Bucket, s3Prefix, workers,
const storageManager = new HostedStorageManager(this.docsRoot, docWorkerId, this._disableS3, '', workers,
this.dbManager, this.create);
this._storageManager = storageManager;
} else {

@ -44,12 +44,15 @@ export class HostedMetadataManager {
}
/**
* Schedule a call to _update some time from now.
* Schedule a call to _update some time from now. When the update is made, it will
* store the given timestamp in the updated_at column of the docs table for the
* specified document. Timestamp should be an ISO 8601 format time, in UTC, e.g.
* the output of new Date().toISOString()
*/
public scheduleUpdate(docId: string): void {
public scheduleUpdate(docId: string, timestamp: string): void {
// Update updatedAt even if an update is already scheduled - if the update has not yet occurred,
// the more recent updatedAt time will be used.
this._updatedAt[docId] = new Date().toISOString();
this._updatedAt[docId] = timestamp;
if (this._timeout || this._closing) { return; }
const minDelay = this._minPushDelay * 1000;
// Set the push to occur at least the minDelay after the last push time.

@ -2,6 +2,7 @@ import * as sqlite3 from '@gristlabs/sqlite3';
import {mapGetOrSet} from 'app/common/AsyncCreate';
import {delay} from 'app/common/delay';
import {DocEntry} from 'app/common/DocListAPI';
import {DocSnapshots} from 'app/common/DocSnapshot';
import {buildUrlId, parseUrlId} from 'app/common/gristUrls';
import {KeyedOps} from 'app/common/KeyedOps';
import {DocReplacementOptions, NEW_DOCUMENT_CODE} from 'app/common/UserAPI';
@ -9,7 +10,7 @@ import {HomeDBManager} from 'app/gen-server/lib/HomeDBManager';
import {getUserId} from 'app/server/lib/Authorizer';
import {checksumFile} from 'app/server/lib/checksumFile';
import {OptDocSession} from 'app/server/lib/DocSession';
import {DocSnapshotPruner, DocSnapshots} from 'app/server/lib/DocSnapshots';
import {DocSnapshotInventory, DocSnapshotPruner} from 'app/server/lib/DocSnapshots';
import {IDocWorkerMap} from 'app/server/lib/DocWorkerMap';
import {ChecksummedExternalStorage, DELETED_TOKEN, ExternalStorage} from 'app/server/lib/ExternalStorage';
import {HostedMetadataManager} from 'app/server/lib/HostedMetadataManager';
@ -20,6 +21,7 @@ import {fromCallback} from 'app/server/lib/serverUtils';
import * as fse from 'fs-extra';
import * as path from 'path';
import * as uuidv4 from "uuid/v4";
import { OpenMode, SQLiteDB } from './SQLiteDB';
// Check for a valid document id.
const docIdRegex = /^[-=_\w~%]+$/;
@ -49,7 +51,8 @@ interface HostedStorageOptions {
secondsBeforePush: number;
secondsBeforeFirstRetry: number;
pushDocUpdateTimes: boolean;
testExternalStorage?: ExternalStorage;
testExternalStorageDoc?: ExternalStorage;
testExternalStorageMeta?: ExternalStorage;
}
const defaultOptions: HostedStorageOptions = {
@ -81,14 +84,22 @@ export class HostedStorageManager implements IDocStorageManager {
// Maps docId to the promise for when the document is present on the local filesystem.
private _localFiles = new Map<string, Promise<boolean>>();
// Label to put in metadata for a document. Only one label supported per snapshot currently.
// Holds the label that should be associated with a backup when a labeled backup is being made.
private _labels = new Map<string, string>();
// Time at which document was last changed.
private _timestamps = new Map<string, string>();
// Access external storage.
private _ext: ChecksummedExternalStorage;
private _extMeta: ChecksummedExternalStorage;
// Prune external storage.
private _pruner: DocSnapshotPruner;
// If _disableS3 is set, don't actually communicate with S3 - keep everything local.
private _disableS3 = (process.env.GRIST_DISABLE_S3 === 'true');
// Access to version information about documents.
private _inventory: DocSnapshotInventory;
// A set of filenames currently being created or downloaded.
private _prepareFiles = new Set<string>();
@ -101,6 +112,9 @@ export class HostedStorageManager implements IDocStorageManager {
private _baseStore: ExternalStorage; // External store for documents, without checksumming.
// Latest checksums of documents.
private _latestVersions = new Map<string, string>();
/**
* Initialize with the given root directory, which should be a fully-resolved path.
* If s3Bucket is blank, S3 storage will be disabled.
@ -108,21 +122,19 @@ export class HostedStorageManager implements IDocStorageManager {
constructor(
private _docsRoot: string,
private _docWorkerId: string,
s3Bucket: string,
s3Prefix: string, // Should end in / if non-empty.
private _disableS3: boolean,
extraS3Prefix: string,
private _docWorkerMap: IDocWorkerMap,
dbManager: HomeDBManager,
create: ICreate,
options: HostedStorageOptions = defaultOptions
) {
if (s3Bucket === '') { this._disableS3 = true; }
// We store documents either in a test store, or in an s3 store
// at s3://<s3Bucket>/<s3Prefix><docId>.grist
const externalStore = options.testExternalStorage ||
(this._disableS3 ? undefined : create.ExternalStorage(s3Bucket, s3Prefix));
if (!externalStore) { this._disableS3 = true; }
const externalStoreDoc = options.testExternalStorageDoc ||
(this._disableS3 ? undefined : create.ExternalStorage('doc', extraS3Prefix));
if (!externalStoreDoc) { this._disableS3 = true; }
const secondsBeforePush = options.secondsBeforePush;
const secondsBeforeFirstRetry = options.secondsBeforeFirstRetry;
if (options.pushDocUpdateTimes) {
this._metadataManager = new HostedMetadataManager(dbManager);
}
@ -135,44 +147,30 @@ export class HostedStorageManager implements IDocStorageManager {
});
if (!this._disableS3) {
this._baseStore = externalStore!;
this._baseStore = externalStoreDoc!;
// Whichever store we have, we use checksums to deal with
// eventual consistency.
const versions = new Map<string, string>();
this._ext = new ChecksummedExternalStorage(this._baseStore, {
maxRetries: 4,
initialDelayMs: secondsBeforeFirstRetry * 1000,
computeFileHash: this._getHash.bind(this),
sharedHash: {
save: async (key, checksum) => {
await this._docWorkerMap.updateDocStatus(key, checksum);
},
load: async (key) => {
const docStatus = await this._docWorkerMap.getDocWorker(key);
return docStatus && docStatus.docMD5 || null;
}
},
localHash: {
save: async (key, checksum) => {
const fname = this._getHashFile(this.getPath(key));
await fse.writeFile(fname, checksum);
},
load: async (key) => {
const fname = this._getHashFile(this.getPath(key));
if (!await fse.pathExists(fname)) { return null; }
return await fse.readFile(fname, 'utf8');
}
},
latestVersion: {
save: async (key, ver) => {
versions.set(key, ver);
},
load: async (key) => versions.get(key) || null
}
});
this._ext = this._getChecksummedExternalStorage('doc', this._baseStore,
this._latestVersions, options);
const baseStoreMeta = options.testExternalStorageMeta || create.ExternalStorage('meta', extraS3Prefix);
if (!baseStoreMeta) {
throw new Error('bug: external storage should be created for "meta" if it is created for "doc"');
}
this._extMeta = this._getChecksummedExternalStorage('meta', baseStoreMeta,
new Map(),
options);
this._inventory = new DocSnapshotInventory(this._ext, this._extMeta,
async docId => {
const dir = this.getAssetPath(docId);
await fse.mkdirp(dir);
return path.join(dir, 'meta.json');
});
// The pruner could use an inconsistent store without any real loss overall,
// but tests are easier if it is consistent.
this._pruner = new DocSnapshotPruner(this._ext, {
this._pruner = new DocSnapshotPruner(this._inventory, {
delayBeforeOperationMs: 0, // prune as soon as we've made a first upload.
minDelayBetweenOperationsMs: secondsBeforePush * 4000, // ... but wait awhile before
// pruning again.
@ -190,9 +188,14 @@ export class HostedStorageManager implements IDocStorageManager {
}
public getPath(docName: string): string {
// docName should just be a docId; we use basename to protect against some possible hack attempts.
return this.getAssetPath(docName) + '.grist';
}
// Where to store files related to a document locally. Document goes in <assetPath>.grist,
// and other files go in <assetPath>/ directory.
public getAssetPath(docName: string): string {
checkValidDocId(docName);
return path.join(this._docsRoot, `${path.basename(docName, '.grist')}.grist`);
return path.join(this._docsRoot, path.basename(docName, '.grist'));
}
// We don't deal with sample docs
@ -285,8 +288,7 @@ export class HostedStorageManager implements IDocStorageManager {
// Make sure the new content is considered new.
// NOTE: fse.remove succeeds also when the file does not exist.
await fse.remove(this._getHashFile(this.getPath(docId)));
this.markAsChanged(docId);
this.markAsEdited(docId);
this.markAsChanged(docId, 'edit');
} catch (err) {
log.error("HostedStorageManager: problem replacing %s: %s", docId, err);
await fse.move(tmpPath, docPath, {overwrite: true});
@ -309,10 +311,13 @@ export class HostedStorageManager implements IDocStorageManager {
await this.closeDocument(docName);
if (!this._disableS3) {
await this._ext.remove(docName);
await this._extMeta.remove(docName);
}
// NOTE: fse.remove succeeds also when the file does not exist.
await fse.remove(this.getPath(docName));
await fse.remove(this._getHashFile(this.getPath(docName)));
await fse.remove(this._getHashFile(this.getPath(docName), 'doc'));
await fse.remove(this._getHashFile(this.getPath(docName), 'meta'));
await fse.remove(this.getAssetPath(docName));
}
// We don't implement document renames.
@ -322,12 +327,20 @@ export class HostedStorageManager implements IDocStorageManager {
/**
* We handle backups by syncing the current version of the file as a new object version in S3,
* with the requested backupTag as an S3 tag.
* with the requested backupTag as metadata.
*/
public async makeBackup(docName: string, backupTag: string): Promise<string> {
// TODO Must implement backups: currently this will prevent open docs that need migration.
// TODO: This method isn't used by SQLiteDB when migrating DB versions, but probably should be.
return "I_totally_did_not_back_up_your_document_sorry_not_sorry";
if (this._labels.get(docName)) {
await this.flushDoc(docName);
}
this._labels.set(docName, backupTag);
this.markAsChanged(docName);
await this.flushDoc(docName);
// TODO: make an alternative way to store backups if operating without an external
// store.
return this._ext ?
(this._ext.url(docName) + ' (' + this._latestVersions.get(docName) + ')') :
'no-external-storage-enabled';
}
/**
@ -411,24 +424,23 @@ export class HostedStorageManager implements IDocStorageManager {
/**
* This is called when a document may have been changed, via edits or migrations etc.
*/
public markAsChanged(docName: string): void {
if (parseUrlId(docName).snapshotId) { return; }
if (this._localFiles.has(docName)) {
// Make sure the file is marked as locally present (it may be newly created).
this._localFiles.set(docName, Promise.resolve(true));
public markAsChanged(docName: string, reason?: string): void {
const timestamp = new Date().toISOString();
this._timestamps.set(docName, timestamp);
try {
if (parseUrlId(docName).snapshotId) { return; }
if (this._localFiles.has(docName)) {
// Make sure the file is marked as locally present (it may be newly created).
this._localFiles.set(docName, Promise.resolve(true));
}
if (this._disableS3) { return; }
if (this._closed) { throw new Error("HostedStorageManager.markAsChanged called after closing"); }
this._uploads.addOperation(docName);
} finally {
if (reason === 'edit') {
this._markAsEdited(docName, timestamp);
}
}
if (this._disableS3) { return; }
if (this._closed) { throw new Error("HostedStorageManager.markAsChanged called after closing"); }
this._uploads.addOperation(docName);
}
/**
* This is called when a document was edited by the user.
*/
public markAsEdited(docName: string): void {
if (parseUrlId(docName).snapshotId) { return; }
// Schedule a metadata update for the modified doc.
if (this._metadataManager) { this._metadataManager.scheduleUpdate(docName); }
}
/**
@ -443,23 +455,34 @@ export class HostedStorageManager implements IDocStorageManager {
return {
snapshots: [{
snapshotId: 'current',
lastModified: new Date(),
lastModified: new Date().toISOString(),
docId: docName,
}]
};
}
const versions = await this._ext.versions(docName);
const versions = await this._inventory.versions(docName,
this._latestVersions.get(docName) || null);
const parts = parseUrlId(docName);
return {
snapshots: versions
.map(v => ({
lastModified: v.lastModified,
snapshotId: v.snapshotId,
docId: buildUrlId({...parts, snapshotId: v.snapshotId}),
}))
.map(v => {
return {
...v,
docId: buildUrlId({...parts, snapshotId: v.snapshotId}),
};
})
};
}
/**
* This is called when a document was edited by the user.
*/
private _markAsEdited(docName: string, timestamp: string): void {
if (parseUrlId(docName).snapshotId) { return; }
// Schedule a metadata update for the modified doc.
if (this._metadataManager) { this._metadataManager.scheduleUpdate(docName, timestamp); }
}
/**
* Makes sure a document is present locally, fetching it from S3 if necessary.
* Returns true on success, false if document not found. It is safe to call
@ -574,8 +597,8 @@ export class HostedStorageManager implements IDocStorageManager {
/**
* We'll save hashes in a file with the suffix -hash.
*/
private _getHashFile(docPath: string): string {
return docPath + "-hash";
private _getHashFile(docPath: string, family: string = 'doc'): string {
return docPath + `-hash-${family}`;
}
/**
@ -601,14 +624,101 @@ export class HostedStorageManager implements IDocStorageManager {
throw new Error('too soon to consider pushing');
}
tmpPath = await this._prepareBackup(docId);
await this._ext.upload(docId, tmpPath);
this._pruner.requestPrune(docId);
const docMetadata = await this._getDocMetadata(tmpPath);
const label = this._labels.get(docId);
const t = this._timestamps.get(docId) || new Date().toISOString();
this._labels.delete(docId);
// Keep metadata keys simple, short, and lowercase.
const metadata = {
...docMetadata,
...label && {label},
t,
};
const prevSnapshotId = this._latestVersions.get(docId) || null;
const newSnapshotId = await this._ext.upload(docId, tmpPath, metadata);
if (!newSnapshotId) {
// This is unexpected.
throw new Error('No snapshotId allocated after upload');
}
const snapshot = {
lastModified: t,
snapshotId: newSnapshotId,
metadata
}
await this._inventory.add(docId, snapshot, prevSnapshotId);
const scheduled = this._pruner.requestPrune(docId);
if (!scheduled) {
await this._inventory.flush(docId);
}
} finally {
// Clean up backup.
// NOTE: fse.remove succeeds also when the file does not exist.
if (tmpPath) { await fse.remove(tmpPath); }
}
}
// Extract actionHash, actionNum, and timezone from a document backup.
private async _getDocMetadata(fname: string): Promise<{[key: string]: string}> {
const result: Record<string, string> = {};
const db = await SQLiteDB.openDBRaw(fname, OpenMode.OPEN_READONLY);
try {
const actionQuery = await db.get('select actionHash, actionNum from _gristsys_ActionHistoryBranch as b ' +
'left join _gristsys_ActionHistory as h on h.id = b.actionRef ' +
'where b.name = ?', 'shared');
const h = actionQuery?.actionHash;
if (h) { result.h = h; }
const n = actionQuery?.actionNum;
if (n) { result.n = String(n); }
} catch (e) {
// Tolerate files that don't have _gristsys_* yet (although we don't need to).
}
try {
const tzQuery = await db.get('select timezone from _grist_DocInfo where id = 1');
const tz = tzQuery?.timezone;
if (tz) { result.tz = tz; }
} catch (e) {
// Tolerate files that don't have _grist_DocInfo yet.
}
await db.close();
return result;
}
// Wrap external storage in a checksum-aware decorator this will retry until
// consistency.
private _getChecksummedExternalStorage(family: string, core: ExternalStorage,
versions: Map<string, string>,
options: HostedStorageOptions) {
return new ChecksummedExternalStorage(core, {
maxRetries: 4,
initialDelayMs: options.secondsBeforeFirstRetry * 1000,
computeFileHash: this._getHash.bind(this),
sharedHash: {
save: async (key, checksum) => {
await this._docWorkerMap.updateChecksum(family, key, checksum);
},
load: async (key) => {
return await this._docWorkerMap.getChecksum(family, key);
}
},
localHash: {
save: async (key, checksum) => {
const fname = this._getHashFile(this.getPath(key), family);
await fse.writeFile(fname, checksum);
},
load: async (key) => {
const fname = this._getHashFile(this.getPath(key), family);
if (!await fse.pathExists(fname)) { return null; }
return await fse.readFile(fname, 'utf8');
}
},
latestVersion: {
save: async (key, ver) => {
versions.set(key, ver);
},
load: async (key) => versions.get(key) || null
}
});
}
}

@ -0,0 +1,8 @@
/**
* Interface for storing checksums. Family is a short string, to allow storing
* checksums for different namespaces.
*/
export interface IChecksumStore {
updateChecksum(family: string, key: string, checksum: string): Promise<void>;
getChecksum(family: string, key: string): Promise<string|null>;
}

@ -21,11 +21,20 @@ export interface ICreate {
Billing(dbManager: HomeDBManager): IBilling;
Notifier(dbManager: HomeDBManager, gristConfig: GristLoadConfig): INotifier;
Shell(): IShell|undefined;
ExternalStorage(bucket: string, prefix: string): ExternalStorage|undefined;
// Create a space to store files externally, for storing either:
// - documents. This store should be versioned, and can be eventually consistent.
// - meta. This store need not be versioned, and can be eventually consistent.
// For test purposes an extra prefix may be supplied. Stores with different prefixes
// should not interfere with each other.
ExternalStorage(purpose: 'doc' | 'meta', testExtraPrefix: string): ExternalStorage|undefined;
ActiveDoc(docManager: DocManager, docName: string): ActiveDoc;
DocManager(storageManager: IDocStorageManager, pluginManager: PluginManager,
homeDbManager: HomeDBManager|null, gristServer: GristServer): DocManager;
NSandbox(options: ISandboxCreationOptions): ISandbox;
sessionSecret(): string;
// Get configuration information to show at start-up.
configurationOptions(): {[key: string]: any};
}

@ -1,7 +1,7 @@
import {DocEntry} from 'app/common/DocListAPI';
import {DocSnapshots} from 'app/common/DocSnapshot';
import {DocReplacementOptions} from 'app/common/UserAPI';
import {OptDocSession} from 'app/server/lib/DocSession';
import {DocSnapshots} from 'app/server/lib/DocSnapshots';
export interface IDocStorageManager {
getPath(docName: string): string;
@ -20,9 +20,9 @@ export interface IDocStorageManager {
showItemInFolder(docName: string): Promise<void>;
closeStorage(): Promise<void>;
closeDocument(docName: string): Promise<void>;
markAsChanged(docName: string): void; // document needs a backup (edits, migrations, etc)
markAsEdited(docName: string): void; // document was edited by a user
// Mark document as needing a backup (due to edits, migrations, etc).
// If reason is set to 'edit' the user-facing timestamp on the document should be updated.
markAsChanged(docName: string, reason?: 'edit'): void;
testReopenStorage(): void; // restart storage during tests
addToStorage(docName: string): void; // add a new local document to storage
prepareToCloseStorage(): void; // speed up sync with remote store

@ -118,6 +118,14 @@ export enum OpenMode {
CREATE_EXCL, // Create new DB or fail if it already exists.
}
/**
* Callbacks to use if a migration is run, so that backups are made.
*/
export interface MigrationHooks {
beforeMigration?(currentVersion: number, newVersion: number): Promise<void>;
afterMigration?(newVersion: number, success: boolean): Promise<void>;
}
/**
* An interface implemented both by SQLiteDB and DocStorage (by forwarding). Methods
* documented in SQLiteDB.
@ -150,7 +158,8 @@ export class SQLiteDB {
* We report the migration error, and expose it via .migrationError property.
*/
public static async openDB(dbPath: string, schemaInfo: SchemaInfo,
mode: OpenMode = OpenMode.OPEN_CREATE): Promise<SQLiteDB> {
mode: OpenMode = OpenMode.OPEN_CREATE,
hooks: MigrationHooks = {}): Promise<SQLiteDB> {
const db = await SQLiteDB.openDBRaw(dbPath, mode);
const userVersion: number = await db.getMigrationVersion();
@ -170,7 +179,7 @@ export class SQLiteDB {
}
} else {
try {
db._migrationBackupPath = await db._migrate(userVersion, schemaInfo);
db._migrationBackupPath = await db._migrate(userVersion, schemaInfo, hooks);
} catch (err) {
db._migrationError = err;
}
@ -447,9 +456,11 @@ export class SQLiteDB {
* If migration succeeded, it leaves a backup file and returns its path. If no migration was
* needed, returns null. If migration failed, leaves DB unchanged and throws Error.
*/
private async _migrate(actualVer: number, schemaInfo: SchemaInfo): Promise<string|null> {
private async _migrate(actualVer: number, schemaInfo: SchemaInfo,
hooks: MigrationHooks): Promise<string|null> {
const targetVer: number = schemaInfo.migrations.length;
let backupPath: string|null = null;
let success: boolean = false;
if (actualVer > targetVer) {
log.warn("SQLiteDB[%s]: DB is at version %s ahead of target version %s",
@ -459,6 +470,7 @@ export class SQLiteDB {
this._dbPath, actualVer, targetVer);
const versions = range(actualVer, targetVer);
backupPath = await createBackupFile(this._dbPath, actualVer);
await hooks.beforeMigration?.(actualVer, targetVer);
try {
await this.execTransaction(async () => {
for (const versionNum of versions) {
@ -466,6 +478,7 @@ export class SQLiteDB {
}
await this.exec(`PRAGMA user_version = ${targetVer}`);
});
success = true;
// After a migration, reduce the sqlite file size. This must be run outside a transaction.
await this.run("VACUUM");
@ -480,6 +493,8 @@ export class SQLiteDB {
this._dbPath, actualVer, targetVer, err);
err.message = `SQLiteDB[${this._dbPath}] migration to ${targetVer} failed: ${err.message}`;
throw err;
} finally {
await hooks.afterMigration?.(targetVer, success);
}
}
return backupPath;

@ -56,6 +56,7 @@
"@gristlabs/pidusage": "2.0.17",
"@gristlabs/sqlite3": "4.0.6-grist.1",
"@popperjs/core": "2.3.3",
"async-mutex": "0.2.4",
"axios": "0.18.0",
"backbone": "1.3.3",
"bootstrap": "3.3.5",

@ -41,5 +41,8 @@ export const create: ICreate = {
sessionSecret() {
return process.env.GRIST_SESSION_SECRET ||
'Phoo2ag1jaiz6Moo2Iese2xoaphahbai3oNg7diemohlah0ohtae9iengafieS2Hae7quungoCi9iaPh';
},
configurationOptions() {
return {};
}
};

Loading…
Cancel
Save