(core) Persist forks in home db

Summary:
Adds information about forks to the home db. This will be used
later by the UI to list forks of documents.

Test Plan: Browser and server tests.

Reviewers: paulfitz

Reviewed By: paulfitz

Differential Revision: https://phab.getgrist.com/D3772
pull/370/head
George Gevoian 1 year ago
parent 3aba7f6208
commit 1ac4931c22

@ -169,6 +169,7 @@ export interface QueryResult extends TableFetchResult {
* docId of XXXXX~FORKID[~USERID] and a urlId of UUUUU~FORKID[~USERID].
*/
export interface ForkResult {
forkId: string;
docId: string;
urlId: string;
}

@ -1,5 +1,5 @@
import {ActionSummary} from 'app/common/ActionSummary';
import {ApplyUAResult, PermissionDataWithExtraUsers, QueryFilters} from 'app/common/ActiveDocAPI';
import {ApplyUAResult, ForkResult, PermissionDataWithExtraUsers, QueryFilters} from 'app/common/ActiveDocAPI';
import {BaseAPI, IOptions} from 'app/common/BaseAPI';
import {BillingAPI, BillingAPIImpl} from 'app/common/BillingAPI';
import {BrowserSettings} from 'app/common/BrowserSettings';
@ -108,6 +108,8 @@ export interface Workspace extends WorkspaceProperties {
isSupportWorkspace?: boolean;
}
export type DocumentType = 'tutorial';
// Non-core options for a document.
// "Non-core" means bundled into a single options column in the database.
// TODO: consider smoothing over this distinction in the API.
@ -120,6 +122,8 @@ export interface DocumentOptions {
export interface DocumentProperties extends CommonProperties {
isPinned: boolean;
urlId: string|null;
trunkId: string|null;
type: DocumentType|null;
options: DocumentOptions|null;
}
@ -130,6 +134,13 @@ export interface Document extends DocumentProperties {
workspace: Workspace;
access: roles.Role;
trunkAccess?: roles.Role|null;
forks?: Fork[];
}
export interface Fork {
id: string;
trunkId: string;
updatedAt: string; // ISO date string
}
// Non-core options for a user.
@ -385,6 +396,7 @@ export interface DocAPI {
updateRows(tableId: string, changes: TableColValues): Promise<number[]>;
addRows(tableId: string, additions: BulkColValues): Promise<number[]>;
removeRows(tableId: string, removals: number[]): Promise<number[]>;
fork(): Promise<ForkResult>;
replace(source: DocReplacementOptions): Promise<void>;
// Get list of document versions (specify raw to bypass caching, which should only make
// a difference if snapshots have "leaked")
@ -837,6 +849,12 @@ export class DocAPIImpl extends BaseAPI implements DocAPI {
});
}
public async fork(): Promise<ForkResult> {
return this.requestJson(`${this._url}/fork`, {
method: 'POST'
});
}
public async replace(source: DocReplacementOptions): Promise<void> {
return this.requestJson(`${this._url}/replace`, {
body: JSON.stringify(source),

@ -1,7 +1,8 @@
import {ApiError} from 'app/common/ApiError';
import {DocumentUsage} from 'app/common/DocUsage';
import {Role} from 'app/common/roles';
import {DocumentOptions, DocumentProperties, documentPropertyKeys, NEW_DOCUMENT_CODE} from "app/common/UserAPI";
import {DocumentOptions, DocumentProperties, documentPropertyKeys,
DocumentType, NEW_DOCUMENT_CODE} from "app/common/UserAPI";
import {nativeValues} from 'app/gen-server/lib/values';
import {Column, Entity, JoinColumn, ManyToOne, OneToMany, PrimaryColumn} from "typeorm";
import {AclRuleDoc} from "./AclRule";
@ -69,6 +70,25 @@ export class Document extends Resource {
@Column({name: 'usage', type: nativeValues.jsonEntityType, nullable: true})
public usage: DocumentUsage | null;
@Column({name: 'created_by', type: 'integer', nullable: true})
public createdBy: number|null;
@Column({name: 'trunk_id', type: 'text', nullable: true})
public trunkId: string|null;
// Property set for forks, containing the URL ID of the trunk.
public trunkUrlId?: string|null;
@ManyToOne(_type => Document, document => document.forks)
@JoinColumn({name: 'trunk_id'})
public trunk: Document|null;
@OneToMany(_type => Document, document => document.trunk)
public forks: Document[];
@Column({name: 'type', type: 'text', nullable: true})
public type: DocumentType|null;
public checkProperties(props: any): props is Partial<DocumentProperties> {
return super.checkProperties(props, documentPropertyKeys);
}
@ -82,6 +102,7 @@ export class Document extends Resource {
}
this.urlId = props.urlId;
}
if (props.type !== undefined) { this.type = props.type; }
if (props.options !== undefined) {
// Options are merged over the existing state - unless options
// object is set to "null", in which case the state is wiped

@ -1213,6 +1213,9 @@ export class HomeDBManager extends EventEmitter {
(doc.workspace as any).owner = doc.workspace.org.owner;
}
if (forkId || snapshotId) {
doc.trunkId = doc.id;
doc.trunkUrlId = doc.urlId;
// Fix up our reply to be correct for the fork, rather than the trunk.
// The "id" and "urlId" fields need updating.
doc.id = buildUrlId({trunkId: doc.id, forkId, forkUserId, snapshotId});
@ -1294,6 +1297,20 @@ export class HomeDBManager extends EventEmitter {
return this._single(await this._verifyAclPermissions(qb));
}
/**
* Gets a list of all forks whose trunk is `docId`.
*
* NOTE: This is not a part of the API. It should only be called by the DocApi when
* deleting a document.
*/
public async getDocForks(docId: string): Promise<Document[]> {
return this._connection.createQueryBuilder()
.select('forks')
.from(Document, 'forks')
.where('forks.trunk_id = :docId', {docId})
.getMany();
}
/**
*
* Adds an org with the given name. Returns a query result with the id of the added org.
@ -1778,6 +1795,7 @@ export class HomeDBManager extends EventEmitter {
doc.aliases = [];
}
doc.workspace = workspace;
doc.createdBy = scope.userId;
// Create the special initial permission groups for the new workspace.
const groupMap = this._createGroups(workspace, scope.userId);
doc.aclRules = this.defaultCommonGroups.map(_grpDesc => {
@ -1872,7 +1890,7 @@ export class HomeDBManager extends EventEmitter {
const queryResult = await verifyIsPermitted(docQuery);
if (queryResult.status !== 200) {
// If the query for the workspace failed, return the failure result.
// If the query for the doc failed, return the failure result.
return queryResult;
}
// Update the name and save.
@ -1945,6 +1963,30 @@ export class HomeDBManager extends EventEmitter {
return this._setDocumentRemovedAt(scope, null);
}
/**
* Like `deleteDocument`, but for deleting a fork.
*
* NOTE: This is not a part of the API. It should only be called by the DocApi when
* deleting a fork.
*/
public async deleteFork(scope: DocScope): Promise<QueryResult<number>> {
return await this._connection.transaction(async manager => {
const forkQuery = this._doc(scope, {
manager,
allowSpecialPermit: true
});
const result = await forkQuery.getRawAndEntities();
if (result.entities.length === 0) {
return {
status: 404,
errMessage: 'fork not found'
};
}
await manager.remove(result.entities[0]);
return {status: 200};
});
}
// Fetches and provides a callback with the billingAccount so it may be updated within
// a transaction. The billingAccount is saved after any changes applied in the callback.
// Will throw an error if the user does not have access to the org's billingAccount.
@ -2540,6 +2582,31 @@ export class HomeDBManager extends EventEmitter {
});
}
/**
* Creates a fork of `doc`, using the specified `forkId`.
*
* NOTE: This is not a part of the API. It should only be called by the ActiveDoc when
* a new fork is initiated.
*/
public async forkDoc(
userId: number,
doc: Document,
forkId: string,
): Promise<QueryResult<string>> {
return await this._connection.transaction(async manager => {
const fork = new Document();
fork.id = forkId;
fork.name = doc.name;
fork.createdBy = userId;
fork.trunkId = doc.trunkId || doc.id;
const result = await manager.save([fork]);
return {
status: 200,
data: result[0].id,
};
});
}
/**
* Updates the updatedAt and usage values for several docs. Takes a map where each entry maps
* a docId to a metadata object containing the updatedAt and/or usage values. This is not a part
@ -2803,6 +2870,9 @@ export class HomeDBManager extends EventEmitter {
let query = this.org(scope, org, options)
.leftJoinAndSelect('orgs.workspaces', 'workspaces')
.leftJoinAndSelect('workspaces.docs', 'docs', this._onDoc(scope))
.leftJoin('docs.forks', 'forks', this._onFork())
.addSelect(['forks.id', 'forks.trunkId', 'forks.createdBy', 'forks.updatedAt'])
.setParameter('anonId', this.getAnonymousUserId())
.leftJoin('orgs.billingAccount', 'account')
.leftJoin('account.product', 'product')
.addSelect('product.features')
@ -3421,6 +3491,13 @@ export class HomeDBManager extends EventEmitter {
}
}
/**
* Like _onDoc, but for joining forks.
*/
private _onFork() {
return 'forks.created_by = :userId AND forks.created_by <> :anonId';
}
/**
* Construct a QueryBuilder for a select query on a specific workspace given by
* wsId. Provides options for running in a transaction and adding permission info.

@ -1,4 +1,5 @@
import { ApiError } from 'app/common/ApiError';
import { buildUrlId } from 'app/common/gristUrls';
import { Document } from 'app/gen-server/entity/Document';
import { Workspace } from 'app/gen-server/entity/Workspace';
import { HomeDBManager, Scope } from 'app/gen-server/lib/HomeDBManager';
@ -119,6 +120,29 @@ export class Housekeeper {
};
await this._dbManager.deleteWorkspace(scope, workspace.id);
}
// Delete old forks
const forks = await this._getForksToDelete();
for (const fork of forks) {
const docId = buildUrlId({trunkId: fork.trunkId!, forkId: fork.id, forkUserId: fork.createdBy!});
const permitKey = await this._permitStore.setPermit({docId});
try {
const result = await fetch(
await this._server.getHomeUrlByDocId(docId, `/api/docs/${docId}`),
{
method: 'DELETE',
headers: {
Permit: permitKey,
},
}
);
if (result.status !== 200) {
log.error(`failed to delete fork ${docId}: error status ${result.status}`);
}
} finally {
await this._permitStore.removePermit(permitKey);
}
}
}
public addEndpoints(app: express.Application) {
@ -202,7 +226,7 @@ export class Housekeeper {
}
private async _getWorkspacesToDelete() {
const docs = await this._dbManager.connection.createQueryBuilder()
const workspaces = await this._dbManager.connection.createQueryBuilder()
.select('workspaces')
.from(Workspace, 'workspaces')
.leftJoin('workspaces.docs', 'docs')
@ -212,7 +236,17 @@ export class Housekeeper {
// wait for workspace to be empty
.andWhere('docs.id IS NULL')
.getMany();
return docs;
return workspaces;
}
private async _getForksToDelete() {
const forks = await this._dbManager.connection.createQueryBuilder()
.select('forks')
.from(Document, 'forks')
.where('forks.trunk_id IS NOT NULL')
.andWhere(`forks.updated_at <= ${this._getThreshold()}`)
.getMany();
return forks;
}
/**

@ -0,0 +1,42 @@
import {MigrationInterface, QueryRunner, TableColumn, TableForeignKey} from "typeorm";
export class Forks1673051005072 implements MigrationInterface {
public async up(queryRunner: QueryRunner): Promise<void> {
await queryRunner.addColumns("docs", [
new TableColumn({
name: "created_by",
type: "integer",
isNullable: true,
}),
new TableColumn({
name: "trunk_id",
type: "text",
isNullable: true,
}),
new TableColumn({
name: "type",
type: "text",
isNullable: true,
}),
]);
await queryRunner.createForeignKeys("docs", [
new TableForeignKey({
columnNames: ["created_by"],
referencedTableName: "users",
referencedColumnNames: ["id"],
onDelete: "CASCADE",
}),
new TableForeignKey({
columnNames: ["trunk_id"],
referencedTableName: "docs",
referencedColumnNames: ["id"],
onDelete: "CASCADE",
}),
]);
}
public async down(queryRunner: QueryRunner): Promise<void> {
await queryRunner.dropColumns("docs", ["created_by", "trunk_id", "type"]);
}
}

@ -78,6 +78,7 @@ import {DocReplacementOptions, DocState, DocStateComparison} from 'app/common/Us
import {convertFromColumn} from 'app/common/ValueConverter';
import {guessColInfoWithDocData} from 'app/common/ValueGuesser';
import {parseUserAction} from 'app/common/ValueParser';
import {Document} from 'app/gen-server/entity/Document';
import {ParseOptions} from 'app/plugin/FileParserAPI';
import {AccessTokenOptions, AccessTokenResult, GristDocAPI} from 'app/plugin/GristAPI';
import {compileAclFormula} from 'app/server/lib/ACLFormula';
@ -1375,10 +1376,16 @@ export class ActiveDoc extends EventEmitter {
}
/**
* Fork the current document. In fact, all that requires is calculating a good
* ID for the fork. TODO: reconcile the two ways there are now of preparing a fork.
* Fork the current document.
*
* TODO: reconcile the two ways there are now of preparing a fork.
*/
public async fork(docSession: OptDocSession): Promise<ForkResult> {
const dbManager = this.getHomeDbManager();
if (!dbManager) {
throw new Error('HomeDbManager not available');
}
const user = getDocSessionUser(docSession);
// For now, fork only if user can read everything (or is owner).
// TODO: allow forks with partial content.
@ -1387,9 +1394,19 @@ export class ActiveDoc extends EventEmitter {
}
const userId = user.id;
const isAnonymous = this._docManager.isAnonymous(userId);
// Get fresh document metadata (the cached metadata doesn't include the urlId).
const doc = await docSession.authorizer?.getDoc();
if (!doc) { throw new Error('document id not known'); }
let doc: Document | undefined;
if (docSession.authorizer) {
doc = await docSession.authorizer.getDoc();
} else if (docSession.req) {
doc = await this.getHomeDbManager()?.getDoc(docSession.req);
}
if (!doc) { throw new Error('Document not found'); }
// Don't allow creating forks of forks (for now).
if (doc.trunkId) { throw new ApiError("Cannot fork a document that's already a fork", 400); }
const trunkDocId = doc.id;
const trunkUrlId = doc.urlId || doc.id;
await this.flushDoc(); // Make sure fork won't be too out of date.
@ -1415,6 +1432,8 @@ export class ActiveDoc extends EventEmitter {
if (resp.status !== 200) {
throw new ApiError(resp.statusText, resp.status);
}
await dbManager.forkDoc(userId, doc, forkIds.forkId);
} finally {
await permitStore.removePermit(permitKey);
}

@ -3,13 +3,13 @@ import {ApiError} from 'app/common/ApiError';
import {BrowserSettings} from "app/common/BrowserSettings";
import {BulkColValues, ColValues, fromTableDataAction, TableColValues, TableRecordValue} from 'app/common/DocActions';
import {isRaisedException} from "app/common/gristTypes";
import {parseUrlId} from "app/common/gristUrls";
import {buildUrlId, parseUrlId} from "app/common/gristUrls";
import {isAffirmative} from "app/common/gutil";
import {SortFunc} from 'app/common/SortFunc';
import {Sort} from 'app/common/SortSpec';
import {MetaRowRecord} from 'app/common/TableData';
import {DocReplacementOptions, DocState, DocStateComparison, DocStates, NEW_DOCUMENT_CODE} from 'app/common/UserAPI';
import {HomeDBManager, makeDocAuthResult} from 'app/gen-server/lib/HomeDBManager';
import {HomeDBManager, makeDocAuthResult, QueryResult} from 'app/gen-server/lib/HomeDBManager';
import * as Types from "app/plugin/DocApiTypes";
import DocApiTypesTI from "app/plugin/DocApiTypes-ti";
import GristDataTI from 'app/plugin/GristData-ti';
@ -1181,12 +1181,26 @@ export class DocWorkerApi {
const scope = getDocScope(req);
const docId = getDocId(req);
if (permanent) {
// Soft delete the doc first, to de-list the document.
await this._dbManager.softDeleteDocument(scope);
// Delete document content from storage.
await this._docManager.deleteDoc(null, docId, true);
const {forkId} = parseUrlId(docId);
if (!forkId) {
// Soft delete the doc first, to de-list the document.
await this._dbManager.softDeleteDocument(scope);
}
// Delete document content from storage. Include forks if doc is a trunk.
const forks = forkId ? [] : await this._dbManager.getDocForks(docId);
const docsToDelete = [
docId,
...forks.map((fork) =>
buildUrlId({forkId: fork.id, forkUserId: fork.createdBy!, trunkId: docId})),
];
await Promise.all(docsToDelete.map(docName => this._docManager.deleteDoc(null, docName, true)));
// Permanently delete from database.
const query = await this._dbManager.deleteDocument(scope);
let query: QueryResult<number>;
if (forkId) {
query = await this._dbManager.deleteFork({...scope, urlId: forkId});
} else {
query = await this._dbManager.deleteDocument(scope);
}
this._dbManager.checkQueryResult(query);
await sendReply(req, res, query);
} else {

@ -556,9 +556,14 @@ export class HostedStorageManager implements IDocStorageManager {
* This is called when a document was edited by the user.
*/
private _markAsEdited(docName: string, timestamp: string): void {
if (parseUrlId(docName).snapshotId || !this._metadataManager) { return; }
if (!this._metadataManager) { return; }
const {forkId, snapshotId} = parseUrlId(docName);
if (snapshotId) { return; }
// Schedule a metadata update for the modified doc.
this._metadataManager.scheduleUpdate(docName, {updatedAt: timestamp});
const docId = forkId || docName;
this._metadataManager.scheduleUpdate(docId, {updatedAt: timestamp});
}
/**

@ -30,6 +30,7 @@ export function makeForkIds(options: { userId: number|null, isAnonymous: boolean
const docId = parseUrlId(options.trunkDocId).trunkId;
const urlId = parseUrlId(options.trunkUrlId).trunkId;
return {
forkId,
docId: buildUrlId({trunkId: docId, forkId, forkUserId}),
urlId: buildUrlId({trunkId: urlId, forkId, forkUserId}),
};

@ -22,7 +22,7 @@ export const TEST_HTTPS_OFFSET = process.env.GRIST_TEST_HTTPS_OFFSET ?
const INTERNAL_FIELDS = new Set([
'apiKey', 'billingAccountId', 'firstLoginAt', 'filteredOut', 'ownerId', 'gracePeriodStart', 'stripeCustomerId',
'stripeSubscriptionId', 'stripePlanId', 'stripeProductId', 'userId', 'isFirstTimeUser', 'allowGoogleLogin',
'authSubject', 'usage'
'authSubject', 'usage', 'createdBy'
]);
/**

@ -0,0 +1,304 @@
import {delay} from 'app/common/delay';
import {Role} from 'app/common/roles';
import {UserAPIImpl, UserProfile} from 'app/common/UserAPI';
import {AclRule, AclRuleDoc, AclRuleOrg, AclRuleWs} from 'app/gen-server/entity/AclRule';
import {BillingAccountManager} from 'app/gen-server/entity/BillingAccountManager';
import {Document} from 'app/gen-server/entity/Document';
import {Group} from 'app/gen-server/entity/Group';
import {Organization} from 'app/gen-server/entity/Organization';
import {Resource} from 'app/gen-server/entity/Resource';
import {User} from 'app/gen-server/entity/User';
import {Workspace} from 'app/gen-server/entity/Workspace';
import {SessionUserObj} from 'app/server/lib/BrowserSession';
import {getDocWorkerMap} from 'app/gen-server/lib/DocWorkerMap';
import {HomeDBManager} from 'app/gen-server/lib/HomeDBManager';
import * as docUtils from 'app/server/lib/docUtils';
import {FlexServer, FlexServerOptions} from 'app/server/lib/FlexServer';
import log from 'app/server/lib/log';
import {main as mergedServerMain, ServerType} from 'app/server/mergedServerMain';
import axios from 'axios';
import FormData from 'form-data';
import fetch from 'node-fetch';
import * as path from 'path';
import {createInitialDb, removeConnection, setUpDB} from 'test/gen-server/seed';
import {setPlan} from 'test/gen-server/testUtils';
import {fixturesRoot} from 'test/server/testUtils';
export class TestServer {
public serverUrl: string;
public server: FlexServer;
public dbManager: HomeDBManager;
public defaultSession: TestSession;
constructor(context?: Mocha.Context) {
setUpDB(context);
}
public async start(servers: ServerType[] = ["home"],
options: FlexServerOptions = {}): Promise<string> {
await createInitialDb();
this.server = await mergedServerMain(0, servers, {logToConsole: false,
externalStorage: false,
...options});
this.serverUrl = this.server.getOwnUrl();
this.dbManager = this.server.getHomeDBManager();
this.defaultSession = new TestSession(this.server);
return this.serverUrl;
}
public async stop() {
await this.server.close();
// Wait a few seconds for any late notifications to finish up.
// TypeORM doesn't give us a very clean way to shut down the db connection,
// and node-sqlite3 has become fussier about this, and in regular tests
// we substitute sqlite for postgres.
for (let i = 0; i < 30; i++) {
if (!this.server.getNotifier().testPending) { break; }
await delay(100);
}
await removeConnection();
}
// Set up a profile for the given org, and return an axios configuration to
// access the api via cookies with that profile. Leave profile null for anonymous
// access.
public async getCookieLogin(
org: string,
profile: UserProfile|null,
options: {clearCache?: boolean, sessionProps?: Partial<SessionUserObj>} = {}
) {
return this.defaultSession.getCookieLogin(org, profile, options);
}
// add named user as billing manager to org (identified by domain)
public async addBillingManager(userName: string, orgDomain: string) {
const ents = this.dbManager.connection.createEntityManager();
const org = await ents.findOne(Organization, {
relations: ['billingAccount'],
where: {domain: orgDomain}
});
const user = await ents.findOne(User, {where: {name: userName}});
const manager = new BillingAccountManager();
manager.user = user!;
manager.billingAccount = org!.billingAccount;
await manager.save();
}
// change a user's personal org to a different product (by default, one that allows anything)
public async upgradePersonalOrg(userName: string, productName: string = 'Free') {
const user = await User.findOne({where: {name: userName}});
if (!user) { throw new Error(`Could not find user ${userName}`); }
const org = await Organization.findOne({
relations: ['billingAccount', 'owner'],
where: {owner: {id: user.id}} // for some reason finding by name generates wrong SQL.
});
if (!org) { throw new Error(`Could not find personal org of ${userName}`); }
await setPlan(this.dbManager, org, productName);
}
// Get an api object for making requests for the named user with the named org.
// Careful: all api objects using cookie access will be in the same session.
public async createHomeApi(userName: string, orgDomain: string,
useApiKey: boolean = false,
checkAccess: boolean = true): Promise<UserAPIImpl> {
return this.defaultSession.createHomeApi(userName, orgDomain, useApiKey, checkAccess);
}
// Get a TestSession representing a distinct session for communicating with the server.
public newSession() {
return new TestSession(this.server);
}
/**
* Lists every resource a user is linked to via direct group
* membership. The same resource can be listed multiple times if
* the user is in multiple of its groups (e.g. viewers and guests).
* A resource the user has access to will not be listed at all if
* access is granted indirectly (e.g. a doc the user is not linked
* to via direct group membership won't be listed even if user is in
* owners group of workspace containing that doc, and the doc
* inherits access from the workspace).
*/
public async listUserMemberships(email: string): Promise<ResourceWithRole[]> {
const rules = await this.dbManager.connection.createQueryBuilder()
.select('acl_rules')
.from(AclRule, 'acl_rules')
.leftJoinAndSelect('acl_rules.group', 'groups')
.leftJoin('groups.memberUsers', 'users')
.leftJoin('users.logins', 'logins')
.where('logins.email = :email', {email})
.getMany();
return Promise.all(rules.map(this._getResourceName.bind(this)));
}
/**
* Lists every user with the specified role on the given org. Only
* roles set by direct group membership are listed, nothing indirect
* is included.
*/
public async listOrgMembership(domain: string, role: Role|null): Promise<User[]> {
return this._listMembers(role)
.leftJoin(Organization, 'orgs', 'orgs.id = acl_rules.org_id')
.andWhere('orgs.domain = :domain', {domain})
.getMany();
}
/**
* Lists every user with the specified role on the given workspace. Only
* roles set by direct group membership are listed, nothing indirect
* is included.
*/
public async listWorkspaceMembership(wsId: number, role: Role|null): Promise<User[]> {
return this._listMembers(role)
.leftJoin(Workspace, 'workspaces', 'workspaces.id = acl_rules.workspace_id')
.andWhere('workspaces.id = :wsId', {wsId})
.getMany();
}
// check that the database structure looks sane.
public async sanityCheck() {
const badGroups = await this.getBadGroupLinks();
if (badGroups.length) {
throw new Error(`badGroups: ${JSON.stringify(badGroups)}`);
}
}
// Find instances of guests and members used in inheritance.
public async getBadGroupLinks(): Promise<Group[]> {
// guests and members should never be in other groups, or have groups.
return this.dbManager.connection.createQueryBuilder()
.select('groups')
.from(Group, 'groups')
.innerJoinAndSelect('groups.memberGroups', 'memberGroups')
.where(`memberGroups.name IN ('guests', 'members')`)
.orWhere(`groups.name IN ('guests', 'members')`)
.getMany();
}
/**
* Copy a fixture doc (e.g. "Hello.grist", no path needed) and make
* it accessible with the given docId (no ".grist" extension or path).
*/
public async copyFixtureDoc(srcName: string, docId: string) {
const docsRoot = this.server.docsRoot;
const srcPath = path.resolve(fixturesRoot, 'docs', srcName);
await docUtils.copyFile(srcPath, path.resolve(docsRoot, `${docId}.grist`));
}
public getWorkStore() {
return getDocWorkerMap();
}
/**
* Looks up the resource related to an aclRule.
* TODO: rework AclRule to automate this kind of step.
*/
private async _getResourceName(aclRule: AclRule): Promise<ResourceWithRole> {
const con = this.dbManager.connection.manager;
let res: Document|Workspace|Organization|null;
if (aclRule instanceof AclRuleDoc) {
res = await con.findOne(Document, {where: {id: aclRule.docId}});
} else if (aclRule instanceof AclRuleWs) {
res = await con.findOne(Workspace, {where: {id: aclRule.workspaceId}});
} else if (aclRule instanceof AclRuleOrg) {
res = await con.findOne(Organization, {where: {id: aclRule.orgId}});
} else {
throw new Error('unknown type');
}
if (!res) { throw new Error('could not find resource'); }
return {res, role: aclRule.group.name};
}
/**
* Lists users and the groups/aclRules they are members of.
* Filters for groups of the specified name.
*/
private _listMembers(role: Role|null) {
let q = this.dbManager.connection.createQueryBuilder()
.select('users')
.from(User, 'users')
.leftJoin('users.groups', 'groups')
.leftJoin('groups.aclRule', 'acl_rules')
.leftJoinAndSelect('users.logins', 'logins');
if (role) {
q = q.andWhere('groups.name = :role', {role});
}
return q;
}
}
/**
* A distinct session. Any api objects created with this that use cookies will share
* the same session as each other, and be in a distinct session to other TestSessions.
*
* Calling createHomeApi on the server object directly results in api objects that are
* all within the same session, which is not always desirable. Api key access can be
* used to work around this, but that can also be awkward.
*/
export class TestSession {
public headers: {[key: string]: string};
constructor(public home: FlexServer) {
this.headers = {};
}
// Set up a profile for the given org, and return an axios configuration to
// access the api via cookies with that profile. Leave profile null for anonymous
// access.
public async getCookieLogin(
org: string,
profile: UserProfile|null,
{clearCache, sessionProps}: {clearCache?: boolean, sessionProps?: Partial<SessionUserObj>} = {}
) {
const resp = await axios.get(`${this.home.getOwnUrl()}/test/session`,
{validateStatus: (s => s < 400), headers: this.headers});
const cookie = this.headers.Cookie || resp.headers['set-cookie'][0];
const cid = decodeURIComponent(cookie.split('=')[1].split(';')[0]);
const sessionId = this.home.getSessions().getSessionIdFromCookie(cid);
const scopedSession = this.home.getSessions().getOrCreateSession(sessionId as string, org, '');
await scopedSession.updateUserProfile({} as any, profile);
if (sessionProps) { await scopedSession.updateUser({} as any, sessionProps); }
if (clearCache) { this.home.getSessions().clearCacheIfNeeded(); }
this.headers.Cookie = cookie;
return {
validateStatus: (status: number) => true,
headers: {
'Cookie': cookie,
'X-Requested-With': 'XMLHttpRequest',
}
};
}
// get an api object for making requests for the named user with the named org.
public async createHomeApi(userName: string, orgDomain: string,
useApiKey: boolean = false,
checkAccess: boolean = true): Promise<UserAPIImpl> {
const headers: {[key: string]: string} = {};
if (useApiKey) {
headers.Authorization = 'Bearer api_key_for_' + userName.toLowerCase();
} else {
const cookie = await this.getCookieLogin(orgDomain, {
email: `${userName.toLowerCase()}@getgrist.com`,
name: userName
});
headers.Cookie = cookie.headers.Cookie;
}
const api = new UserAPIImpl(`${this.home.getOwnUrl()}/o/${orgDomain}`, {
fetch: fetch as any,
headers,
newFormData: () => new FormData() as any,
logger: log,
});
// Make sure api is functioning, and create user if this is their first time to hit API.
if (checkAccess) { await api.getOrg('current'); }
return api;
}
}
/**
* A resource and the name of the group associated with it that the user is in.
*/
export interface ResourceWithRole {
res: Resource;
role: string;
}

@ -26,6 +26,39 @@ describe("Fork", function() {
doc = await team.tempDoc(cleanup, 'Hello.grist', {load: false});
}
async function getForks(api: UserAPI) {
const wss = await api.getOrgWorkspaces('current');
const forks = wss
.find(ws => ws.name === team.workspace)
?.docs
.find(d => d.id === doc.id)
?.forks;
return forks ?? [];
}
async function testForksOfForks(isLoggedIn: boolean = true) {
const session = isLoggedIn ? await team.login() : await team.anon.login();
await session.loadDoc(`/doc/${doc.id}/m/fork`);
await gu.getCell({rowNum: 1, col: 0}).click();
await gu.enterCell('1');
await gu.waitForServer();
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '1');
const forkUrl = await driver.getCurrentUrl();
assert.match(forkUrl, isLoggedIn ? /^[^~]*~[^~]+~\d+$/ : /^[^~]*~[^~]*$/);
await session.loadDoc((new URL(forkUrl)).pathname + '/m/fork');
await gu.getCell({rowNum: 1, col: 0}).click();
await gu.enterCell('2');
await gu.waitForServer();
await driver.findContentWait(
'.test-notifier-toast-wrapper',
/Cannot fork a document that's already a fork.*Report a problem/s,
2000
);
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '1');
assert.equal(await driver.getCurrentUrl(), `${forkUrl}/m/fork`);
await gu.wipeToasts();
}
// Run tests with both regular docId and a custom urlId in URL, to make sure
// ids are kept straight during forking.
@ -182,27 +215,7 @@ describe("Fork", function() {
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '2');
});
it('allows an anonymous fork to be forked', async function() {
const anonSession = await team.anon.login();
await anonSession.loadDoc(`/doc/${doc.id}/m/fork`);
await gu.getCell({rowNum: 1, col: 0}).click();
await gu.enterCell('1');
await gu.waitForServer();
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '1');
const fork1 = await driver.getCurrentUrl();
assert.match(fork1, /^[^~]*~[^~]*$/); // just one ~
await anonSession.loadDoc((new URL(fork1)).pathname + '/m/fork');
await gu.getCell({rowNum: 1, col: 0}).click();
await gu.enterCell('2');
await gu.waitForServer();
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '2');
const fork2 = await driver.getCurrentUrl();
assert.match(fork2, /^[^~]*~[^~]*$/); // just one ~
await anonSession.loadDoc((new URL(fork1)).pathname);
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '1');
await anonSession.loadDoc((new URL(fork2)).pathname);
assert.equal(await gu.getCell({rowNum: 1, col: 0}).getText(), '2');
});
it('does not allow an anonymous fork to be forked', () => testForksOfForks(false));
it('shows the right page item after forking', async function() {
const anonSession = await team.anon.login();
@ -236,6 +249,14 @@ describe("Fork", function() {
const forkUrl = await driver.getCurrentUrl();
assert.match(forkUrl, /~/);
// Check that the fork was saved to the home db
const api = userSession.createHomeApi();
const forks = await getForks(api);
const forkId = forkUrl.match(/\/[\w-]+~(\w+)(?:~\w)?/)?.[1];
const fork = forks.find(f => f.id === forkId);
assert.isDefined(fork);
assert.equal(fork!.trunkId, doc.id);
// Open the original url and make sure the change we made is not there
await userSession.loadDoc(`/doc/${doc.id}`);
assert.notEqual(await gu.getCell({rowNum: 1, col: 0}).value(), '123');
@ -619,6 +640,8 @@ describe("Fork", function() {
await api.updateDocPermissions(doc.id, {users: {[altSession.email]: null}});
}
});
it('does not allow a fork to be forked', testForksOfForks);
});
}
});

