Refactor update-locks implementation

The refactor simplifies the implementation and ensures that locks per
app can only be held by one supervisor task at the time.

Change-type: patch
This commit is contained in:
Felipe Lalanne 2024-11-19 19:01:25 -03:00
parent d8f54c05e7
commit 3c6e9dd209
No known key found for this signature in database
GPG Key ID: 03E696BFD472B26A
16 changed files with 965 additions and 1202 deletions

View File

@ -77,9 +77,6 @@ fi
# not a problem. # not a problem.
modprobe ip6_tables || true modprobe ip6_tables || true
export BASE_LOCK_DIR="/tmp/balena-supervisor/services"
export LOCKFILE_UID=65534
if [ "${LIVEPUSH}" = "1" ]; then if [ "${LIVEPUSH}" = "1" ]; then
exec npx nodemon --watch src --watch typings --ignore tests -e js,ts,json \ exec npx nodemon --watch src --watch typings --ignore tests -e js,ts,json \
--exec node -r ts-node/register/transpile-only src/app.ts --exec node -r ts-node/register/transpile-only src/app.ts

View File

@ -170,7 +170,7 @@ class AppImpl implements App {
if (services.size > 0) { if (services.size > 0) {
steps.push( steps.push(
generateStep('takeLock', { generateStep('takeLock', {
appId: parseInt(appId, 10), appId,
services: Array.from(services), services: Array.from(services),
force: state.force, force: state.force,
}), }),
@ -200,19 +200,14 @@ class AppImpl implements App {
}), }),
); );
} }
// Current & target should be the same appId, but one of either current
// or target may not have any services, so we need to check both // If the app still has a lock, release it
const allServices = this.services.concat(target.services); if (state.lock != null) {
if (
allServices.length > 0 &&
allServices.some((s) =>
state.locksTaken.isLocked(s.appId, s.serviceName),
)
) {
// Release locks for all services before settling state // Release locks for all services before settling state
steps.push( steps.push(
generateStep('releaseLock', { generateStep('releaseLock', {
appId: target.appId, appId: target.appId,
lock: state.lock,
}), }),
); );
} }
@ -225,11 +220,7 @@ class AppImpl implements App {
): CompositionStep[] { ): CompositionStep[] {
if (Object.keys(this.services).length > 0) { if (Object.keys(this.services).length > 0) {
// Take all locks before killing // Take all locks before killing
if ( if (state.lock == null) {
this.services.some(
(svc) => !state.locksTaken.isLocked(svc.appId, svc.serviceName),
)
) {
return [ return [
generateStep('takeLock', { generateStep('takeLock', {
appId: this.appId, appId: this.appId,
@ -628,9 +619,7 @@ class AppImpl implements App {
appsToLock: AppsToLockMap; appsToLock: AppsToLockMap;
} & UpdateState, } & UpdateState,
): CompositionStep[] { ): CompositionStep[] {
const servicesLocked = this.services const servicesLocked = context.lock != null;
.concat(context.targetApp.services)
.every((svc) => context.locksTaken.isLocked(svc.appId, svc.serviceName));
if (current?.status === 'Stopping') { if (current?.status === 'Stopping') {
// There's a kill step happening already, emit a noop to ensure // There's a kill step happening already, emit a noop to ensure
// we stay alive while this happens // we stay alive while this happens

View File

@ -12,7 +12,7 @@ import * as contracts from '../lib/contracts';
import * as constants from '../lib/constants'; import * as constants from '../lib/constants';
import log from '../lib/supervisor-console'; import log from '../lib/supervisor-console';
import { InternalInconsistencyError } from '../lib/errors'; import { InternalInconsistencyError } from '../lib/errors';
import { getServicesLockedByAppId, LocksTakenMap } from '../lib/update-lock'; import type { Lock } from '../lib/update-lock';
import { checkTruthy } from '../lib/validation'; import { checkTruthy } from '../lib/validation';
import { App } from './app'; import { App } from './app';
@ -61,6 +61,13 @@ export function resetTimeSpentFetching(value: number = 0) {
timeSpentFetching = value; timeSpentFetching = value;
} }
interface LockRegistry {
[appId: string | number]: Lock;
}
// In memory registry of all app locks
const lockRegistry: LockRegistry = {};
const actionExecutors = getExecutors({ const actionExecutors = getExecutors({
callbacks: { callbacks: {
fetchStart: () => { fetchStart: () => {
@ -76,6 +83,12 @@ const actionExecutors = getExecutors({
reportCurrentState(state); reportCurrentState(state);
}, },
bestDeltaSource, bestDeltaSource,
registerLock: (appId: string | number, lock: Lock) => {
lockRegistry[appId.toString()] = lock;
},
unregisterLock: (appId: string | number) => {
delete lockRegistry[appId.toString()];
},
}, },
}); });
@ -134,10 +147,21 @@ export async function getRequiredSteps(
downloading, downloading,
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
locksTaken: await getServicesLockedByAppId(), appLocks: lockRegistry,
}); });
} }
interface InferNextOpts {
keepImages: boolean;
keepVolumes: boolean;
delta: boolean;
force: boolean;
downloading: UpdateState['downloading'];
availableImages: UpdateState['availableImages'];
containerIdsByAppId: { [appId: number]: UpdateState['containerIds'] };
appLocks: LockRegistry;
}
// Calculate the required steps from the current to the target state // Calculate the required steps from the current to the target state
export async function inferNextSteps( export async function inferNextSteps(
currentApps: InstancedAppState, currentApps: InstancedAppState,
@ -147,13 +171,11 @@ export async function inferNextSteps(
keepVolumes = false, keepVolumes = false,
delta = true, delta = true,
force = false, force = false,
downloading = [] as UpdateState['downloading'], downloading = [],
availableImages = [] as UpdateState['availableImages'], availableImages = [],
containerIdsByAppId = {} as { containerIdsByAppId = {},
[appId: number]: UpdateState['containerIds']; appLocks = {},
}, }: Partial<InferNextOpts>,
locksTaken = new LocksTakenMap(),
} = {},
) { ) {
const currentAppIds = Object.keys(currentApps).map((i) => parseInt(i, 10)); const currentAppIds = Object.keys(currentApps).map((i) => parseInt(i, 10));
const targetAppIds = Object.keys(targetApps).map((i) => parseInt(i, 10)); const targetAppIds = Object.keys(targetApps).map((i) => parseInt(i, 10));
@ -215,8 +237,8 @@ export async function inferNextSteps(
availableImages, availableImages,
containerIds: containerIdsByAppId[id], containerIds: containerIdsByAppId[id],
downloading, downloading,
locksTaken,
force, force,
lock: appLocks[id],
}, },
targetApps[id], targetApps[id],
), ),
@ -230,8 +252,8 @@ export async function inferNextSteps(
keepVolumes, keepVolumes,
downloading, downloading,
containerIds: containerIdsByAppId[id], containerIds: containerIdsByAppId[id],
locksTaken,
force, force,
lock: appLocks[id],
}), }),
); );
} }
@ -255,8 +277,8 @@ export async function inferNextSteps(
availableImages, availableImages,
containerIds: containerIdsByAppId[id] ?? {}, containerIds: containerIdsByAppId[id] ?? {},
downloading, downloading,
locksTaken,
force, force,
lock: appLocks[id],
}, },
targetApps[id], targetApps[id],
), ),

