mirror of
				https://github.com/gristlabs/grist-core.git
				synced 2025-06-13 20:53:59 +00:00 
			
		
		
		
	(core) Add flexibility to daily API usage limit
Summary: Allow exceeding the daily API usage limit for a doc based on additional allocations for the current hour and minute. See the doc comment on getDocApiUsageKeysToIncr for details. This means that up to 5 redis keys may be relevant at a time for a single document. Test Plan: Updated and expanded 'Daily API Limit' tests. Reviewers: dsagal Reviewed By: dsagal Differential Revision: https://phab.getgrist.com/D3368
This commit is contained in:
		
							parent
							
								
									4de5928396
								
							
						
					
					
						commit
						0beb2898cb
					
				@ -43,6 +43,11 @@ export const teamFreeFeatures: Features = {
 | 
			
		||||
  gracePeriodDays: 14,
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
export const testDailyApiLimitFeatures = {
 | 
			
		||||
  ...teamFreeFeatures,
 | 
			
		||||
  baseMaxApiUnitsPerDocumentPerDay: 3,
 | 
			
		||||
};
 | 
			
		||||
 | 
			
		||||
/**
 | 
			
		||||
 * A summary of features used in unrestricted grandfathered accounts, and also
 | 
			
		||||
 * in some test settings.
 | 
			
		||||
@ -87,7 +92,7 @@ export interface IProduct {
 | 
			
		||||
 * TODO: change capitalization of name of grandfather product.
 | 
			
		||||
 *
 | 
			
		||||
 */
 | 
			
		||||
const PRODUCTS: IProduct[] = [
 | 
			
		||||
export const PRODUCTS: IProduct[] = [
 | 
			
		||||
  // This is a product for grandfathered accounts/orgs.
 | 
			
		||||
  {
 | 
			
		||||
    name: 'Free',
 | 
			
		||||
@ -166,7 +171,9 @@ export class Product extends BaseEntity {
 | 
			
		||||
 * If `apply` is set, the products are changed in the db, otherwise
 | 
			
		||||
 * the are left unchanged.  A summary of affected products is returned.
 | 
			
		||||
 */
 | 
			
		||||
export async function synchronizeProducts(connection: Connection, apply: boolean): Promise<string[]> {
 | 
			
		||||
export async function synchronizeProducts(
 | 
			
		||||
  connection: Connection, apply: boolean, products = PRODUCTS
 | 
			
		||||
): Promise<string[]> {
 | 
			
		||||
  try {
 | 
			
		||||
    await connection.query('select name, features, stripe_product_id from products limit 1');
 | 
			
		||||
  } catch (e) {
 | 
			
		||||
@ -175,7 +182,7 @@ export async function synchronizeProducts(connection: Connection, apply: boolean
 | 
			
		||||
  }
 | 
			
		||||
  const changingProducts: string[] = [];
 | 
			
		||||
  await connection.transaction(async transaction => {
 | 
			
		||||
    const desiredProducts = new Map(PRODUCTS.map(p => [p.name, p]));
 | 
			
		||||
    const desiredProducts = new Map(products.map(p => [p.name, p]));
 | 
			
		||||
    const existingProducts = new Map((await transaction.find(Product))
 | 
			
		||||
                                     .map(p => [p.name, p]));
 | 
			
		||||
    for (const product of desiredProducts.values()) {
 | 
			
		||||
 | 
			
		||||
@ -137,8 +137,8 @@ class DummyDocWorkerMap implements IDocWorkerMap {
 | 
			
		||||
    return null;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  public incrementDocApiUsage(key: string): Promise<number> {
 | 
			
		||||
    return Promise.resolve(0);
 | 
			
		||||
  public getRedisClient(): RedisClient {
 | 
			
		||||
    throw new Error("No redis client here");
 | 
			
		||||
  }
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -517,16 +517,8 @@ export class DocWorkerMap implements IDocWorkerMap {
 | 
			
		||||
    return this._client.getAsync(`doc-${docId}-group`);
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  /**
 | 
			
		||||
   * Increment the value at the given redis key representing API usage of one document in one day.
 | 
			
		||||
   * Expire the key after a day just so that it cleans itself up.
 | 
			
		||||
   * Returns the value after incrementing.
 | 
			
		||||
   * This is not related to other responsibilities of this class,
 | 
			
		||||
   * but this class conveniently manages the redis client.
 | 
			
		||||
   */
 | 
			
		||||
  public async incrementDocApiUsage(key: string): Promise<number | null> {
 | 
			
		||||
    const result = await this._client.multi().incr(key).expire(key, 24 * 60 * 60).execAsync();
 | 
			
		||||
    return Number(result?.[0]);
 | 
			
		||||
  public getRedisClient(): RedisClient {
 | 
			
		||||
    return this._client;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  /**
 | 
			
		||||
 | 
			
		||||
@ -61,6 +61,7 @@ import * as contentDisposition from 'content-disposition';
 | 
			
		||||
import {Application, NextFunction, Request, RequestHandler, Response} from "express";
 | 
			
		||||
import * as _ from "lodash";
 | 
			
		||||
import * as LRUCache from 'lru-cache';
 | 
			
		||||
import * as moment from 'moment';
 | 
			
		||||
import fetch from 'node-fetch';
 | 
			
		||||
import * as path from 'path';
 | 
			
		||||
import * as t from "ts-interface-checker";
 | 
			
		||||
@ -72,6 +73,12 @@ import * as uuidv4 from "uuid/v4";
 | 
			
		||||
// reply with status 429.
 | 
			
		||||
const MAX_PARALLEL_REQUESTS_PER_DOC = 10;
 | 
			
		||||
 | 
			
		||||
// This is NOT the number of docs that can be handled at a time.
 | 
			
		||||
// It's a very generous upper bound of what that number might be.
 | 
			
		||||
// If there are more docs than this for which API requests are being regularly made at any moment,
 | 
			
		||||
// then the _dailyUsage cache may become unreliable and users may be able to exceed their allocated requests.
 | 
			
		||||
const MAX_ACTIVE_DOCS_USAGE_CACHE = 1000;
 | 
			
		||||
 | 
			
		||||
type WithDocHandler = (activeDoc: ActiveDoc, req: RequestWithLogin, resp: Response) => Promise<void>;
 | 
			
		||||
 | 
			
		||||
// Schema validators for api endpoints that creates or updates records.
 | 
			
		||||
@ -99,6 +106,14 @@ function validate(checker: Checker): RequestHandler {
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
export class DocWorkerApi {
 | 
			
		||||
  // Map from docId to number of requests currently being handled for that doc
 | 
			
		||||
  private _currentUsage = new Map<string, number>();
 | 
			
		||||
 | 
			
		||||
  // Map from (docId, time period) combination produced by docPeriodicApiUsageKey
 | 
			
		||||
  // to number of requests previously served for that combination.
 | 
			
		||||
  // We multiply by 5 because there are 5 relevant keys per doc at any time (current/next day/hour and current minute).
 | 
			
		||||
  private _dailyUsage = new LRUCache<string, number>({max: 5 * MAX_ACTIVE_DOCS_USAGE_CACHE});
 | 
			
		||||
 | 
			
		||||
  constructor(private _app: Application, private _docWorker: DocWorker,
 | 
			
		||||
              private _docWorkerMap: IDocWorkerMap, private _docManager: DocManager,
 | 
			
		||||
              private _dbManager: HomeDBManager, private _grist: GristServer) {}
 | 
			
		||||
@ -771,19 +786,17 @@ export class DocWorkerApi {
 | 
			
		||||
  private _apiThrottle(callback: (req: RequestWithLogin,
 | 
			
		||||
                                  resp: Response,
 | 
			
		||||
                                  next: NextFunction) => void | Promise<void>): RequestHandler {
 | 
			
		||||
    const usage = new Map<string, number>();
 | 
			
		||||
    const dailyUsage = new LRUCache<string, number>({max: 1024});
 | 
			
		||||
    return async (req, res, next) => {
 | 
			
		||||
      const docId = getDocId(req);
 | 
			
		||||
      try {
 | 
			
		||||
        const count = usage.get(docId) || 0;
 | 
			
		||||
        usage.set(docId, count + 1);
 | 
			
		||||
        const count = this._currentUsage.get(docId) || 0;
 | 
			
		||||
        this._currentUsage.set(docId, count + 1);
 | 
			
		||||
        if (count + 1 > MAX_PARALLEL_REQUESTS_PER_DOC) {
 | 
			
		||||
          throw new ApiError(`Too many backlogged requests for document ${docId} - ` +
 | 
			
		||||
            `try again later?`, 429);
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
        if (await this._checkDailyDocApiUsage(req, docId, dailyUsage)) {
 | 
			
		||||
        if (await this._checkDailyDocApiUsage(req, docId)) {
 | 
			
		||||
          throw new ApiError(`Exceeded daily limit for document ${docId}`, 429);
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
@ -791,12 +804,12 @@ export class DocWorkerApi {
 | 
			
		||||
      } catch (err) {
 | 
			
		||||
        next(err);
 | 
			
		||||
      } finally {
 | 
			
		||||
        const count = usage.get(docId);
 | 
			
		||||
        const count = this._currentUsage.get(docId);
 | 
			
		||||
        if (count) {
 | 
			
		||||
          if (count === 1) {
 | 
			
		||||
            usage.delete(docId);
 | 
			
		||||
            this._currentUsage.delete(docId);
 | 
			
		||||
          } else {
 | 
			
		||||
            usage.set(docId, count - 1);
 | 
			
		||||
            this._currentUsage.set(docId, count - 1);
 | 
			
		||||
          }
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
@ -805,57 +818,66 @@ export class DocWorkerApi {
 | 
			
		||||
 | 
			
		||||
  /**
 | 
			
		||||
   * Usually returns true if too many requests (based on the user's product plan)
 | 
			
		||||
   * have been made today for this document.
 | 
			
		||||
   * have been made today for this document and the request should be rejected.
 | 
			
		||||
   * Access to a document must already have been authorized.
 | 
			
		||||
   * This is called frequently so it uses caches to check quickly in the common case,
 | 
			
		||||
   * which allows a few ways for users to exceed the limit slightly if the timing works out,
 | 
			
		||||
   * but these should be acceptable.
 | 
			
		||||
   */
 | 
			
		||||
  private async _checkDailyDocApiUsage(req: Request, docId: string, dailyUsage: LRUCache<string, number>) {
 | 
			
		||||
    // Start with the possibly stale cached doc to avoid a database call.
 | 
			
		||||
    // This leaves a small window for the user to bypass this limit after downgrading.
 | 
			
		||||
    let doc = (req as RequestWithLogin).docAuth!.cachedDoc!;
 | 
			
		||||
  private async _checkDailyDocApiUsage(req: Request, docId: string): Promise<boolean> {
 | 
			
		||||
    // Use the cached doc to avoid a database call.
 | 
			
		||||
    // This leaves a small window (currently 5 seconds) for the user to bypass this limit after downgrading,
 | 
			
		||||
    // or to be wrongly rejected after upgrading.
 | 
			
		||||
    const doc = (req as RequestWithLogin).docAuth!.cachedDoc!;
 | 
			
		||||
 | 
			
		||||
    function getMax() {
 | 
			
		||||
      return doc.workspace.org.billingAccount?.product.features.baseMaxApiUnitsPerDocumentPerDay;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    let max = getMax();
 | 
			
		||||
    const max = doc.workspace.org.billingAccount?.product.features.baseMaxApiUnitsPerDocumentPerDay;
 | 
			
		||||
    if (!max) {
 | 
			
		||||
      // This doc has no associated product (happens to new unsaved docs)
 | 
			
		||||
      // or the product has no API limit.
 | 
			
		||||
      return;
 | 
			
		||||
      // or the product has no API limit. Allow the request through.
 | 
			
		||||
      return false;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    // Get the current count from the dailyUsage cache rather than waiting for redis.
 | 
			
		||||
    // The cache will not have a count if this is the first request for this document served by this worker process
 | 
			
		||||
    // or if so many other documents have been served since then that this key was evicted from the LRU cache.
 | 
			
		||||
    // Check the counts in the dailyUsage cache rather than waiting for redis.
 | 
			
		||||
    // The cache will not have counts if this is the first request for this document served by this worker process
 | 
			
		||||
    // or if so many other documents have been served since then that the keys were evicted from the LRU cache.
 | 
			
		||||
    // Both scenarios are temporary and unlikely when usage has been exceeded.
 | 
			
		||||
    const key = docDailyApiUsageKey(docId);
 | 
			
		||||
    const count = dailyUsage.get(key);
 | 
			
		||||
 | 
			
		||||
    if (count && count >= max) {
 | 
			
		||||
      // The limit has apparently been exceeded.
 | 
			
		||||
      // In case the user just upgraded, get a fresh Document entity from the DB and check again.
 | 
			
		||||
      doc = await this._dbManager.getDoc(getDocScope(req));
 | 
			
		||||
      max = getMax();
 | 
			
		||||
      if (max && count >= max) {
 | 
			
		||||
        return true;
 | 
			
		||||
      }
 | 
			
		||||
    // Note that if the limits are exceeded then `keys` below will be undefined,
 | 
			
		||||
    // otherwise it will be an array of three keys corresponding to a day, hour, and minute.
 | 
			
		||||
    const m = moment.utc();
 | 
			
		||||
    const keys = getDocApiUsageKeysToIncr(docId, this._dailyUsage, max, m);
 | 
			
		||||
    if (!keys) {
 | 
			
		||||
      // The limit has been exceeded, reject the request.
 | 
			
		||||
      return true;
 | 
			
		||||
    }
 | 
			
		||||
 | 
			
		||||
    // Note the increased API usage on redis and in our local cache.
 | 
			
		||||
    // Do this in the background so that the rest of the request can continue without waiting for redis.
 | 
			
		||||
    // If the user makes many concurrent requests quickly,
 | 
			
		||||
    // a few extra might slip through before we see the count exceeding the limit, but this is basically unavoidable.
 | 
			
		||||
    this._docWorkerMap.incrementDocApiUsage(key).then(newCount => {
 | 
			
		||||
      if (newCount) {
 | 
			
		||||
    // Update redis in the background so that the rest of the request can continue without waiting for redis.
 | 
			
		||||
    const multi = this._docWorkerMap.getRedisClient().multi();
 | 
			
		||||
    for (let i = 0; i < keys.length; i++) {
 | 
			
		||||
      const key = keys[i];
 | 
			
		||||
      // Incrementing the local count immediately prevents many requests from being squeezed through every minute
 | 
			
		||||
      // before counts are received from redis.
 | 
			
		||||
      // But this cache is not 100% reliable and the count from redis may be higher.
 | 
			
		||||
      this._dailyUsage.set(key, (this._dailyUsage.get(key) ?? 0) + 1);
 | 
			
		||||
      const period = docApiUsagePeriods[i];
 | 
			
		||||
      // Expire the key just so that it cleans itself up and saves memory on redis.
 | 
			
		||||
      // Expire after two periods to handle 'next' buckets.
 | 
			
		||||
      const expiry = 2 * 24 * 60 * 60 / period.periodsPerDay;
 | 
			
		||||
      multi.incr(key).expire(key, expiry);
 | 
			
		||||
    }
 | 
			
		||||
    multi.execAsync().then(result => {
 | 
			
		||||
      for (let i = 0; i < keys.length; i++) {
 | 
			
		||||
        const key = keys[i];
 | 
			
		||||
        const newCount = Number(result![i * 2]);  // incrs are at even positions, expires at odd positions
 | 
			
		||||
        // Theoretically this could be overwritten by a lower count that was requested earlier
 | 
			
		||||
        // but somehow arrived after.
 | 
			
		||||
        // This doesn't really matter, and the count on redis will still increase reliably.
 | 
			
		||||
        dailyUsage.set(key, newCount);
 | 
			
		||||
        this._dailyUsage.set(key, newCount);
 | 
			
		||||
      }
 | 
			
		||||
    }).catch(e => console.error(`Error tracking API usage for doc ${docId}`, e));
 | 
			
		||||
 | 
			
		||||
    // Allow the request through.
 | 
			
		||||
    return false;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
  private async _assertAccess(role: 'viewers'|'editors'|'owners'|null, allowRemoved: boolean,
 | 
			
		||||
@ -1140,14 +1162,78 @@ async function handleSandboxError<T>(tableId: string, colNames: string[], p: Pro
 | 
			
		||||
  return handleSandboxErrorOnPlatform(tableId, colNames, p, getErrorPlatform(tableId));
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
export interface DocApiUsagePeriod {
 | 
			
		||||
  unit: 'day' | 'hour' | 'minute',
 | 
			
		||||
  format: string;
 | 
			
		||||
  periodsPerDay: number;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
export const docApiUsagePeriods: DocApiUsagePeriod[] = [
 | 
			
		||||
  {
 | 
			
		||||
    unit: 'day',
 | 
			
		||||
    format: 'YYYY-MM-DD',
 | 
			
		||||
    periodsPerDay: 1,
 | 
			
		||||
  },
 | 
			
		||||
  {
 | 
			
		||||
    unit: 'hour',
 | 
			
		||||
    format: 'YYYY-MM-DDTHH',
 | 
			
		||||
    periodsPerDay: 24,
 | 
			
		||||
  },
 | 
			
		||||
  {
 | 
			
		||||
    unit: 'minute',
 | 
			
		||||
    format: 'YYYY-MM-DDTHH:mm',
 | 
			
		||||
    periodsPerDay: 24 * 60,
 | 
			
		||||
  },
 | 
			
		||||
];
 | 
			
		||||
 | 
			
		||||
/**
 | 
			
		||||
 * Returns a key used for redis and a local cache
 | 
			
		||||
 * which store the number of API requests made for the given document today.
 | 
			
		||||
 * Defined here so that it can easily be accessed in tests.
 | 
			
		||||
 * The key contains the current UTC date so that counts from previous days are simply ignored and eventually evicted.
 | 
			
		||||
 * which store the number of API requests made for the given document in the given period.
 | 
			
		||||
 * The key contains the current UTC date (and maybe hour and minute)
 | 
			
		||||
 * so that counts from previous periods are simply ignored and eventually evicted.
 | 
			
		||||
 * This means that the daily measured usage conceptually 'resets' at UTC midnight.
 | 
			
		||||
 * If `current` is false, returns a key for the next day/hour.
 | 
			
		||||
 */
 | 
			
		||||
export function docDailyApiUsageKey(docId: string) {
 | 
			
		||||
  const d = new Date();
 | 
			
		||||
  return `doc-${docId}-dailyApiUsage-${d.getUTCFullYear()}-${d.getUTCMonth() + 1}-${d.getUTCDate()}`;
 | 
			
		||||
export function docPeriodicApiUsageKey(docId: string, current: boolean, period: DocApiUsagePeriod, m: moment.Moment) {
 | 
			
		||||
  if (!current) {
 | 
			
		||||
    m = m.clone().add(1, period.unit);
 | 
			
		||||
  }
 | 
			
		||||
  return `doc-${docId}-periodicApiUsage-${m.format(period.format)}`;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
/**
 | 
			
		||||
 * Checks whether the doc API usage fits within the daily maximum.
 | 
			
		||||
 * If so, returns an array of keys for each unit of time whose usage should be incremented.
 | 
			
		||||
 * If not, returns undefined.
 | 
			
		||||
 *
 | 
			
		||||
 * Description of the algorithm this is implementing:
 | 
			
		||||
 *
 | 
			
		||||
 * Maintain up to 5 buckets: current day, next day, current hour, next hour, current minute.
 | 
			
		||||
 * For each API request, check in order:
 | 
			
		||||
 * - if current_day < DAILY_LIMIT, allow; increment all 3 current buckets
 | 
			
		||||
 * - else if current_hour < DAILY_LIMIT/24, allow; increment next_day, current_hour, and current_minute buckets.
 | 
			
		||||
 * - else if current_minute < DAILY_LIMIT/24/60, allow; increment next_day, next_hour, and current_minute buckets.
 | 
			
		||||
 * - else reject.
 | 
			
		||||
 * I think it has pretty good properties:
 | 
			
		||||
 * - steady low usage may be maintained even if a burst exhausted the daily limit
 | 
			
		||||
 * - user could get close to twice the daily limit on the first day with steady usage after a burst,
 | 
			
		||||
 *   but would then be limited to steady usage the next day.
 | 
			
		||||
 */
 | 
			
		||||
export function getDocApiUsageKeysToIncr(
 | 
			
		||||
  docId: string, usage: LRUCache<string, number>, dailyMax: number, m: moment.Moment
 | 
			
		||||
): string[] | undefined {
 | 
			
		||||
  // Start with keys for the current day, minute, and hour
 | 
			
		||||
  const keys = docApiUsagePeriods.map(p => docPeriodicApiUsageKey(docId, true, p, m));
 | 
			
		||||
  for (let i = 0; i < docApiUsagePeriods.length; i++) {
 | 
			
		||||
    const period = docApiUsagePeriods[i];
 | 
			
		||||
    const key = keys[i];
 | 
			
		||||
    const periodMax = Math.ceil(dailyMax / period.periodsPerDay);
 | 
			
		||||
    const count = usage.get(key) || 0;
 | 
			
		||||
    if (count < periodMax) {
 | 
			
		||||
      return keys;
 | 
			
		||||
    }
 | 
			
		||||
    // Allocation for the current day/hour/minute has been exceeded, increment the next day/hour/minute instead.
 | 
			
		||||
    keys[i] = docPeriodicApiUsageKey(docId, false, period, m);
 | 
			
		||||
  }
 | 
			
		||||
  // Usage exceeded all the time buckets, so return undefined to reject the request.
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
@ -6,6 +6,7 @@
 | 
			
		||||
import { IChecksumStore } from 'app/server/lib/IChecksumStore';
 | 
			
		||||
import { IElectionStore } from 'app/server/lib/IElectionStore';
 | 
			
		||||
import { IPermitStores } from 'app/server/lib/Permit';
 | 
			
		||||
import {RedisClient} from 'redis';
 | 
			
		||||
 | 
			
		||||
export interface DocWorkerInfo {
 | 
			
		||||
  id: string;
 | 
			
		||||
@ -67,5 +68,5 @@ export interface IDocWorkerMap extends IPermitStores, IElectionStore, IChecksumS
 | 
			
		||||
  getWorkerGroup(workerId: string): Promise<string|null>;
 | 
			
		||||
  getDocGroup(docId: string): Promise<string|null>;
 | 
			
		||||
 | 
			
		||||
  incrementDocApiUsage(key: string): Promise<number|null>;
 | 
			
		||||
  getRedisClient(): RedisClient;
 | 
			
		||||
}
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										1
									
								
								stubs/app/server/declarations.d.ts
									
									
									
									
										vendored
									
									
								
							
							
						
						
									
										1
									
								
								stubs/app/server/declarations.d.ts
									
									
									
									
										vendored
									
									
								
							@ -66,6 +66,7 @@ declare module "redis" {
 | 
			
		||||
    public sadd(key: string, val: string): Multi;
 | 
			
		||||
    public set(key: string, val: string): Multi;
 | 
			
		||||
    public setex(key: string, ttl: number, val: string): Multi;
 | 
			
		||||
    public ttl(key: string): Multi;
 | 
			
		||||
    public smembers(key: string): Multi;
 | 
			
		||||
    public srandmember(key: string): Multi;
 | 
			
		||||
    public srem(key: string, val: string): Multi;
 | 
			
		||||
 | 
			
		||||
@ -37,7 +37,7 @@ import {Document} from "app/gen-server/entity/Document";
 | 
			
		||||
import {Group} from "app/gen-server/entity/Group";
 | 
			
		||||
import {Login} from "app/gen-server/entity/Login";
 | 
			
		||||
import {Organization} from "app/gen-server/entity/Organization";
 | 
			
		||||
import {Product, synchronizeProducts} from "app/gen-server/entity/Product";
 | 
			
		||||
import {Product, PRODUCTS, synchronizeProducts, testDailyApiLimitFeatures} from "app/gen-server/entity/Product";
 | 
			
		||||
import {User} from "app/gen-server/entity/User";
 | 
			
		||||
import {Workspace} from "app/gen-server/entity/Workspace";
 | 
			
		||||
import {EXAMPLE_WORKSPACE_NAME} from 'app/gen-server/lib/HomeDBManager';
 | 
			
		||||
@ -48,6 +48,14 @@ import * as fse from 'fs-extra';
 | 
			
		||||
 | 
			
		||||
const ACCESS_GROUPS = ['owners', 'editors', 'viewers', 'guests', 'members'];
 | 
			
		||||
 | 
			
		||||
const testProducts = [
 | 
			
		||||
  ...PRODUCTS,
 | 
			
		||||
    {
 | 
			
		||||
    name: 'testDailyApiLimit',
 | 
			
		||||
    features: testDailyApiLimitFeatures,
 | 
			
		||||
  },
 | 
			
		||||
];
 | 
			
		||||
 | 
			
		||||
export const exampleOrgs = [
 | 
			
		||||
  {
 | 
			
		||||
    name: 'NASA',
 | 
			
		||||
@ -179,11 +187,23 @@ export const exampleOrgs = [
 | 
			
		||||
      }
 | 
			
		||||
    ]
 | 
			
		||||
  },
 | 
			
		||||
  {
 | 
			
		||||
    name: 'TestDailyApiLimit',
 | 
			
		||||
    domain: 'testdailyapilimit',
 | 
			
		||||
    product: 'testDailyApiLimit',
 | 
			
		||||
    workspaces: [
 | 
			
		||||
      {
 | 
			
		||||
        name: 'TestDailyApiLimitWs',
 | 
			
		||||
        docs: [],
 | 
			
		||||
      }
 | 
			
		||||
    ]
 | 
			
		||||
  },
 | 
			
		||||
];
 | 
			
		||||
 | 
			
		||||
 | 
			
		||||
const exampleUsers: {[user: string]: {[org: string]: string}} = {
 | 
			
		||||
  Chimpy: {
 | 
			
		||||
    TestDailyApiLimit: 'owners',
 | 
			
		||||
    FreeTeam: 'owners',
 | 
			
		||||
    Chimpyland: 'owners',
 | 
			
		||||
    NASA: 'owners',
 | 
			
		||||
@ -527,7 +547,7 @@ export async function createInitialDb(connection?: Connection, migrateAndSeedDat
 | 
			
		||||
 | 
			
		||||
// add some test data to the database.
 | 
			
		||||
export async function addSeedData(connection: Connection) {
 | 
			
		||||
  await synchronizeProducts(connection, true);
 | 
			
		||||
  await synchronizeProducts(connection, true, testProducts);
 | 
			
		||||
  await connection.transaction(async tr => {
 | 
			
		||||
    const seed = new Seed(tr.connection);
 | 
			
		||||
    await seed.run();
 | 
			
		||||
 | 
			
		||||
@ -25,6 +25,7 @@ import {tmpdir} from 'os';
 | 
			
		||||
import * as path from 'path';
 | 
			
		||||
import {removeConnection} from 'test/gen-server/seed';
 | 
			
		||||
import {HomeUtil} from 'test/nbrowser/homeUtil';
 | 
			
		||||
import {getDatabase} from 'test/testUtils';
 | 
			
		||||
 | 
			
		||||
export class TestServerMerged implements IMochaServer {
 | 
			
		||||
  public testDir: string;
 | 
			
		||||
@ -225,22 +226,7 @@ export class TestServerMerged implements IMochaServer {
 | 
			
		||||
   */
 | 
			
		||||
  public async getDatabase(): Promise<HomeDBManager> {
 | 
			
		||||
    if (!this._dbManager) {
 | 
			
		||||
      const origTypeormDB = process.env.TYPEORM_DATABASE;
 | 
			
		||||
      process.env.TYPEORM_DATABASE = this._getDatabaseFile();
 | 
			
		||||
      this._dbManager = new HomeDBManager();
 | 
			
		||||
      await this._dbManager.connect();
 | 
			
		||||
      await this._dbManager.initializeSpecialIds();
 | 
			
		||||
      if (origTypeormDB) {
 | 
			
		||||
        process.env.TYPEORM_DATABASE = origTypeormDB;
 | 
			
		||||
      }
 | 
			
		||||
      // If this is Sqlite, we are making a separate connection to the database,
 | 
			
		||||
      // so could get busy errors. We bump up our timeout. The rest of Grist could
 | 
			
		||||
      // get busy errors if we do slow writes though.
 | 
			
		||||
      const connection = this._dbManager.connection;
 | 
			
		||||
      const sqlite = connection.driver.options.type === 'sqlite';
 | 
			
		||||
      if (sqlite) {
 | 
			
		||||
        await this._dbManager.connection.query('PRAGMA busy_timeout = 3000');
 | 
			
		||||
      }
 | 
			
		||||
      this._dbManager = await getDatabase(this._getDatabaseFile());
 | 
			
		||||
    }
 | 
			
		||||
    return this._dbManager;
 | 
			
		||||
  }
 | 
			
		||||
 | 
			
		||||
@ -2,10 +2,15 @@ import {ActionSummary} from 'app/common/ActionSummary';
 | 
			
		||||
import {BulkColValues, UserAction} from 'app/common/DocActions';
 | 
			
		||||
import {arrayRepeat} from 'app/common/gutil';
 | 
			
		||||
import {DocState, UserAPIImpl} from 'app/common/UserAPI';
 | 
			
		||||
import {teamFreeFeatures} from 'app/gen-server/entity/Product';
 | 
			
		||||
import {testDailyApiLimitFeatures} from 'app/gen-server/entity/Product';
 | 
			
		||||
import {AddOrUpdateRecord, Record as ApiRecord} from 'app/plugin/DocApiTypes';
 | 
			
		||||
import {CellValue, GristObjCode} from 'app/plugin/GristData';
 | 
			
		||||
import {applyQueryParameters, docDailyApiUsageKey} from 'app/server/lib/DocApi';
 | 
			
		||||
import {
 | 
			
		||||
  applyQueryParameters,
 | 
			
		||||
  docApiUsagePeriods,
 | 
			
		||||
  docPeriodicApiUsageKey,
 | 
			
		||||
  getDocApiUsageKeysToIncr
 | 
			
		||||
} from 'app/server/lib/DocApi';
 | 
			
		||||
import * as log from 'app/server/lib/log';
 | 
			
		||||
import {exitPromise} from 'app/server/lib/serverUtils';
 | 
			
		||||
import {connectTestingHooks, TestingHooksClient} from 'app/server/lib/TestingHooks';
 | 
			
		||||
@ -17,6 +22,8 @@ import {ChildProcess, execFileSync, spawn} from 'child_process';
 | 
			
		||||
import * as FormData from 'form-data';
 | 
			
		||||
import * as fse from 'fs-extra';
 | 
			
		||||
import * as _ from 'lodash';
 | 
			
		||||
import * as LRUCache from 'lru-cache';
 | 
			
		||||
import * as moment from 'moment';
 | 
			
		||||
import fetch from 'node-fetch';
 | 
			
		||||
import {tmpdir} from 'os';
 | 
			
		||||
import * as path from 'path';
 | 
			
		||||
@ -2305,42 +2312,158 @@ function testDocApi() {
 | 
			
		||||
 | 
			
		||||
  describe("Daily API Limit", () => {
 | 
			
		||||
    let redisClient: RedisClient;
 | 
			
		||||
    let workspaceId: number;
 | 
			
		||||
    let freeTeamApi: UserAPIImpl;
 | 
			
		||||
 | 
			
		||||
    before(async function() {
 | 
			
		||||
      if (!process.env.TEST_REDIS_URL) { this.skip(); }
 | 
			
		||||
      redisClient = createClient(process.env.TEST_REDIS_URL);
 | 
			
		||||
      freeTeamApi = makeUserApi('freeteam');
 | 
			
		||||
      workspaceId = await getWorkspaceId(freeTeamApi, 'FreeTeamWs');
 | 
			
		||||
    });
 | 
			
		||||
 | 
			
		||||
    it("limits daily API usage", async function() {
 | 
			
		||||
      // Make a new document in a free team site, currently the only product which limits daily API usage.
 | 
			
		||||
      const docId = await freeTeamApi.newDoc({name: 'TestDoc'}, workspaceId);
 | 
			
		||||
      const key = docDailyApiUsageKey(docId);
 | 
			
		||||
      const limit = teamFreeFeatures.baseMaxApiUnitsPerDocumentPerDay!;
 | 
			
		||||
      // Rather than making 5000 requests, set a high count directly in redis.
 | 
			
		||||
      await redisClient.setAsync(key, String(limit - 2));
 | 
			
		||||
      // Make a new document in a test product with a low daily limit
 | 
			
		||||
      const api = makeUserApi('testdailyapilimit');
 | 
			
		||||
      const workspaceId = await getWorkspaceId(api, 'TestDailyApiLimitWs');
 | 
			
		||||
      const docId = await api.newDoc({name: 'TestDoc1'}, workspaceId);
 | 
			
		||||
      const max = testDailyApiLimitFeatures.baseMaxApiUnitsPerDocumentPerDay;
 | 
			
		||||
 | 
			
		||||
      // Make three requests. The first two should succeed since we set the count to `limit - 2`.
 | 
			
		||||
      // Wait a little after each request to allow time for the local cache to be updated with the redis count.
 | 
			
		||||
      let response = await axios.get(`${serverUrl}/api/docs/${docId}/tables/Table1/records`, chimpy);
 | 
			
		||||
      assert.equal(response.status, 200);
 | 
			
		||||
      await delay(100);
 | 
			
		||||
      for (let i = 1; i <= max + 2; i++) {
 | 
			
		||||
        let success = true;
 | 
			
		||||
        try {
 | 
			
		||||
          // Make some doc request so that it fails or succeeds
 | 
			
		||||
          await api.getTable(docId, "Table1");
 | 
			
		||||
        } catch (e) {
 | 
			
		||||
          success = false;
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
      response = await axios.get(`${serverUrl}/api/docs/${docId}/tables/Table1/records`, chimpy);
 | 
			
		||||
      assert.equal(response.status, 200);
 | 
			
		||||
      await delay(100);
 | 
			
		||||
        // Usually the first `max` requests should succeed and the rest should fail having exceeded the daily limit.
 | 
			
		||||
        // If a new minute starts in the middle of the requests, an extra request will be allowed for that minute.
 | 
			
		||||
        // If a new day starts in the middle of the requests, this test will fail.
 | 
			
		||||
        if (success) {
 | 
			
		||||
          assert.isAtMost(i, max + 1);
 | 
			
		||||
        } else {
 | 
			
		||||
          assert.isAtLeast(i, max + 1);
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    });
 | 
			
		||||
 | 
			
		||||
      // The count should now have reached the limit, and the key should expire in one day.
 | 
			
		||||
      assert.equal(await redisClient.ttlAsync(key), 86400);
 | 
			
		||||
      assert.equal(await redisClient.getAsync(key), String(limit));
 | 
			
		||||
    it("limits daily API usage and sets the correct keys in redis", async function() {
 | 
			
		||||
      // Make a new document in a free team site, currently the only real product which limits daily API usage.
 | 
			
		||||
      const freeTeamApi = makeUserApi('freeteam');
 | 
			
		||||
      const workspaceId = await getWorkspaceId(freeTeamApi, 'FreeTeamWs');
 | 
			
		||||
      const docId = await freeTeamApi.newDoc({name: 'TestDoc2'}, workspaceId);
 | 
			
		||||
      // Rather than making 5000 requests, set high counts directly for the current and next daily and hourly keys
 | 
			
		||||
      const used = 999999;
 | 
			
		||||
      let m = moment.utc();
 | 
			
		||||
      const currentDay = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[0], m);
 | 
			
		||||
      const currentHour = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[1], m);
 | 
			
		||||
      const nextDay = docPeriodicApiUsageKey(docId, false, docApiUsagePeriods[0], m);
 | 
			
		||||
      const nextHour = docPeriodicApiUsageKey(docId, false, docApiUsagePeriods[1], m);
 | 
			
		||||
      await redisClient.multi()
 | 
			
		||||
        .set(currentDay, String(used))
 | 
			
		||||
        .set(currentHour, String(used))
 | 
			
		||||
        .set(nextDay, String(used))
 | 
			
		||||
        .set(nextHour, String(used))
 | 
			
		||||
        .execAsync();
 | 
			
		||||
 | 
			
		||||
      // Making the same request a third time should fail.
 | 
			
		||||
      response = await axios.get(`${serverUrl}/api/docs/${docId}/tables/Table1/records`, chimpy);
 | 
			
		||||
      assert.equal(response.status, 429);
 | 
			
		||||
      assert.deepEqual(response.data, {error: `Exceeded daily limit for document ${docId}`});
 | 
			
		||||
      // Make 9 requests. The first 4 should succeed by fitting into the allocation for the minute.
 | 
			
		||||
      // (Free team plans get 5000 requests per day, and 5000/24/60 ~= 3.47 which is rounded up to 4)
 | 
			
		||||
      // The last request should fail. Don't check the middle 4 in case we're on the boundary of a minute.
 | 
			
		||||
      for (let i = 1; i <= 9; i++) {
 | 
			
		||||
        const last = i === 9;
 | 
			
		||||
        m = moment.utc();  // get this before delaying to calculate accurate keys below
 | 
			
		||||
        const response = await axios.get(`${serverUrl}/api/docs/${docId}/tables/Table1/records`, chimpy);
 | 
			
		||||
        // Allow time for redis to be updated.
 | 
			
		||||
        await delay(100);
 | 
			
		||||
        if (i <= 4) {
 | 
			
		||||
          assert.equal(response.status, 200);
 | 
			
		||||
          // Keys of the periods we expect to be incremented.
 | 
			
		||||
          // For the first request, the server's usage cache is empty and it hasn't seen the redis values.
 | 
			
		||||
          // So it thinks there hasn't been any usage and increments the current day/hour.
 | 
			
		||||
          // After that it increments the next day/hour.
 | 
			
		||||
          // We're only checking this for the first 4 requests
 | 
			
		||||
          // because once the limit is exceeded the counts aren't incremented.
 | 
			
		||||
          const first = i === 1;
 | 
			
		||||
          const day = docPeriodicApiUsageKey(docId, first, docApiUsagePeriods[0], m);
 | 
			
		||||
          const hour = docPeriodicApiUsageKey(docId, first, docApiUsagePeriods[1], m);
 | 
			
		||||
          const minute = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[2], m);
 | 
			
		||||
 | 
			
		||||
          if (!first) {
 | 
			
		||||
            // The first request takes longer to serve because the document gets loaded,
 | 
			
		||||
            // so only check the TTL (which gets set before request processing starts) on subsequent requests.
 | 
			
		||||
            assert.deepEqual(
 | 
			
		||||
              await redisClient.multi()
 | 
			
		||||
                .ttl(minute)
 | 
			
		||||
                .ttl(hour)
 | 
			
		||||
                .ttl(day)
 | 
			
		||||
                .execAsync(),
 | 
			
		||||
              [
 | 
			
		||||
                2 * 60,
 | 
			
		||||
                2 * 60 * 60,
 | 
			
		||||
                2 * 60 * 60 * 24,
 | 
			
		||||
              ],
 | 
			
		||||
            );
 | 
			
		||||
          }
 | 
			
		||||
 | 
			
		||||
          assert.deepEqual(
 | 
			
		||||
            await redisClient.multi()
 | 
			
		||||
              .get(minute)
 | 
			
		||||
              .get(hour)
 | 
			
		||||
              .get(day)
 | 
			
		||||
              .execAsync(),
 | 
			
		||||
            [
 | 
			
		||||
              String(i),
 | 
			
		||||
              String(used + (first ? 1 : i - 1)),
 | 
			
		||||
              String(used + (first ? 1 : i - 1)),
 | 
			
		||||
            ],
 | 
			
		||||
          );
 | 
			
		||||
        }
 | 
			
		||||
 | 
			
		||||
        if (last) {
 | 
			
		||||
          assert.equal(response.status, 429);
 | 
			
		||||
          assert.deepEqual(response.data, {error: `Exceeded daily limit for document ${docId}`});
 | 
			
		||||
        }
 | 
			
		||||
      }
 | 
			
		||||
    });
 | 
			
		||||
 | 
			
		||||
    it("correctly allocates API requests based on the day, hour, and minute", async function() {
 | 
			
		||||
      const m = moment.utc("1999-12-31T23:59:59Z");
 | 
			
		||||
      const docId = "myDocId";
 | 
			
		||||
      const currentDay = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[0], m);
 | 
			
		||||
      const currentHour = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[1], m);
 | 
			
		||||
      const currentMinute = docPeriodicApiUsageKey(docId, true, docApiUsagePeriods[2], m);
 | 
			
		||||
      const nextDay = docPeriodicApiUsageKey(docId, false, docApiUsagePeriods[0], m);
 | 
			
		||||
      const nextHour = docPeriodicApiUsageKey(docId, false, docApiUsagePeriods[1], m);
 | 
			
		||||
      assert.equal(currentDay, `doc-myDocId-periodicApiUsage-1999-12-31`);
 | 
			
		||||
      assert.equal(currentHour, `doc-myDocId-periodicApiUsage-1999-12-31T23`);
 | 
			
		||||
      assert.equal(currentMinute, `doc-myDocId-periodicApiUsage-1999-12-31T23:59`);
 | 
			
		||||
      assert.equal(nextDay, `doc-myDocId-periodicApiUsage-2000-01-01`);
 | 
			
		||||
      assert.equal(nextHour, `doc-myDocId-periodicApiUsage-2000-01-01T00`);
 | 
			
		||||
 | 
			
		||||
      const usage = new LRUCache<string, number>({max: 1024});
 | 
			
		||||
      function check(expected: string[] | undefined) {
 | 
			
		||||
        assert.deepEqual(getDocApiUsageKeysToIncr(docId, usage, dailyMax, m), expected);
 | 
			
		||||
      }
 | 
			
		||||
 | 
			
		||||
      const dailyMax = 5000;
 | 
			
		||||
      const hourlyMax = 209;  // 5000/24    ~= 208.33
 | 
			
		||||
      const minuteMax = 4;    // 5000/24/60 ~= 3.47
 | 
			
		||||
      check([currentDay, currentHour, currentMinute]);
 | 
			
		||||
      usage.set(currentDay, dailyMax - 1);
 | 
			
		||||
      check([currentDay, currentHour, currentMinute]);
 | 
			
		||||
      usage.set(currentDay, dailyMax);
 | 
			
		||||
      check([nextDay, currentHour, currentMinute]);  // used up daily allocation
 | 
			
		||||
      usage.set(currentHour, hourlyMax - 1);
 | 
			
		||||
      check([nextDay, currentHour, currentMinute]);
 | 
			
		||||
      usage.set(currentHour, hourlyMax);
 | 
			
		||||
      check([nextDay, nextHour, currentMinute]);  // used up hourly allocation
 | 
			
		||||
      usage.set(currentMinute, minuteMax - 1);
 | 
			
		||||
      check([nextDay, nextHour, currentMinute]);
 | 
			
		||||
      usage.set(currentMinute, minuteMax);
 | 
			
		||||
      check(undefined);  // used up minutely allocation
 | 
			
		||||
      usage.set(currentDay, 0);
 | 
			
		||||
      check([currentDay, currentHour, currentMinute]);
 | 
			
		||||
      usage.set(currentDay, dailyMax);
 | 
			
		||||
      usage.set(currentHour, 0);
 | 
			
		||||
      check([nextDay, currentHour, currentMinute]);
 | 
			
		||||
    });
 | 
			
		||||
 | 
			
		||||
    after(async function() {
 | 
			
		||||
 | 
			
		||||
							
								
								
									
										23
									
								
								test/testUtils.ts
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										23
									
								
								test/testUtils.ts
									
									
									
									
									
										Normal file
									
								
							@ -0,0 +1,23 @@
 | 
			
		||||
import {HomeDBManager} from 'app/gen-server/lib/HomeDBManager';
 | 
			
		||||
 | 
			
		||||
export async function getDatabase(typeormDb?: string): Promise<HomeDBManager> {
 | 
			
		||||
  const origTypeormDB = process.env.TYPEORM_DATABASE;
 | 
			
		||||
  if (typeormDb) {
 | 
			
		||||
    process.env.TYPEORM_DATABASE = typeormDb;
 | 
			
		||||
  }
 | 
			
		||||
  const db = new HomeDBManager();
 | 
			
		||||
  await db.connect();
 | 
			
		||||
  await db.initializeSpecialIds();
 | 
			
		||||
  if (origTypeormDB) {
 | 
			
		||||
    process.env.TYPEORM_DATABASE = origTypeormDB;
 | 
			
		||||
  }
 | 
			
		||||
  // If this is Sqlite, we are making a separate connection to the database,
 | 
			
		||||
  // so could get busy errors. We bump up our timeout. The rest of Grist could
 | 
			
		||||
  // get busy errors if we do slow writes though.
 | 
			
		||||
  const connection = db.connection;
 | 
			
		||||
  const sqlite = connection.driver.options.type === 'sqlite';
 | 
			
		||||
  if (sqlite) {
 | 
			
		||||
    await db.connection.query('PRAGMA busy_timeout = 3000');
 | 
			
		||||
  }
 | 
			
		||||
  return db;
 | 
			
		||||
}
 | 
			
		||||
@ -1,6 +1,7 @@
 | 
			
		||||
{
 | 
			
		||||
  "extends": "../buildtools/tsconfig-base.json",
 | 
			
		||||
  "include": [
 | 
			
		||||
    "*",
 | 
			
		||||
    "**/*",
 | 
			
		||||
    "../app/server/declarations.d.ts",
 | 
			
		||||
    "../app/server/declarations/**/*.d.ts",
 | 
			
		||||
 | 
			
		||||
		Loading…
	
		Reference in New Issue
	
	Block a user