@ -0,0 +1,74 @@
import {UserAPI} from 'app/common/UserAPI';
import axios from 'axios';
import {assert} from 'chai';
import * as fse from 'fs-extra';
import {TestServer} from 'test/gen-server/apiUtils';
import {configForUser} from 'test/gen-server/testUtils';
import {createTmpDir} from 'test/server/docTools';
import {openClient} from 'test/server/gristClient';
import * as testUtils from 'test/server/testUtils';
const chimpy = configForUser('Chimpy');
describe('DocApi2', function() {
this.timeout(40000);
let server: TestServer;
let serverUrl: string;
let owner: UserAPI;
let wsId: number;
testUtils.setTmpLogLevel('error');
const oldEnv = new testUtils.EnvironmentSnapshot();
before(async function() {
const tmpDir = await createTmpDir();
process.env.GRIST_DATA_DIR = tmpDir;
process.env.STRIPE_ENDPOINT_SECRET = 'TEST_WITHOUT_ENDPOINT_SECRET';
// Use the TEST_REDIS_URL as the global redis url, if supplied.
if (process.env.TEST_REDIS_URL && !process.env.REDIS_URL) {
process.env.REDIS_URL = process.env.TEST_REDIS_URL;
}
server = new TestServer(this);
serverUrl = await server.start(['home', 'docs']);
const api = await server.createHomeApi('chimpy', 'docs', true);
await api.newOrg({name: 'testy', domain: 'testy'});
owner = await server.createHomeApi('chimpy', 'testy', true);
wsId = await owner.newWorkspace({name: 'ws'}, 'current');
});
after(async function() {
const api = await server.createHomeApi('chimpy', 'docs');
await api.deleteOrg('testy');
await server.stop();
oldEnv.restore();
});
describe('DELETE /docs/{did}', async () => {
it('permanently deletes a document and all of its forks', async function() {
// Create a new document and fork it twice.
const docId = await owner.newDoc({name: 'doc'}, wsId);
const session = await owner.getSessionActive();
const client = await openClient(server.server, session.user.email, session.org?.domain || 'docs');
await client.openDocOnConnect(docId);
const forkDocResponse1 = await client.send('fork', 0);
const forkDocResponse2 = await client.send('fork', 0);
// Check that files were created for the trunk and forks.
const docPath = server.server.getStorageManager().getPath(docId);
const forkPath1 = server.server.getStorageManager().getPath(forkDocResponse1.data.docId);
const forkPath2 = server.server.getStorageManager().getPath(forkDocResponse2.data.docId);
assert.equal(await fse.pathExists(docPath), true);
assert.equal(await fse.pathExists(forkPath1), true);
assert.equal(await fse.pathExists(forkPath2), true);
// Delete the trunk via API.
const deleteDocResponse = await axios.delete(`${serverUrl}/api/docs/${docId}`, chimpy);
assert.equal(deleteDocResponse.status, 200);
// Check that files for the trunk and forks were deleted.
assert.equal(await fse.pathExists(docPath), false);
assert.equal(await fse.pathExists(forkPath1), false);
assert.equal(await fse.pathExists(forkPath2), false);
});
});
});
Loading…
Cancel
Save