View File

@ -5,9 +5,10 @@ import * as serviceManager from './service-manager';
import * as networkManager from './network-manager'; import * as networkManager from './network-manager';
import * as volumeManager from './volume-manager'; import * as volumeManager from './volume-manager';
import * as commitStore from './commit'; import * as commitStore from './commit';
import * as updateLock from '../lib/update-lock'; import { Lockable } from '../lib/update-lock';
import type { DeviceLegacyReport } from '../types/state'; import type { DeviceLegacyReport } from '../types/state';
import type { CompositionStepAction, CompositionStepT } from './types'; import type { CompositionStepAction, CompositionStepT } from './types';
import type { Lock } from '../lib/update-lock';
export type { export type {
CompositionStep, CompositionStep,
@ -27,6 +28,8 @@ interface CompositionCallbacks {
fetchTime: (time: number) => void; fetchTime: (time: number) => void;
stateReport: (state: DeviceLegacyReport) => void; stateReport: (state: DeviceLegacyReport) => void;
bestDeltaSource: (image: Image, available: Image[]) => string | null; bestDeltaSource: (image: Image, available: Image[]) => string | null;
registerLock: (appId: string | number, lock: Lock) => void;
unregisterLock: (appId: string | number) => void;
} }
export function generateStep<T extends CompositionStepAction>( export function generateStep<T extends CompositionStepAction>(
@ -141,10 +144,14 @@ export function getExecutors(app: { callbacks: CompositionCallbacks }) {
/* async noop */ /* async noop */
}, },
takeLock: async (step) => { takeLock: async (step) => {
await updateLock.takeLock(step.appId, step.services, step.force); const lockable = Lockable.from(step.appId, step.services);
const lockOverride = await config.get('lockOverride');
const lock = await lockable.lock({ force: step.force || lockOverride });
app.callbacks.registerLock(step.appId, lock);
}, },
releaseLock: async (step) => { releaseLock: async (step) => {
await updateLock.releaseLock(step.appId); app.callbacks.unregisterLock(step.appId);
await step.lock.unlock();
}, },
}; };

View File

@ -1,7 +1,7 @@
import type { Network } from './network'; import type { Network } from './network';
import type { Volume } from './volume'; import type { Volume } from './volume';
import type { Service } from './service'; import type { Service } from './service';
import type { LocksTakenMap } from '../../lib/update-lock'; import type { Lock } from '../../lib/update-lock';
import type { Image } from './image'; import type { Image } from './image';
import type { CompositionStep } from './composition-step'; import type { CompositionStep } from './composition-step';
@ -9,7 +9,7 @@ export interface UpdateState {
availableImages: Image[]; availableImages: Image[];
containerIds: Dictionary<string>; containerIds: Dictionary<string>;
downloading: string[]; downloading: string[];
locksTaken: LocksTakenMap; lock: Lock | null;
force: boolean; force: boolean;
} }

View File

