(core) implement cleaner row-level access control for outgoing messages

Summary:
This implements row-level access control for outgoing messages, replacing the document reloading placeholder that was there before.

 * Prior to broadcasting messages, GranularAccess is notified of actions+undo.
 * While broadcasting messages to different sessions, if we find we need row level access control information, rows before and after the change are reconstructed.
 * Messages are rewritten if rows that were previously forbidden are now allowed, and vice versa.

The diff is somewhat under-tested and under-optimized. Next step would be to implement row-level access control for incoming actions, which may result in some rejiggering of the code from this diff to avoid duplication of effort under some conditions.

Test Plan: added test

Reviewers: dsagal

Reviewed By: dsagal

Differential Revision: https://phab.getgrist.com/D2670
This commit is contained in:
Paul Fitzpatrick 2020-11-30 10:50:00 -05:00
parent c1c17bf54e
commit 0e2deecc55
8 changed files with 433 additions and 61 deletions

View File

@ -82,7 +82,7 @@ const SCHEMA_ACTIONS = new Set(['AddTable', 'RemoveTable', 'RenameTable', 'AddCo
/** /**
* Determines whether a given action is a schema action or not. * Determines whether a given action is a schema action or not.
*/ */
export function isSchemaAction(action: DocAction): boolean { export function isSchemaAction(action: DocAction): action is AddTable | RemoveTable | RenameTable | AddColumn | RemoveColumn | RenameColumn | ModifyColumn {
return SCHEMA_ACTIONS.has(action[0]); return SCHEMA_ACTIONS.has(action[0]);
} }

View File

@ -16,8 +16,14 @@ type FetchTableFunc = (tableId: string) => Promise<TableDataAction>;
export class DocData extends ActionDispatcher { export class DocData extends ActionDispatcher {
private _tables: Map<string, TableData> = new Map(); private _tables: Map<string, TableData> = new Map();
constructor(private _fetchTableFunc: FetchTableFunc, metaTableData: {[tableId: string]: TableDataAction}) { /**
* If metaTableData is not supplied, then any tables needed should be loaded manually,
* using syncTable(). All column types will be set to Any, which will affect default
* values.
*/
constructor(private _fetchTableFunc: FetchTableFunc, metaTableData: {[tableId: string]: TableDataAction} | null) {
super(); super();
if (metaTableData === null) { return; }
// Create all meta tables, and populate data we already have. // Create all meta tables, and populate data we already have.
for (const tableId in schema) { for (const tableId in schema) {
if (schema.hasOwnProperty(tableId)) { if (schema.hasOwnProperty(tableId)) {
@ -67,6 +73,17 @@ export class DocData extends ActionDispatcher {
return (!table.isLoaded || force) ? table.fetchData(this._fetchTableFunc) : Promise.resolve(); return (!table.isLoaded || force) ? table.fetchData(this._fetchTableFunc) : Promise.resolve();
} }
/**
* Fetches the data for tableId unconditionally, and without knowledge of its metadata.
* Columns will be assumed to have type 'Any'.
*/
public async syncTable(tableId: string): Promise<void> {
const tableData = await this._fetchTableFunc(tableId);
const colTypes = fromPairs(Object.keys(tableData[3]).map(c => [c, 'Any']));
colTypes.id = 'Any';
this._tables.set(tableId, this.createTableData(tableId, tableData, colTypes));
}
/** /**
* Handles an action received from the server, by forwarding it to the appropriate TableData * Handles an action received from the server, by forwarding it to the appropriate TableData
* object. * object.

View File

@ -304,6 +304,7 @@ export interface DocAPI {
getRows(tableId: string): Promise<TableColValues>; getRows(tableId: string): Promise<TableColValues>;
updateRows(tableId: string, changes: TableColValues): Promise<number[]>; updateRows(tableId: string, changes: TableColValues): Promise<number[]>;
addRows(tableId: string, additions: BulkColValues): Promise<number[]>; addRows(tableId: string, additions: BulkColValues): Promise<number[]>;
removeRows(tableId: string, removals: number[]): Promise<number[]>;
replace(source: DocReplacementOptions): Promise<void>; replace(source: DocReplacementOptions): Promise<void>;
getSnapshots(): Promise<DocSnapshots>; getSnapshots(): Promise<DocSnapshots>;
forceReload(): Promise<void>; forceReload(): Promise<void>;
@ -690,6 +691,13 @@ export class DocAPIImpl extends BaseAPI implements DocAPI {
}); });
} }
public async removeRows(tableId: string, removals: number[]): Promise<number[]> {
return this.requestJson(`${this._url}/tables/${tableId}/data/delete`, {
body: JSON.stringify(removals),
method: 'POST'
});
}
public async replace(source: DocReplacementOptions): Promise<void> { public async replace(source: DocReplacementOptions): Promise<void> {
return this.requestJson(`${this._url}/replace`, { return this.requestJson(`${this._url}/replace`, {
body: JSON.stringify(source), body: JSON.stringify(source),

View File

@ -396,7 +396,7 @@ export class ActiveDoc extends EventEmitter {
await this._actionHistory.initialize(); await this._actionHistory.initialize();
this._granularAccess = new GranularAccess(this.docData, (query) => { this._granularAccess = new GranularAccess(this.docData, (query) => {
return this.fetchQuery(makeExceptionalDocSession('system'), query, true); return this._fetchQueryFromDB(query, false);
}); });
await this._granularAccess.update(); await this._granularAccess.update();
this._sharing = new Sharing(this, this._actionHistory); this._sharing = new Sharing(this, this._actionHistory);
@ -960,6 +960,14 @@ export class ActiveDoc extends EventEmitter {
return this._docManager.makeAccessId(userId); return this._docManager.makeAccessId(userId);
} }
public async beforeBroadcast(docActions: DocAction[], undo: DocAction[]) {
await this._granularAccess.beforeBroadcast(docActions, undo);
}
public async afterBroadcast() {
await this._granularAccess.afterBroadcast();
}
/** /**
* Broadcast document changes to all the document's clients. Doesn't involve * Broadcast document changes to all the document's clients. Doesn't involve
* ActiveDoc directly, but placed here to facilitate future work on granular * ActiveDoc directly, but placed here to facilitate future work on granular
@ -1258,14 +1266,14 @@ export class ActiveDoc extends EventEmitter {
* This filters a message being broadcast to all clients to be appropriate for one * This filters a message being broadcast to all clients to be appropriate for one
* particular client, if that client may need some material filtered out. * particular client, if that client may need some material filtered out.
*/ */
private _filterDocUpdate(docSession: OptDocSession, message: { private async _filterDocUpdate(docSession: OptDocSession, message: {
actionGroup: ActionGroup, actionGroup: ActionGroup,
docActions: DocAction[] docActions: DocAction[]
}) { }) {
if (this._granularAccess.canReadEverything(docSession)) { return message; } if (this._granularAccess.canReadEverything(docSession)) { return message; }
const result = { const result = {
actionGroup: this._granularAccess.filterActionGroup(docSession, message.actionGroup), actionGroup: this._granularAccess.filterActionGroup(docSession, message.actionGroup),
docActions: this._granularAccess.filterOutgoingDocActions(docSession, message.docActions), docActions: await this._granularAccess.filterOutgoingDocActions(docSession, message.docActions),
}; };
if (result.docActions.length === 0) { return null; } if (result.docActions.length === 0) { return null; }
return result; return result;

View File

@ -77,7 +77,7 @@ export class DocClients {
*/ */
public async broadcastDocMessage(client: Client|null, type: string, messageData: any, public async broadcastDocMessage(client: Client|null, type: string, messageData: any,
filterMessage?: (docSession: OptDocSession, filterMessage?: (docSession: OptDocSession,
messageData: any) => any): Promise<void> { messageData: any) => Promise<any>): Promise<void> {
await Promise.all(this._docSessions.map(async curr => { await Promise.all(this._docSessions.map(async curr => {
const fromSelf = (curr.client === client); const fromSelf = (curr.client === client);
try { try {
@ -87,7 +87,7 @@ export class DocClients {
sendDocMessage(curr.client, curr.fd, type, messageData, fromSelf); sendDocMessage(curr.client, curr.fd, type, messageData, fromSelf);
} else { } else {
try { try {
const filteredMessageData = filterMessage(curr, messageData); const filteredMessageData = await filterMessage(curr, messageData);
if (filteredMessageData) { if (filteredMessageData) {
sendDocMessage(curr.client, curr.fd, type, filteredMessageData, fromSelf); sendDocMessage(curr.client, curr.fd, type, filteredMessageData, fromSelf);
} else { } else {

View File

@ -4,7 +4,8 @@ import { emptyPermissionSet, parsePermissions, toMixed } from 'app/common/ACLPer
import { ActionGroup } from 'app/common/ActionGroup'; import { ActionGroup } from 'app/common/ActionGroup';
import { createEmptyActionSummary } from 'app/common/ActionSummary'; import { createEmptyActionSummary } from 'app/common/ActionSummary';
import { Query } from 'app/common/ActiveDocAPI'; import { Query } from 'app/common/ActiveDocAPI';
import { BulkColValues, CellValue, ColValues, DocAction } from 'app/common/DocActions'; import { AsyncCreate } from 'app/common/AsyncCreate';
import { BulkAddRecord, BulkColValues, BulkRemoveRecord, CellValue, ColValues, DocAction, getTableId, isSchemaAction } from 'app/common/DocActions';
import { TableDataAction, UserAction } from 'app/common/DocActions'; import { TableDataAction, UserAction } from 'app/common/DocActions';
import { DocData } from 'app/common/DocData'; import { DocData } from 'app/common/DocData';
import { ErrorWithCode } from 'app/common/ErrorWithCode'; import { ErrorWithCode } from 'app/common/ErrorWithCode';
@ -14,6 +15,7 @@ import { getSetMapValue } from 'app/common/gutil';
import { canView } from 'app/common/roles'; import { canView } from 'app/common/roles';
import { compileAclFormula } from 'app/server/lib/ACLFormula'; import { compileAclFormula } from 'app/server/lib/ACLFormula';
import { getDocSessionAccess, getDocSessionUser, OptDocSession } from 'app/server/lib/DocSession'; import { getDocSessionAccess, getDocSessionUser, OptDocSession } from 'app/server/lib/DocSession';
import { getRowIdsFromDocAction, getRelatedRows } from 'app/server/lib/RowAccess';
import * as log from 'app/server/lib/log'; import * as log from 'app/server/lib/log';
import cloneDeep = require('lodash/cloneDeep'); import cloneDeep = require('lodash/cloneDeep');
import get = require('lodash/get'); import get = require('lodash/get');
@ -115,8 +117,12 @@ export class GranularAccess {
// both to be garbage-collected once docSession is no longer in use. // both to be garbage-collected once docSession is no longer in use.
private _permissionInfoMap = new WeakMap<OptDocSession, PermissionInfo>(); private _permissionInfoMap = new WeakMap<OptDocSession, PermissionInfo>();
// When broadcasting a sequence of DocAction[]s, this contains the state of
// affected rows for the relevant table before and after each DocAction. It
// may contain some unaffected rows as well.
private _rowSnapshots: AsyncCreate<Array<[TableDataAction, TableDataAction]>>|null;
public constructor(private _docData: DocData, private _fetchQuery: (query: Query) => Promise<TableDataAction>) { public constructor(private _docData: DocData, private _fetchQueryFromDB: (query: Query) => Promise<TableDataAction>) {
} }
// Return the RuleSet for "tableId:colId", or undefined if there isn't one for this column. // Return the RuleSet for "tableId:colId", or undefined if there isn't one for this column.
@ -216,12 +222,77 @@ export class GranularAccess {
return pset.read !== 'deny'; return pset.read !== 'deny';
} }
/**
* This should be called after each action bundle has been applied to the database,
* but before the actions are broadcast to clients. It will set us up to be able
* to efficiently filter those broadcasts.
*
* We expect actions bundles for a document to be applied+broadcast serially (the
* broadcasts can be parallelized, but should complete before moving on to further
* document mutation).
*/
public async beforeBroadcast(docActions: DocAction[], undo: DocAction[]) {
if (!this._haveRules) { return; }
// Prepare to compute row snapshots if it turns out we need them.
// If we never need them, they will never be computed.
this._rowSnapshots = new AsyncCreate(async () => {
// If we arrive here, the actions have been applied to the database.
// For row access work, we'll need to know the state of affected rows before and
// after the actions. One way to get that is to apply the undo actions to the
// affected part of the database.
// NOTE: the approach may need tweaking once row access control can be applied to
// incoming actions, not just outgoing ones -- in that case, it may be that some
// calculations are done earlier that could be reused here.
// First figure out what rows in which tables are touched during the undo actions.
const rows = new Map(getRelatedRows(undo));
// Populate a minimal in-memory version of the database with these rows.
const docData = new DocData(
(tableId) => this._fetchQueryFromDB({tableId, filters: {id: [...rows.get(tableId)!]}}),
null,
);
await Promise.all([...rows.keys()].map(tableId => docData.syncTable(tableId)));
// Now apply the undo actions.
for (const docAction of undo) { docData.receiveAction(docAction); }
// Now step forward, storing the before and after state for the table
// involved in each action. We'll use this to compute row access changes.
// For simple changes, the rows will be just the minimal set needed.
// This could definitely be optimized. E.g. for pure table updates, these
// states could be extracted while applying undo actions, with no need for
// a forward pass. And for a series of updates to the same table, there'll
// be duplicated before/after states that could be optimized.
const rowSnapshots = new Array<[TableDataAction, TableDataAction]>();
for (const docAction of docActions) {
const tableId = getTableId(docAction);
const tableData = docData.getTable(tableId)!;
const before = cloneDeep(tableData.getTableDataAction());
docData.receiveAction(docAction);
// If table is deleted, state afterwards doesn't matter.
const after = docData.getTable(tableId) ? cloneDeep(tableData.getTableDataAction()) : before;
rowSnapshots.push([before, after]);
}
return rowSnapshots;
});
}
/**
* This should be called once an action bundle has been broadcast to all clients.
* It will clean up any temporary state cached for filtering those broadcasts.
*/
public async afterBroadcast() {
if (this._rowSnapshots) { this._rowSnapshots.clear(); }
this._rowSnapshots = null;
}
/** /**
* Filter DocActions to be sent to a client. * Filter DocActions to be sent to a client.
*/ */
public filterOutgoingDocActions(docSession: OptDocSession, docActions: DocAction[]): DocAction[] { public async filterOutgoingDocActions(docSession: OptDocSession, docActions: DocAction[]): Promise<DocAction[]> {
return docActions.map(action => this.pruneOutgoingDocAction(docSession, action)) const actions = await Promise.all(
.filter(_docActions => _docActions !== null) as DocAction[]; docActions.map((action, idx) => this.pruneOutgoingDocAction(docSession, action, idx)));
return ([] as DocAction[]).concat(...actions);
} }
/** /**
@ -292,43 +363,21 @@ export class GranularAccess {
* Cut out any rows/columns not accessible to the user. May throw a NEED_RELOAD * Cut out any rows/columns not accessible to the user. May throw a NEED_RELOAD
* exception if the information needed to achieve the desired pruning is not available. * exception if the information needed to achieve the desired pruning is not available.
* Returns null if the action is entirely pruned. The action passed in is never modified. * Returns null if the action is entirely pruned. The action passed in is never modified.
* The idx parameter is a record of which action in the bundle this action is, and can
* be used to access information in this._rowSnapshots if needed.
*/ */
public pruneOutgoingDocAction(docSession: OptDocSession, a: DocAction): DocAction|null { public async pruneOutgoingDocAction(docSession: OptDocSession, a: DocAction, idx: number): Promise<DocAction[]> {
const tableId = a[1] as string; const tableId = getTableId(a);
const permInfo = this._getAccess(docSession); const permInfo = this._getAccess(docSession);
const tableAccess = permInfo.getTableAccess(tableId); const tableAccess = permInfo.getTableAccess(tableId);
if (tableAccess.read === 'deny') { return null; } if (tableAccess.read === 'deny') { return []; }
if (tableAccess.read === 'allow') { return a; } if (tableAccess.read === 'allow') { return [a]; }
if (tableAccess.read === 'mixedColumns') {
if (tableAccess.read === 'mixed') { return [this._pruneColumns(a, permInfo, tableId)].filter(isObject);
// For now, trigger a reload, since we don't have the
// information we need to filter rows. Reloads would be very
// annoying if user is working on something, but at least data
// won't be stale. TODO: improve!
throw new ErrorWithCode('NEED_RELOAD', 'document needs reload');
} }
// The remainder is the mixed condition.
if (a[0] === 'RemoveRecord' || a[0] === 'BulkRemoveRecord') { const revisedDocActions = await this._pruneRows(docSession, a, idx);
return a; return revisedDocActions.map(na => this._pruneColumns(na, permInfo, tableId)).filter(isObject);
} else if (a[0] === 'AddRecord' || a[0] === 'BulkAddRecord' || a[0] === 'UpdateRecord' ||
a[0] === 'BulkUpdateRecord' || a[0] === 'ReplaceTableData' || a[0] === 'TableData') {
const na = cloneDeep(a);
this._filterColumns(na[3], (colId) => permInfo.getColumnAccess(tableId, colId).read !== 'deny');
if (Object.keys(na[3]).length === 0) { return null; }
return na;
} else if (a[0] === 'AddColumn' || a[0] === 'RemoveColumn' || a[0] === 'RenameColumn' ||
a[0] === 'ModifyColumn') {
const na = cloneDeep(a);
const colId: string = na[2];
if (permInfo.getColumnAccess(tableId, colId).read === 'deny') { return null; }
throw new ErrorWithCode('NEED_RELOAD', 'document needs reload');
} else {
// Remaining cases of AddTable, RemoveTable, RenameTable should have
// been handled at the table level.
}
// TODO: handle access to changes in metadata (trigger a reload at least, if
// all else fails).
return a;
} }
/** /**
@ -487,9 +536,9 @@ export class GranularAccess {
*/ */
public filterData(docSession: OptDocSession, data: TableDataAction) { public filterData(docSession: OptDocSession, data: TableDataAction) {
const permInfo = this._getAccess(docSession); const permInfo = this._getAccess(docSession);
const tableId = data[1] as string; const tableId = getTableId(data);
if (permInfo.getTableAccess(tableId).read === 'mixed') { if (permInfo.getTableAccess(tableId).read === 'mixed') {
this.filterRows(docSession, data); this._filterRowsAndCells(docSession, data, data);
} }
// Filter columns, omitting any to which the user has no access, regardless of rows. // Filter columns, omitting any to which the user has no access, regardless of rows.
@ -497,17 +546,155 @@ export class GranularAccess {
} }
/** /**
* Modify table data in place, removing any rows and scrubbing any cells to which access * Strip out any denied columns from an action. Returns null if nothing is left.
* is not granted.
*/ */
public filterRows(docSession: OptDocSession, data: TableDataAction) { private _pruneColumns(a: DocAction, permInfo: PermissionInfo, tableId: string): DocAction|null {
if (a[0] === 'RemoveRecord' || a[0] === 'BulkRemoveRecord') {
return a;
} else if (a[0] === 'AddRecord' || a[0] === 'BulkAddRecord' || a[0] === 'UpdateRecord' ||
a[0] === 'BulkUpdateRecord' || a[0] === 'ReplaceTableData' || a[0] === 'TableData') {
const na = cloneDeep(a);
this._filterColumns(na[3], (colId) => permInfo.getColumnAccess(tableId, colId).read !== 'deny');
if (Object.keys(na[3]).length === 0) { return null; }
return na;
} else if (a[0] === 'AddColumn' || a[0] === 'RemoveColumn' || a[0] === 'RenameColumn' ||
a[0] === 'ModifyColumn') {
const na = cloneDeep(a);
const colId: string = na[2];
if (permInfo.getColumnAccess(tableId, colId).read === 'deny') { return null; }
throw new ErrorWithCode('NEED_RELOAD', 'document needs reload');
} else {
// Remaining cases of AddTable, RemoveTable, RenameTable should have
// been handled at the table level.
}
// TODO: handle access to changes in metadata (trigger a reload at least, if
// all else fails).
return a;
}
/**
* Strip out any denied rows from an action. The action may be rewritten if rows
* become allowed or denied during the action. An action to add newly-allowed
* rows may be included, or an action to remove newly-forbidden rows. The result
* is a list rather than a single action. It may be the empty list.
*/
private async _pruneRows(docSession: OptDocSession, a: DocAction, idx: number): Promise<DocAction[]> {
// For the moment, only deal with Record-related actions.
// TODO: process table/column schema changes more carefully.
if (isSchemaAction(a)) { return [a]; }
// Get before/after state for this action. Broadcasts to other users can make use of the
// same state, so we share it (and only compute it if needed).
if (!this._rowSnapshots) { throw new Error('Actions not available'); }
const allRowSnapshots = await this._rowSnapshots.get();
const [rowsBefore, rowsAfter] = allRowSnapshots[idx];
// Figure out which rows were forbidden to this session before this action vs
// after this action. We need to know both so that we can infer the state of the
// client and send the correct change.
const ids = new Set(getRowIdsFromDocAction(a));
const forbiddenBefores = new Set(this._getForbiddenRows(docSession, rowsBefore, ids));
const forbiddenAfters = new Set(this._getForbiddenRows(docSession, rowsAfter, ids));
/**
* For rows forbidden before and after: just remove them.
* For rows allowed before and after: just leave them unchanged.
* For rows that were allowed before and are now forbidden:
* - strip them from the current action.
* - add a BulkRemoveRecord for them.
* For rows that were forbidden before and are now allowed:
* - remove them from the current action.
* - add a BulkAddRecord for them.
*/
const removals = new Set<number>(); // rows to remove from current action.
const forceAdds = new Set<number>(); // rows to add, that were previously stripped.
const forceRemoves = new Set<number>(); // rows to remove, that have become forbidden.
for (const id of ids) {
const forbiddenBefore = forbiddenBefores.has(id);
const forbiddenAfter = forbiddenAfters.has(id);
if (!forbiddenBefore && !forbiddenAfter) { continue; }
if (forbiddenBefore && forbiddenAfter) {
removals.add(id);
continue;
}
// If we reach here, then access right to the row changed and we have fancy footwork to do.
if (forbiddenBefore) {
// The row was forbidden and now is allowed. That's trivial if the row was just added.
if (a[0] === 'AddRecord' || a[0] === 'BulkAddRecord' ||
a[0] === 'ReplaceTableData' || a[0] === 'TableData') {
continue;
}
// Otherwise, strip the row from the current action.
removals.add(id);
if (a[0] === 'UpdateRecord' || a[0] === 'BulkUpdateRecord') {
// For updates, we need to send the entire row as an add, since the client
// doesn't know anything about it yet.
forceAdds.add(id);
} else {
// Remaining cases are [Bulk]RemoveRecord.
}
} else {
// The row was allowed and now is forbidden.
// If the action is a removal, that is just right.
if (a[0] === 'RemoveRecord' || a[0] === 'BulkRemoveRecord') { continue; }
// Otherwise, strip the row from the current action.
removals.add(id);
if (a[0] === 'UpdateRecord' || a[0] === 'BulkUpdateRecord') {
// For updates, we need to remove the entire row.
forceRemoves.add(id);
} else {
// Remaining cases are add-like actions.
}
}
}
// Execute our cunning plans for DocAction revisions.
const revisedDocActions = [
this._makeAdditions(rowsAfter, forceAdds),
this._removeRows(a, removals),
this._makeRemovals(rowsAfter, forceRemoves),
].filter(isObject);
// Return the results, also applying any cell-level access control.
for (const docAction of revisedDocActions) {
this._filterRowsAndCells(docSession, rowsAfter, docAction);
}
return revisedDocActions;
}
/**
* Modify action in place, scrubbing any rows and cells to which access is not granted.
*/
private _filterRowsAndCells(docSession: OptDocSession, data: TableDataAction, docAction: DocAction) {
if (docAction && isSchemaAction(docAction)) {
// TODO should filter out metadata about an unavailable column, probably.
return [];
}
const rowCursor = new RecordView(data, 0); const rowCursor = new RecordView(data, 0);
const input: AclMatchInput = {user: this._getUser(docSession), rec: rowCursor}; const input: AclMatchInput = {user: this._getUser(docSession), rec: rowCursor};
const [, tableId, rowIds, colValues] = data; const [, tableId, , colValues] = docAction;
if (colValues === undefined) { return []; }
const rowIds = getRowIdsFromDocAction(docAction);
const toRemove: number[] = []; const toRemove: number[] = [];
let censorAt: (colId: string, idx: number) => void;
if (Array.isArray(docAction[2])) {
censorAt = (colId, idx) => (colValues as BulkColValues)[colId][idx] = 'CENSORED'; // TODO Pick a suitable value
} else {
censorAt = (colId) => (colValues as ColValues)[colId] = 'CENSORED'; // TODO Pick a suitable value
}
let getDataIndex: (idx: number) => number = (idx) => idx;
if (docAction !== data) {
const indexes = new Map(data[2].map((rowId, idx) => [rowId, idx]));
getDataIndex = (idx) => indexes.get(rowIds[idx])!;
}
for (let idx = 0; idx < rowIds.length; idx++) { for (let idx = 0; idx < rowIds.length; idx++) {
rowCursor.index = idx; rowCursor.index = getDataIndex(idx);
const rowPermInfo = new PermissionInfo(this, input); const rowPermInfo = new PermissionInfo(this, input);
// getTableAccess() evaluates all column rules for THIS record. So it's really rowAccess. // getTableAccess() evaluates all column rules for THIS record. So it's really rowAccess.
@ -519,16 +706,54 @@ export class GranularAccess {
for (const colId of Object.keys(colValues)) { for (const colId of Object.keys(colValues)) {
const colAccess = rowPermInfo.getColumnAccess(tableId, colId); const colAccess = rowPermInfo.getColumnAccess(tableId, colId);
if (colAccess.read !== 'allow') { if (colAccess.read !== 'allow') {
colValues[colId][idx] = 'CENSORED'; // TODO Pick a suitable value censorAt(colId, idx);
} }
} }
} }
} }
if (toRemove.length > 0) {
if (data === docAction) {
this._removeRowsAt(toRemove, data[2], data[3]);
} else {
// If there are still rows to remove, we must have a logic error.
throw new Error('Unexpected row removal');
}
}
}
// Compute which of the row ids supplied are for rows forbidden for this session.
private _getForbiddenRows(docSession: OptDocSession, data: TableDataAction, ids: Set<number>): number[] {
const rowCursor = new RecordView(data, 0);
const input: AclMatchInput = {user: this._getUser(docSession), rec: rowCursor};
const [, tableId, rowIds,] = data;
const toRemove: number[] = [];
for (let idx = 0; idx < rowIds.length; idx++) {
rowCursor.index = idx;
if (!ids.has(rowIds[idx])) { continue; }
const rowPermInfo = new PermissionInfo(this, input);
// getTableAccess() evaluates all column rules for THIS record. So it's really rowAccess.
const rowAccess = rowPermInfo.getTableAccess(tableId);
if (rowAccess.read === 'deny') {
toRemove.push(rowIds[idx]);
}
}
return toRemove;
}
/**
* Removes the toRemove rows (indexes, not row ids) from the rowIds list and from
* the colValues structure.
*/
private _removeRowsAt(toRemove: number[], rowIds: number[], colValues: BulkColValues|undefined) {
if (toRemove.length > 0) { if (toRemove.length > 0) {
pullAt(rowIds, toRemove); pullAt(rowIds, toRemove);
for (const values of Object.values(colValues)) { if (colValues) {
pullAt(values, toRemove); for (const values of Object.values(colValues)) {
pullAt(values, toRemove);
}
} }
} }
} }
@ -588,7 +813,7 @@ export class GranularAccess {
if (this._characteristicTables.get(clause.name)) { if (this._characteristicTables.get(clause.name)) {
throw new Error(`User attribute ${clause.name} ignored: duplicate name`); throw new Error(`User attribute ${clause.name} ignored: duplicate name`);
} }
const data = await this._fetchQuery({tableId: clause.tableId, filters: {}}); const data = await this._fetchQueryFromDB({tableId: clause.tableId, filters: {}});
const rowNums = new Map<string, number>(); const rowNums = new Map<string, number>();
const matches = data[3][clause.lookupColId]; const matches = data[3][clause.lookupColId];
for (let i = 0; i < matches.length; i++) { for (let i = 0; i < matches.length; i++) {
@ -648,6 +873,45 @@ export class GranularAccess {
} }
return user; return user;
} }
/**
* Remove a set of rows from a DocAction. If the DocAction ends up empty, null is returned.
* If the DocAction needs modification, it is copied first - the original is never
* changed.
*/
private _removeRows(a: DocAction, rowIds: Set<number>): DocAction|null {
// If there are no rows, there's nothing to do.
if (isSchemaAction(a)) { return a; }
if (a[0] === 'AddRecord' || a[0] === 'UpdateRecord' || a[0] === 'RemoveRecord') {
return rowIds.has(a[2]) ? null : a;
}
const na = cloneDeep(a);
const [, , oldIds, bulkColValues] = na;
const mask = oldIds.map((id, idx) => rowIds.has(id) && idx || -1).filter(v => v !== -1);
this._removeRowsAt(mask, oldIds, bulkColValues);
if (oldIds.length === 0) { return null; }
return na;
}
/**
* Make a BulkAddRecord for a set of rows.
*/
private _makeAdditions(data: TableDataAction, rowIds: Set<number>): BulkAddRecord|null {
if (rowIds.size === 0) { return null; }
// TODO: optimize implementation, this does an unnecessary clone.
const notAdded = data[2].filter(id => !rowIds.has(id));
const partialData = this._removeRows(data, new Set(notAdded)) as TableDataAction|null;
if (partialData === null) { return partialData; }
return ['BulkAddRecord', partialData[1], partialData[2], partialData[3]];
}
/**
* Make a BulkRemoveRecord for a set of rows.
*/
private _makeRemovals(data: TableDataAction, rowIds: Set<number>): BulkRemoveRecord|null {
if (rowIds.size === 0) { return null; }
return ['BulkRemoveRecord', getTableId(data), [...rowIds]];
}
} }
/** /**
@ -785,3 +1049,7 @@ interface CharacteristicTable {
rowNums: Map<string, number>; rowNums: Map<string, number>;
data: TableDataAction; data: TableDataAction;
} }
function isObject<T>(value: T | null | undefined): value is T {
return value !== null && value !== undefined;
}

View File

@ -0,0 +1,63 @@
import { AddRecord, BulkAddRecord, BulkRemoveRecord, BulkUpdateRecord, DocAction, getTableId,
RemoveRecord, ReplaceTableData, TableDataAction, UpdateRecord } from "app/common/DocActions";
import { getSetMapValue } from "app/common/gutil";
/**
* A little class for tracking pre-existing rows touched by a sequence of DocActions for
* a given table.
*/
class RowIdTracker {
public blockedIds = new Set<number>(); // row ids minted within the DocActions (so NOT pre-existing).
public blocked: boolean = false; // set if all pre-existing rows are wiped/
public ids = new Set<number>(); // set of pre-existing rows touched.
}
/**
* This gets a list of pre-existing rows that the DocActions may touch. Returns
* a list of form [tableId, Set{rowId1, rowId2, ...}].
*/
export function getRelatedRows(docActions: DocAction[]): ReadonlyArray<readonly [string, Set<number>]> {
// Relate tableIds for tables with what they were before the actions, if renamed.
const tableIds = new Map<string, string>();
const rowIds = new Map<string, RowIdTracker>();
for (const docAction of docActions) {
const currentTableId = getTableId(docAction);
const tableId = tableIds.get(currentTableId) || currentTableId;
if (docAction[0] === 'RenameTable') {
tableIds.delete(currentTableId);
tableIds.set(docAction[2], tableId);
continue;
}
// tableId will now be that prior to docActions, regardless of renames.
const tracker = getSetMapValue(rowIds, tableId, () => new RowIdTracker());
if (docAction[0] === 'RemoveRecord' || docAction[0] === 'BulkRemoveRecord' ||
docAction[0] === 'UpdateRecord' || docAction[0] === 'BulkUpdateRecord') {
// All row ids mentioned are external, unless created within this set of DocActions.
if (!tracker.blocked) {
for (const id of getRowIdsFromDocAction(docAction)) {
if (!tracker.blockedIds.has(id)) { tracker.ids.add(id); }
}
}
} else if (docAction[0] === 'AddRecord' || docAction[0] === 'BulkAddRecord') {
// All row ids mentioned are created within this set of DocActions, and are not external.
for (const id of getRowIdsFromDocAction(docAction)) { tracker.blockedIds.add(id); }
} else if (docAction[0] === 'ReplaceTableData' || docAction[0] === 'TableData') {
// No pre-existing rows can be referred to for this table from now on.
tracker.blocked = true;
}
}
return [...rowIds.entries()].map(([tableId, tracker]) => [tableId, tracker.ids] as const);
}
/**
* Tiny helper to get the row ids mentioned in a record-related DocAction as a list
* (even if the action is not a bulk action).
*/
export function getRowIdsFromDocAction(docActions: RemoveRecord | BulkRemoveRecord | AddRecord |
BulkAddRecord | UpdateRecord | BulkUpdateRecord | ReplaceTableData | TableDataAction) {
const ids = docActions[2];
return (typeof ids === 'number') ? [ids] : ids;
}

View File

@ -222,6 +222,7 @@ export class Sharing {
(branch === Branch.Shared ? this._actionHistory.getNextHubActionNum() : (branch === Branch.Shared ? this._actionHistory.getNextHubActionNum() :
this._actionHistory.getNextLocalActionNum()); this._actionHistory.getNextLocalActionNum());
const undo = getEnvContent(sandboxActionBundle.undo);
const localActionBundle: LocalActionBundle = { const localActionBundle: LocalActionBundle = {
actionNum, actionNum,
// The ActionInfo should go into the envelope that includes all recipients. // The ActionInfo should go into the envelope that includes all recipients.
@ -236,6 +237,9 @@ export class Sharing {
}; };
this._logActionBundle(`doApplyUserActions (${Branch[branch]})`, localActionBundle); this._logActionBundle(`doApplyUserActions (${Branch[branch]})`, localActionBundle);
const docActions = getEnvContent(localActionBundle.stored).concat(
getEnvContent(localActionBundle.calc));
// TODO Note that the sandbox may produce actions which are not addressed to us (e.g. when we // TODO Note that the sandbox may produce actions which are not addressed to us (e.g. when we
// have EDIT permission without VIEW). These are not sent to the browser or the database. But // have EDIT permission without VIEW). These are not sent to the browser or the database. But
// today they are reflected in the sandbox. Should we (or the sandbox) immediately undo the // today they are reflected in the sandbox. Should we (or the sandbox) immediately undo the
@ -281,11 +285,15 @@ export class Sharing {
// date and other changes from external values may count as internal. // date and other changes from external values may count as internal.
internal: isCalculate, internal: isCalculate,
}); });
await this._activeDoc.broadcastDocUpdate(client || null, 'docUserAction', { await this._activeDoc.beforeBroadcast(docActions, undo);
actionGroup, try {
docActions: getEnvContent(localActionBundle.stored).concat( await this._activeDoc.broadcastDocUpdate(client || null, 'docUserAction', {
getEnvContent(localActionBundle.calc)) actionGroup,
}); docActions,
});
} finally {
await this._activeDoc.afterBroadcast();
}
return { return {
actionNum: localActionBundle.actionNum, actionNum: localActionBundle.actionNum,
retValues: sandboxActionBundle.retValues, retValues: sandboxActionBundle.retValues,