@ -2,6 +2,7 @@ import type { Image } from './image';
import type { Service } from './service'; import type { Service } from './service';
import type { Network } from './network'; import type { Network } from './network';
import type { Volume } from './volume'; import type { Volume } from './volume';
import type { Lock } from '../../lib/update-lock';
export interface CompositionStepArgs { export interface CompositionStepArgs {
stop: { stop: {
@ -67,12 +68,13 @@ export interface CompositionStepArgs {
ensureSupervisorNetwork: object; ensureSupervisorNetwork: object;
noop: object; noop: object;
takeLock: { takeLock: {
appId: number; appId: string | number;
services: string[]; services: string[];
force: boolean; force: boolean;
}; };
releaseLock: { releaseLock: {
appId: number; appId: string | number;
lock: Lock;
}; };
} }

View File

@ -26,6 +26,7 @@ import {
NotFoundError, NotFoundError,
BadRequestError, BadRequestError,
} from '../lib/errors'; } from '../lib/errors';
import { withLock } from '../lib/update-lock';
/** /**
* Run an array of healthchecks, outputting whether all passed or not * Run an array of healthchecks, outputting whether all passed or not
@ -233,39 +234,24 @@ const executeDeviceActionWithLock = async ({
targetService?: Service; targetService?: Service;
force: boolean; force: boolean;
}) => { }) => {
try { const lockOverride = await config.get('lockOverride');
if (currentService) { await withLock(
const lockOverride = await config.get('lockOverride'); appId,
// Take lock for current service to be modified / stopped async () => {
// Execute action on service
await executeDeviceAction( await executeDeviceAction(
generateStep('takeLock', { generateStep(action, {
appId, current: currentService,
services: [currentService.serviceName], target: targetService,
force: force || lockOverride, wait: true,
}), }),
// FIXME: deviceState.executeStepAction only accepts force as a separate arg // FIXME: deviceState.executeStepAction only accepts force as a separate arg
// instead of reading force from the step object, so we have to pass it twice // instead of reading force from the step object, so we have to pass it twice
force || lockOverride, force,
); );
} },
{ force: force || lockOverride },
// Execute action on service );
await executeDeviceAction(
generateStep(action, {
current: currentService,
target: targetService,
wait: true,
}),
force,
);
} finally {
// Release lock regardless of action success to prevent leftover lockfile
await executeDeviceAction(
generateStep('releaseLock', {
appId,
}),
);
}
}; };
/** /**

View File

@ -456,22 +456,26 @@ export async function shutdown({
const apps = await applicationManager.getCurrentApps(); const apps = await applicationManager.getCurrentApps();
const appIds = Object.keys(apps).map((strId) => parseInt(strId, 10)); const appIds = Object.keys(apps).map((strId) => parseInt(strId, 10));
// Try to create a lock for all the services before shutting down // Try to create a lock for all the services before shutting down
return updateLock.lock(appIds, { force }, async () => { return updateLock.withLock(
let dbusAction; appIds,
switch (reboot) { async () => {
case true: let dbusAction;
logger.logSystemMessage('Rebooting', {}, 'Reboot'); switch (reboot) {
dbusAction = await dbus.reboot(); case true:
break; logger.logSystemMessage('Rebooting', {}, 'Reboot');
case false: dbusAction = await dbus.reboot();
logger.logSystemMessage('Shutting down', {}, 'Shutdown'); break;
dbusAction = await dbus.shutdown(); case false:
break; logger.logSystemMessage('Shutting down', {}, 'Shutdown');
} dbusAction = await dbus.shutdown();
shuttingDown = true; break;
emitAsync('shutdown', undefined); }
return dbusAction; shuttingDown = true;
}); emitAsync('shutdown', undefined);
return dbusAction;
},
{ force },
);
} }
// FIXME: this method should not be exported, all target state changes // FIXME: this method should not be exported, all target state changes

View File

@ -103,25 +103,29 @@ export async function patch(
const { noProxy, ...targetConf } = conf.network.proxy; const { noProxy, ...targetConf } = conf.network.proxy;
// It's possible for appIds to be an empty array, but patch shouldn't fail // It's possible for appIds to be an empty array, but patch shouldn't fail
// as it's not dependent on there being any running user applications. // as it's not dependent on there being any running user applications.
return updateLock.lock(appIds, { force }, async () => { return updateLock.withLock(
const proxyConf = await readProxy(); appIds,
let currentConf: ProxyConfig | undefined = undefined; async () => {
if (proxyConf) { const proxyConf = await readProxy();
delete proxyConf.noProxy; let currentConf: ProxyConfig | undefined = undefined;
currentConf = proxyConf; if (proxyConf) {
} delete proxyConf.noProxy;
currentConf = proxyConf;
}
// Merge current & target redsocks.conf // Merge current & target redsocks.conf
const patchedConf = patchProxy( const patchedConf = patchProxy(
{ {
redsocks: currentConf, redsocks: currentConf,
}, },
{ {
redsocks: targetConf, redsocks: targetConf,
}, },
); );
await setProxy(patchedConf, noProxy); await setProxy(patchedConf, noProxy);
}); },
{ force },
);
} }
} }

View File

@ -26,9 +26,12 @@ interface FindAllArgs {
recursive: boolean; recursive: boolean;
} }
// Returns all current locks taken under a directory (default: /tmp) /**
// Optionally accepts filter function for only getting locks that match a condition. * Find all existing lockfiles under a given root directory (defaults to /mp)
// A file is counted as a lock by default if it ends with `.lock`. *
* Optionally accepts filter function for only getting locks that match a condition.
* It will recursively look for all locks unless `recursive` is set to `false`
*/
export async function findAll({ export async function findAll({
root = '/tmp', root = '/tmp',
filter = (l) => l.path.endsWith('.lock'), filter = (l) => l.path.endsWith('.lock'),
@ -96,6 +99,11 @@ export class LockfileExistsError implements ChildProcessError {
} }
} }
/**
* Lock the file provided as path
*
* Optionally accepts a user id to lock the path as
*/
export async function lock(path: string, uid: number = os.userInfo().uid) { export async function lock(path: string, uid: number = os.userInfo().uid) {
/** /**
* Set parent directory permissions to `drwxrwxrwt` (octal 1777), which are needed * Set parent directory permissions to `drwxrwxrwt` (octal 1777), which are needed
@ -145,6 +153,9 @@ export async function lock(path: string, uid: number = os.userInfo().uid) {
} }
} }
/**
* Removes the lock indicated by the path
*/
export async function unlock(path: string): Promise<void> { export async function unlock(path: string): Promise<void> {
// Removing the lockfile releases the lock // Removing the lockfile releases the lock
await fs.unlink(path).catch((e) => { await fs.unlink(path).catch((e) => {

View File

@ -1,32 +1,27 @@
import { promises as fs } from 'fs'; import { promises as fs } from 'fs';
import path from 'path'; import path from 'path';
import { isRight } from 'fp-ts/lib/Either'; import { setTimeout } from 'timers/promises';
import type ReadWriteLock from 'rwlock';
import { import { isENOENT, UpdatesLockedError } from './errors';
isENOENT,
UpdatesLockedError,
InternalInconsistencyError,
} from './errors';
import { pathOnRoot, pathExistsOnState } from './host-utils'; import { pathOnRoot, pathExistsOnState } from './host-utils';
import { mkdirp } from './fs-utils'; import { mkdirp } from './fs-utils';
import * as config from '../config';
import * as lockfile from './lockfile'; import * as lockfile from './lockfile';
import { NumericIdentifier, StringIdentifier, DockerName } from '../types';
import { takeGlobalLockRW } from './process-lock'; import { takeGlobalLockRW } from './process-lock';
import * as logger from '../logger';
import * as logTypes from './log-types'; import * as logTypes from './log-types';
import * as logger from '../logger';
const decodedUid = NumericIdentifier.decode(process.env.LOCKFILE_UID); export const LOCKFILE_UID = 65534;
export const LOCKFILE_UID = isRight(decodedUid) ? decodedUid.right : 65534; export const BASE_LOCK_DIR = '/tmp/balena-supervisor/services';
export const BASE_LOCK_DIR = export function lockPath(appId: string | number, serviceName?: string): string {
process.env.BASE_LOCK_DIR || '/tmp/balena-supervisor/services';
export function lockPath(appId: number, serviceName?: string): string {
return path.join(BASE_LOCK_DIR, appId.toString(), serviceName ?? ''); return path.join(BASE_LOCK_DIR, appId.toString(), serviceName ?? '');
} }
function lockFilesOnHost(appId: number, serviceName: string): string[] { function lockFilesOnHost(
appId: string | number,
serviceName: string,
): string[] {
return pathOnRoot( return pathOnRoot(
...['updates.lock', 'resin-updates.lock'].map((filename) => ...['updates.lock', 'resin-updates.lock'].map((filename) =>
path.join(lockPath(appId), serviceName, filename), path.join(lockPath(appId), serviceName, filename),
@ -40,308 +35,297 @@ function lockFilesOnHost(appId: number, serviceName: string): string[] {
* prevent reboot. If the Supervisor reboots while those services are still running, * prevent reboot. If the Supervisor reboots while those services are still running,
* the device may become stuck in an invalid state during HUP. * the device may become stuck in an invalid state during HUP.
*/ */
export function abortIfHUPInProgress({ export async function abortIfHUPInProgress({
force = false, force = false,
}: { }: {
force: boolean | undefined; force?: boolean;
}): Promise<boolean | never> { }): Promise<boolean | never> {
return Promise.all( const breadcrumbs = await Promise.all(
['rollback-health-breadcrumb', 'rollback-altboot-breadcrumb'].map( ['rollback-health-breadcrumb', 'rollback-altboot-breadcrumb'].map(
(filename) => pathExistsOnState(filename), (filename) => pathExistsOnState(filename),
), ),
).then((existsArray) => { );
const anyExists = existsArray.some((e) => e);
if (anyExists && !force) { const hasHUPBreadcrumb = breadcrumbs.some((e) => e);
throw new UpdatesLockedError('Waiting for Host OS update to finish'); if (hasHUPBreadcrumb && !force) {
} throw new UpdatesLockedError('Waiting for Host OS update to finish');
return anyExists; }
});
return hasHUPBreadcrumb;
} }
/** interface LockingOpts {
* Unlock all lockfiles of an appId | appUuid, then release resources. /**
* Meant for use in update-lock module only as as it assumes that a * Delete existing user locks if any
* write lock has been acquired. */
*/ force: boolean;
async function dispose(
appIdentifier: string | number,
release: () => void,
): Promise<void> {
try {
const locks = await getLocksTaken(
pathOnRoot(`${BASE_LOCK_DIR}/${appIdentifier}`),
);
// Try to unlock all locks taken
await Promise.all(locks.map((l) => lockfile.unlock(l)));
} finally {
// Release final resource
release();
}
}
/**
* Composition step used by Supervisor compose module.
* Take all locks for an appId | appUuid, creating directories if they don't exist.
*/
export async function takeLock(
appId: number,
services: string[],
force: boolean = false,
) {
logger.logSystemEvent(logTypes.takeLock, {
appId,
services,
force,
});
const release = await takeGlobalLockRW(appId);
let lockOverride: boolean;
try {
lockOverride = await config.get('lockOverride');
} catch (err: any) {
throw new InternalInconsistencyError(
`Error getting lockOverride config value: ${err?.message ?? err}`,
);
}
try {
const actuallyLocked: string[] = [];
const locksTaken = await getServicesLockedByAppId();
// Filter out services that already have Supervisor-taken locks.
// This needs to be done after taking the appId write lock to avoid
// race conditions with locking.
const servicesWithoutLock = services.filter(
(svc) => !locksTaken.isLocked(appId, svc),
);
for (const service of servicesWithoutLock) {
await mkdirp(pathOnRoot(lockPath(appId, service)));
await lockService(appId, service, force || lockOverride);
actuallyLocked.push(service);
}
return actuallyLocked;
} catch (err) {
// If something errors while taking the lock, we should remove any
// lockfiles that may have been created so that all services return
// to unlocked status.
await dispose(appId, release);
// Re-throw error to be handled in caller
throw err;
} finally {
// If not already released from catch, released the RW process lock.
// If already released, this will not error.
release();
}
}
/**
* Composition step used by Supervisor compose module.
* Release all locks for an appId | appUuid.
*/
export async function releaseLock(appId: number) {
logger.logSystemEvent(logTypes.releaseLock, { appId });
const release = await takeGlobalLockRW(appId);
await dispose(appId, release);
}
/**
* Given a lockfile path `p`, return an array [appId, serviceName, filename] of that path.
* Paths are assumed to end in the format /:appId/:serviceName/(resin-)updates.lock.
*/
function getIdentifiersFromPath(p: string) {
const parts = p.split('/');
const filename = parts.pop();
if (filename?.match(/updates\.lock/) === null) {
return [];
}
const serviceName = parts.pop();
const appId = parts.pop();
return [appId, serviceName, filename];
}
type LockedEntity = { appId: number; services: string[] };
/**
* A map of locked services by appId.
* Exported for tests only; getServicesLockedByAppId is the public generator interface.
*/
export class LocksTakenMap extends Map<number, Set<string>> {
constructor(lockedEntities: LockedEntity[] = []) {
// Construct a Map<number, Set<string>> from user-friendly input args
super(
lockedEntities.map(({ appId, services }) => [appId, new Set(services)]),
);
}
// Add one or more locked services to an appId
public add(appId: number, services: string | string[]): void {
if (typeof services === 'string') {
services = [services];
}
if (this.has(appId)) {
const lockedSvcs = this.get(appId)!;
services.forEach((s) => lockedSvcs.add(s));
} else {
this.set(appId, new Set(services));
}
}
/** /**
* @private Use this.getServices instead as there is no need to return * If the locks are being held by another operation
* a mutable reference to the internal Set data structure. * on the supervisor, this is the max time that the call
* will wait before throwing
*/ */
public get(appId: number): Set<string> | undefined { maxWaitMs: number;
return super.get(appId);
}
// Return an array copy of locked services under an appId
public getServices(appId: number): string[] {
return this.has(appId) ? Array.from(this.get(appId)!) : [];
}
// Return whether a service is locked under an appId
public isLocked(appId: number, service: string): boolean {
return this.has(appId) && this.get(appId)!.has(service);
}
} }
// A wrapper function for lockfile.getLocksTaken that filters for Supervisor-taken locks. async function takeGlobalLockOrFail(
// Exported for tests only; getServicesLockedByAppId is the intended public interface. appId: string,
export async function getLocksTaken( maxWaitMs = 0, // 0 === wait forever
rootDir: string = pathOnRoot(BASE_LOCK_DIR), ): Promise<ReadWriteLock.Release> {
): Promise<string[]> { let abort = false;
return await lockfile.findAll({ const lockingPromise = takeGlobalLockRW(appId).then((disposer) => {
root: rootDir, // Even if the timer resolves first, takeGlobalLockRW
filter: (l) => l.path.endsWith('updates.lock') && l.owner === LOCKFILE_UID, // will eventually resolve and in that case we need to call the disposer
// to avoid a deadlock
if (abort) {
disposer();
return () => {
/* noop */
};
} else {
return disposer;
}
}); });
}
/** const promises: Array<Promise<ReadWriteLock.Release | string>> = [
* Return a list of services that are locked by the Supervisor under each appId. lockingPromise,
* Both `resin-updates.lock` and `updates.lock` should be present per ];
* [appId, serviceName] pair for a service to be considered locked.
*/ const ac = new AbortController();
export async function getServicesLockedByAppId(): Promise<LocksTakenMap> { const signal = ac.signal;
const locksTaken = await getLocksTaken(); if (maxWaitMs > 0) {
// Group locksTaken paths by appId & serviceName. promises.push(setTimeout(maxWaitMs, 'abort', { signal }));
// filesTakenByAppId is of type Map<appId, Map<serviceName, Set<filename>>>
// and represents files taken under every [appId, serviceName] pair.
const filesTakenByAppId = new Map<number, Map<string, Set<string>>>();
for (const lockTakenPath of locksTaken) {
const [appId, serviceName, filename] =
getIdentifiersFromPath(lockTakenPath);
if (
!StringIdentifier.is(appId) ||
!DockerName.is(serviceName) ||
!filename?.match(/updates\.lock/)
) {
continue;
}
const numAppId = +appId;
if (!filesTakenByAppId.has(numAppId)) {
filesTakenByAppId.set(numAppId, new Map());
}
const servicesTaken = filesTakenByAppId.get(numAppId)!;
if (!servicesTaken.has(serviceName)) {
servicesTaken.set(serviceName, new Set());
}
servicesTaken.get(serviceName)!.add(filename);
} }
// Construct a LocksTakenMap from filesTakenByAppId, which represents try {
// services locked by the Supervisor. const res = await Promise.race(promises);
const servicesByAppId = new LocksTakenMap(); if (res === 'abort') {
for (const [appId, servicesTaken] of filesTakenByAppId) { abort = true;
for (const [serviceName, filenames] of servicesTaken) { throw new UpdatesLockedError(
if ( `Locks for app ${appId} are being held by another supervisor operation`,
filenames.has('resin-updates.lock') && );
filenames.has('updates.lock') }
) { return lockingPromise;
servicesByAppId.add(appId, serviceName); } finally {
// Clear the timeout
ac.abort();
}
}
export interface Lock {
unlock(): Promise<void>;
}
export interface Lockable {
lock(opts?: Partial<LockingOpts>): Promise<Lock>;
}
function newLockable(appId: string, services: string[]): Lockable {
async function unlockApp(locks: string[], release: () => void) {
try {
logger.logSystemEvent(logTypes.releaseLock, { appId });
await Promise.all(locks.map((l) => lockfile.unlock(l)));
} finally {
release();
}
}
async function lockApp({
force = false,
maxWaitMs = 0,
}: Partial<LockingOpts> = {}) {
// Log before taking the global lock to detect
// possible deadlocks
logger.logSystemEvent(logTypes.takeLock, {
appId,
services,
force,
});
// Try to take the global lock for the given appId
// this ensures the user app locks are only taken in a single
// place on the supervisor
const release: ReadWriteLock.Release = await takeGlobalLockOrFail(
appId,
maxWaitMs,
);
// This keeps a list of all locks currently being
// held by the lockable
let currentLocks: string[] = [];
try {
// Find all the locks already taken for the appId
// if this is not empty it probably means these locks are from
// a previous run of the supervisor
currentLocks = await lockfile.findAll({
root: pathOnRoot(lockPath(appId)),
filter: (l) =>
l.path.endsWith('updates.lock') && l.owner === LOCKFILE_UID,
});
// Group locks by service
const locksByService = services.map((service) => {
const existing = currentLocks.filter((lockFile) =>
lockFilesOnHost(appId, service).includes(lockFile),
);
return { service, existing };
}, {});
// Filter out services that already have Supervisor-taken locks.
// This needs to be done after taking the appId write lock to avoid
// race conditions with locking.
const servicesWithMissingLocks = locksByService.filter(
({ existing }) => existing.length < 2,
);
// For every service that has yet to be fully locked we need to
// take the locks
for (const { service, existing } of servicesWithMissingLocks) {
// Create the directory if it doesn't exist
await mkdirp(pathOnRoot(lockPath(appId, service)));
// We will only take those locks that have not yet been taken by
// the supervisor
const missingLocks = lockFilesOnHost(appId, service).filter(
(l) => !existing.includes(l),
);
for await (const file of missingLocks) {
try {
if (force) {
// If force: true we remove the lock first
await lockfile.unlock(file);
}
await lockfile.lock(file, LOCKFILE_UID);
// If locking was successful, we update the current locks
// list
currentLocks.push(file);
} catch (e) {
if (lockfile.LockfileExistsError.is(e)) {
// Throw more descriptive error
throw new UpdatesLockedError(
`Lockfile exists for { appId: ${appId}, service: ${service} }`,
);
}
// Otherwise just throw the error
throw e;
}
}
} }
return {
unlock: async () => {
await unlockApp(currentLocks, release);
},
};
} catch (err) {
// If any error happens while taking locks, we need to unlock
// to avoid some locks being held by user apps and some by
// the supervisor
await unlockApp(currentLocks, release);
// Re-throw error to be handled in caller
throw err;
} }
} }
return servicesByAppId;
return { lock: lockApp };
} }
export const Lockable = {
from(appId: string | number, services: string[]) {
// Convert appId to string so we always
// use the same value when taking the process lock
return newLockable(appId.toString(), services);
},
};
/** /**
* Try to take the locks for an application. If force is set, it will remove * Call the given function after locks for the given apps
* all existing lockfiles before performing the operation * have been taken.
* *
* TODO: convert to native Promises and async/await. May require native implementation of Bluebird's dispose / using * This is compatible with Lockable.lock() in the sense that only one locking
* operation is allowed at the time on the supervisor
*
* By default the call will wait 10 seconds for shared process locks before
* giving up
*/ */
export async function lock<T>( export async function withLock<T>(
appId: number | number[], appIds: number | number[],
{ force = false }: { force: boolean },
fn: () => Resolvable<T>, fn: () => Resolvable<T>,
{ force = false, maxWaitMs = 10 * 1000 }: Partial<LockingOpts> = {},
): Promise<T> { ): Promise<T> {
const appIdsToLock = Array.isArray(appId) ? appId : [appId]; appIds = Array.isArray(appIds) ? appIds : [appIds];
if (!appId || !appIdsToLock.length) { if (appIds.length === 0) {
return fn(); return fn();
} }
// Sort appIds so they are always locked in the same sequence // Always lock in the same order
const sortedIds = appIdsToLock.sort(); appIds = appIds.sort();
let lockOverride: boolean; const locks: Lock[] = [];
try { try {
lockOverride = await config.get('lockOverride'); const lockables: Lockable[] = [];
} catch (err: any) { for (const appId of appIds) {
throw new InternalInconsistencyError( const appLockDir = pathOnRoot(lockPath(appId));
`Error getting lockOverride config value: ${err?.message ?? err}`, const services: string[] = [];
); try {
} // Read the contents of the app lockdir
const dirs = await fs.readdir(appLockDir);
const statResults = await Promise.allSettled(
dirs.map(async (service) => ({
service,
stat: await fs.lstat(path.join(appLockDir, service)),
})),
);
const releases = new Map<number, () => void>(); for (const res of statResults) {
try { // Ignore rejected results
for (const id of sortedIds) { if (
const lockDir = pathOnRoot(lockPath(id)); res.status === 'fulfilled' &&
// Acquire write lock for appId res.value.stat.isDirectory() &&
releases.set(id, await takeGlobalLockRW(id)); !res.value.stat.isSymbolicLink()
// Get list of service folders in lock directory ) {
const serviceFolders = await fs.readdir(lockDir).catch((e) => { services.push(res.value.service);
}
}
} catch (e) {
// If the directory does not exist, continue
if (isENOENT(e)) { if (isENOENT(e)) {
return []; continue;
} }
throw e; throw e;
}); }
// Attempt to create a lock for each service
for (const service of serviceFolders) { lockables.push(Lockable.from(appId, services));
await lockService(id, service, force || lockOverride); }
// Lock all apps at once to avoid adding up the wait times
const lockingResults = await Promise.allSettled(
lockables.map((l) => l.lock({ force, maxWaitMs })),
);
let err = null;
for (const res of lockingResults) {
if (res.status === 'fulfilled') {
locks.push(res.value);
} else {
err = res.reason;
} }
} }
// Resolve the function passed
// If there are any errors, then throw before calling the function
if (err != null) {
throw err;
}
// If we got here, then all locks were successfully acquired
// call the function now
return await fn(); return await fn();
} finally { } finally {
for (const [id, release] of releases.entries()) { // Unlock all taken locks
// Try to dispose all the locks await Promise.all(locks.map((l) => l.unlock()));
await dispose(id, release);
}
}
}
async function lockService(
appId: number,
service: string,
force: boolean = false,
): Promise<void> {
const serviceLockFiles = lockFilesOnHost(appId, service);
for await (const file of serviceLockFiles) {
try {
if (force) {
await lockfile.unlock(file);
}
await lockfile.lock(file, LOCKFILE_UID);
} catch (e) {
if (lockfile.LockfileExistsError.is(e)) {
// Throw more descriptive error
throw new UpdatesLockedError(
`Lockfile exists for { appId: ${appId}, service: ${service} }`,
);
}
// Otherwise just throw the error
throw e;
}
} }
} }

View File

@ -8,7 +8,6 @@ import { Network } from '~/src/compose/network';
import * as networkManager from '~/src/compose/network-manager'; import * as networkManager from '~/src/compose/network-manager';
import { Volume } from '~/src/compose/volume'; import { Volume } from '~/src/compose/volume';
import * as config from '~/src/config'; import * as config from '~/src/config';
import { LocksTakenMap } from '~/lib/update-lock';
import { createDockerImage } from '~/test-lib/docker-helper'; import { createDockerImage } from '~/test-lib/docker-helper';
import { import {
createService, createService,
@ -113,7 +112,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock lock taken to avoid takeLock step // Mock lock taken to avoid takeLock step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -225,7 +230,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock lock taken to avoid takeLock step // Mock lock taken to avoid takeLock step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -277,7 +288,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock lock taken to avoid takeLock step // Mock lock taken to avoid takeLock step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -410,7 +427,13 @@ describe('compose/application-manager', () => {
availableImages: c1.availableImages, availableImages: c1.availableImages,
containerIdsByAppId: c1.containerIdsByAppId, containerIdsByAppId: c1.containerIdsByAppId,
// Mock lock taken for `main` service which just needs metadata updated // Mock lock taken for `main` service which just needs metadata updated
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
// There should be two noop steps, one for target service which is still downloading, // There should be two noop steps, one for target service which is still downloading,
@ -457,10 +480,13 @@ describe('compose/application-manager', () => {
downloading, downloading,
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken for all services in either current or target state appLocks: {
locksTaken: new LocksTakenMap([ '1': {
{ appId: 1, services: ['old', 'main', 'new'] }, async unlock() {
]), /* noop */
},
},
},
}, },
); );
// Service `old` is safe to kill after download for `new` has completed // Service `old` is safe to kill after download for `new` has completed
@ -506,11 +532,14 @@ describe('compose/application-manager', () => {
// to avoid removeImage steps // to avoid removeImage steps
availableImages: [], availableImages: [],
containerIdsByAppId: c1.containerIdsByAppId, containerIdsByAppId: c1.containerIdsByAppId,
// Mock locks for service to be updated via updateMetadata // Mock locks to avoid takeLock step
// or kill to avoid takeLock step appLocks: {
locksTaken: new LocksTakenMap([ '1': {
{ appId: 1, services: ['old', 'main', 'new'] }, async unlock() {
]), /* noop */
},
},
},
}, },
); );
// Service `new` should be fetched // Service `new` should be fetched
@ -583,10 +612,13 @@ describe('compose/application-manager', () => {
}), }),
], ],
containerIdsByAppId: c1.containerIdsByAppId, containerIdsByAppId: c1.containerIdsByAppId,
// Mock lock taken for all services in target state appLocks: {
locksTaken: new LocksTakenMap([ '1': {
{ appId: 1, services: ['old', 'main', 'new'] }, async unlock() {
]), /* noop */
},
},
},
}, },
); );
// Service `new` should be started // Service `new` should be started
@ -629,9 +661,13 @@ describe('compose/application-manager', () => {
containerIdsByAppId: c1.containerIdsByAppId, containerIdsByAppId: c1.containerIdsByAppId,
// Mock locks for service to be updated via updateMetadata // Mock locks for service to be updated via updateMetadata
// or kill to avoid takeLock step // or kill to avoid takeLock step
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['old', 'main', 'new'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
// Service `new` should be fetched // Service `new` should be fetched
@ -705,9 +741,13 @@ describe('compose/application-manager', () => {
], ],
containerIdsByAppId: c1.containerIdsByAppId, containerIdsByAppId: c1.containerIdsByAppId,
// Mock lock taken for all services in target state // Mock lock taken for all services in target state
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['main', 'new'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
// Service `new` should be started // Service `new` should be started
@ -805,9 +845,13 @@ describe('compose/application-manager', () => {
], ],
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken for all services in target state // Mock locks taken for all services in target state
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('start', steps3, 2); expectSteps('start', steps3, 2);
@ -877,7 +921,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock lock taken to avoid takeLock step // Mock lock taken to avoid takeLock step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -959,9 +1009,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken to avoid takeLock step // Mock locks taken to avoid takeLock step
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['main', 'dep'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1028,9 +1082,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken to avoid takeLock step // Mock locks taken to avoid takeLock step
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['main', 'dep'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1100,9 +1158,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken to avoid takeLock step // Mock locks taken to avoid takeLock step
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['main', 'dep'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1146,10 +1208,18 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock lock already taken for the new and leftover services // Mock lock already taken for the new and leftover services
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 5, services: ['old-service'] }, '1': {
{ appId: 1, services: ['main'] }, async unlock() {
]), /* noop */
},
},
'5': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1665,7 +1735,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken to avoid takeLock step // Mock locks taken to avoid takeLock step
locksTaken: new LocksTakenMap([{ appId: 2, services: ['main'] }]), appLocks: {
'2': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1747,10 +1823,18 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken to avoid takeLock step // Mock locks taken to avoid takeLock step
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['main'] }, '1': {
{ appId: 2, services: ['main'] }, async unlock() {
]), /* noop */
},
},
'2': {
async unlock() {
/* noop */
},
},
},
}, },
); );
@ -1816,9 +1900,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps2, 2); expectSteps('kill', steps2, 2);
@ -1875,9 +1963,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('stop', steps2, 2); expectSteps('stop', steps2, 2);
@ -1934,9 +2026,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('start', steps2, 2); expectSteps('start', steps2, 2);
@ -2100,9 +2196,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps4, 2); expectSteps('kill', steps4, 2);
@ -2172,9 +2272,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps2, 2); expectSteps('kill', steps2, 2);
@ -2231,7 +2335,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps2); expectSteps('kill', steps2);
@ -2254,7 +2364,13 @@ describe('compose/application-manager', () => {
availableImages: intermediateCurrent.availableImages, availableImages: intermediateCurrent.availableImages,
containerIdsByAppId: intermediateCurrent.containerIdsByAppId, containerIdsByAppId: intermediateCurrent.containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('removeNetwork', steps3); expectSteps('removeNetwork', steps3);
@ -2320,7 +2436,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps2, 1); expectSteps('kill', steps2, 1);
@ -2343,7 +2465,13 @@ describe('compose/application-manager', () => {
availableImages: intermediateCurrent.availableImages, availableImages: intermediateCurrent.availableImages,
containerIdsByAppId: intermediateCurrent.containerIdsByAppId, containerIdsByAppId: intermediateCurrent.containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('removeNetwork', steps3); expectSteps('removeNetwork', steps3);
@ -2405,7 +2533,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('kill', steps2, 1); expectSteps('kill', steps2, 1);
@ -2424,7 +2558,13 @@ describe('compose/application-manager', () => {
availableImages: intermediateCurrent.availableImages, availableImages: intermediateCurrent.availableImages,
containerIdsByAppId: intermediateCurrent.containerIdsByAppId, containerIdsByAppId: intermediateCurrent.containerIdsByAppId,
// Mock locks taken // Mock locks taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), appLocks: {
'1': {
async unlock() {
/* noop */
},
},
},
}, },
); );
expectSteps('removeVolume', steps3); expectSteps('removeVolume', steps3);
@ -2456,9 +2596,13 @@ describe('compose/application-manager', () => {
downloading, downloading,
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}, },
); );
const [releaseLockStep] = expectSteps('releaseLock', steps, 1, 1); const [releaseLockStep] = expectSteps('releaseLock', steps, 1, 1);
@ -2950,9 +3094,13 @@ describe('compose/application-manager', () => {
availableImages, availableImages,
containerIdsByAppId, containerIdsByAppId,
// Mock locks taken for all services in target state // Mock locks taken for all services in target state
locksTaken: new LocksTakenMap([ appLocks: {
{ appId: 1, services: ['one', 'two', 'three', 'four'] }, '1': {
]), async unlock() {
/* noop */
},
},
},
}); });
[startStep1, startStep2, startStep3, startStep4].forEach((step) => { [startStep1, startStep2, startStep3, startStep4].forEach((step) => {

View File

@ -5,6 +5,7 @@ import Docker from 'dockerode';
import request from 'supertest'; import request from 'supertest';
import { setTimeout } from 'timers/promises'; import { setTimeout } from 'timers/promises';
import { testfs } from 'mocha-pod'; import { testfs } from 'mocha-pod';
import { promises as fs } from 'fs';
import * as deviceState from '~/src/device-state'; import * as deviceState from '~/src/device-state';
import * as config from '~/src/config'; import * as config from '~/src/config';
@ -378,7 +379,9 @@ describe('manages application lifecycle', () => {
await setTimeout(1000); await setTimeout(1000);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart an application when user locks are present if lockOverride is specified', async () => { it('should restart an application when user locks are present if lockOverride is specified', async () => {
@ -423,7 +426,9 @@ describe('manages application lifecycle', () => {
await setTimeout(1000); await setTimeout(1000);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart service by removing and recreating corresponding container', async () => { it('should restart service by removing and recreating corresponding container', async () => {
@ -519,7 +524,9 @@ describe('manages application lifecycle', () => {
await setTimeout(1000); await setTimeout(1000);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart service when user locks are present if lockOverride is specified', async () => { it('should restart service when user locks are present if lockOverride is specified', async () => {
@ -566,7 +573,9 @@ describe('manages application lifecycle', () => {
await setTimeout(1000); await setTimeout(1000);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should stop a running service', async () => { it('should stop a running service', async () => {
@ -873,7 +882,9 @@ describe('manages application lifecycle', () => {
await setTimeout(500); await setTimeout(500);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart an application when user locks are present if lockOverride is specified', async () => { it('should restart an application when user locks are present if lockOverride is specified', async () => {
@ -918,7 +929,9 @@ describe('manages application lifecycle', () => {
await setTimeout(500); await setTimeout(500);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart service by removing and recreating corresponding container', async () => { it('should restart service by removing and recreating corresponding container', async () => {
@ -1026,7 +1039,9 @@ describe('manages application lifecycle', () => {
await setTimeout(500); await setTimeout(500);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should restart service when user locks are present if lockOverride is specified', async () => { it('should restart service when user locks are present if lockOverride is specified', async () => {
@ -1073,7 +1088,9 @@ describe('manages application lifecycle', () => {
await setTimeout(500); await setTimeout(500);
// User lock should be overridden // User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]); await expect(
fs.readdir(pathOnRoot(updateLock.lockPath(1, 'server'))),
).to.eventually.deep.equal([]);
}); });
it('should stop a running service', async () => { it('should stop a running service', async () => {

File diff suppressed because it is too large Load Diff

View File

@ -2,7 +2,7 @@ import { expect } from 'chai';
import type { Image } from '~/src/compose/images'; import type { Image } from '~/src/compose/images';
import { Network } from '~/src/compose/network'; import { Network } from '~/src/compose/network';
import { Volume } from '~/src/compose/volume'; import { Volume } from '~/src/compose/volume';
import { LocksTakenMap } from '~/lib/update-lock'; import type { Lock } from '~/lib/update-lock';
import { import {
createService, createService,
@ -19,7 +19,13 @@ const defaultContext = {
availableImages: [] as Image[], availableImages: [] as Image[],
containerIds: {}, containerIds: {},
downloading: [] as string[], downloading: [] as string[],
locksTaken: new LocksTakenMap(), lock: null,
};
const mockLock: Lock = {
async unlock() {
/* noop */
},
}; };
describe('compose/app', () => { describe('compose/app', () => {
@ -190,7 +196,7 @@ describe('compose/app', () => {
...defaultContext, ...defaultContext,
availableImages, availableImages,
// Mock lock already taken // Mock lock already taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -287,7 +293,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks already taken // Mock locks already taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
intermediateTarget, intermediateTarget,
); );
@ -299,7 +305,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks already taken // Mock locks already taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
intermediateTarget, intermediateTarget,
); );
@ -362,7 +368,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks already taken // Mock locks already taken
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -525,7 +531,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -553,7 +559,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -678,7 +684,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -806,7 +812,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -879,9 +885,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['one', 'two'] },
]),
}, },
target, target,
); );
@ -958,9 +962,7 @@ describe('compose/app', () => {
{ {
...defaultContext, ...defaultContext,
availableImages, availableImages,
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['one', 'two'] },
]),
}, },
target, target,
); );
@ -1108,9 +1110,7 @@ describe('compose/app', () => {
...defaultContext, ...defaultContext,
availableImages: [createImage({ serviceName: 'main' })], availableImages: [createImage({ serviceName: 'main' })],
// Mock locks already taken // Mock locks already taken
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['main', 'aux'] },
]),
}, },
target, target,
); );
@ -1247,7 +1247,7 @@ describe('compose/app', () => {
const steps2 = current.nextStepsForAppUpdate( const steps2 = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1282,7 +1282,7 @@ describe('compose/app', () => {
const steps2 = current.nextStepsForAppUpdate( const steps2 = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1356,7 +1356,7 @@ describe('compose/app', () => {
const stepsToIntermediate = current.nextStepsForAppUpdate( const stepsToIntermediate = current.nextStepsForAppUpdate(
{ {
...contextWithImages, ...contextWithImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1376,7 +1376,7 @@ describe('compose/app', () => {
const stepsToTarget = intermediate.nextStepsForAppUpdate( const stepsToTarget = intermediate.nextStepsForAppUpdate(
{ {
...contextWithImages, ...contextWithImages,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1435,9 +1435,7 @@ describe('compose/app', () => {
const contextWithImages = { const contextWithImages = {
...defaultContext, ...defaultContext,
...{ availableImages }, ...{ availableImages },
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['main', 'dep'] },
]),
}; };
// Only one start step and it should be that of the 'dep' service // Only one start step and it should be that of the 'dep' service
@ -1555,7 +1553,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks taken before kill // Mock locks taken before kill
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1575,7 +1573,7 @@ describe('compose/app', () => {
...contextWithImages, ...contextWithImages,
// Mock locks still taken after kill (releaseLock not // Mock locks still taken after kill (releaseLock not
// yet inferred as state is not yet settled) // yet inferred as state is not yet settled)
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1703,7 +1701,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks taken from previous step // Mock locks taken from previous step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1719,7 +1717,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks taken from previous step // Mock locks taken from previous step
locksTaken: new LocksTakenMap([{ appId: 1, services: ['main'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1762,7 +1760,7 @@ describe('compose/app', () => {
const steps2 = current.nextStepsForAppUpdate( const steps2 = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['test'] }]), lock: mockLock,
}, },
target, target,
); );
@ -1849,9 +1847,7 @@ describe('compose/app', () => {
{ {
...contextWithImages, ...contextWithImages,
// Mock locks already taken // Mock locks already taken
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['one', 'two', 'three'] },
]),
}, },
target, target,
); );
@ -2029,9 +2025,7 @@ describe('compose/app', () => {
serviceName: 'other', serviceName: 'other',
}), }),
], ],
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['main', 'other'] },
]),
}, },
target, target,
); );
@ -2115,9 +2109,7 @@ describe('compose/app', () => {
const steps = current.nextStepsForAppUpdate( const steps = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['server', 'client'] },
]),
}, },
target, target,
); );
@ -2128,7 +2120,7 @@ describe('compose/app', () => {
const steps2 = current.nextStepsForAppUpdate( const steps2 = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([{ appId: 1, services: ['server'] }]), lock: mockLock,
}, },
target, target,
); );
@ -2173,10 +2165,7 @@ describe('compose/app', () => {
const steps = current.nextStepsForAppUpdate( const steps = current.nextStepsForAppUpdate(
{ {
...defaultContext, ...defaultContext,
locksTaken: new LocksTakenMap([ lock: mockLock,
{ appId: 1, services: ['server', 'client'] },
{ appId: 2, services: ['main'] },
]),
}, },
target, target,
); );

View File

@ -1,58 +0,0 @@
import { expect } from 'chai';
import * as path from 'path';
import * as updateLock from '~/lib/update-lock';
describe('lib/update-lock: unit tests', () => {
describe('lockPath', () => {
it('should return path prefix of service lockfiles on host', () => {
expect(updateLock.lockPath(123)).to.equal(
path.join(updateLock.BASE_LOCK_DIR, '123'),
);
expect(updateLock.lockPath(123, 'main')).to.equal(
path.join(updateLock.BASE_LOCK_DIR, '123', 'main'),
);
});
});
describe('LocksTakenMap', () => {
it('should be an instance of Map<number, Set<string>>', () => {
const map = new updateLock.LocksTakenMap();
expect(map).to.be.an.instanceof(Map);
});
it('should add services while ignoring duplicates', () => {
const map = new updateLock.LocksTakenMap();
map.add(123, 'main');
expect(map.getServices(123)).to.deep.include.members(['main']);
map.add(123, 'main');
expect(map.getServices(123)).to.deep.include.members(['main']);
map.add(123, ['main', 'aux']);
expect(map.getServices(123)).to.deep.include.members(['main', 'aux']);
});
it('should track any number of appIds', () => {
const map = new updateLock.LocksTakenMap();
map.add(123, 'main');
map.add(456, ['aux', 'dep']);
expect(map.getServices(123)).to.deep.include.members(['main']);
expect(map.getServices(456)).to.deep.include.members(['aux', 'dep']);
expect(map.size).to.equal(2);
});
it('should return empty array for non-existent appIds', () => {
const map = new updateLock.LocksTakenMap();
expect(map.getServices(123)).to.deep.equal([]);
});
it('should return whether a service is locked under an appId', () => {
const map = new updateLock.LocksTakenMap();
map.add(123, 'main');
expect(map.isLocked(123, 'main')).to.be.true;
expect(map.isLocked(123, 'aux')).to.be.false;
expect(map.isLocked(456, 'main')).to.be.false;
});
});
});