Simplify lock interface to prep for adding takeLock to state funnel

This commit changes a few things:

* Pass `force` to `takeLock` step directly. This allows us to remove
the `lockFn` used by app manager's action executors, setting takeLock
as the main interface to interact with the update lock module. Note
that this commit by itself will not pass tests, as no update locking
occurs where it once did. This will be amended in the next commit.

* Remove locking functions from doRestart & doPurge, as this is
the only area where skipLock is required.

* Remove `skipLock` interface, as it's redundant with the functionality
of `force`. The only time `skipLock` is true is in doRestart/doPurge,
as those API methods are already run within a lock function. We removed
the lock function which removes the need for skipLock, and in the next
commit we'll add locking as a composition step to replace the
functionality removed here.

* Remove some methods not in use, such as app manager's `stopAll`.

Signed-off-by: Christina Ying Wang <christina@balena.io>
This commit is contained in:
Christina Ying Wang 2024-03-05 23:44:31 -08:00
parent 2f728ee43e
commit cf8d8cedd7
9 changed files with 480 additions and 222 deletions

View File

@ -45,6 +45,7 @@ export interface UpdateState {
containerIds: Dictionary<string>;
downloading: string[];
locksTaken: LocksTakenMap;
force: boolean;
}
interface ChangingPair<T> {
@ -111,8 +112,15 @@ export class App {
// Check to see if we need to polyfill in some "new" data for legacy services
this.migrateLegacy(target);
// Check for changes in the volumes. We don't remove any volumes until we remove an
// entire app
let steps: CompositionStep[] = [];
// Any services which have died get a remove step
for (const service of this.services) {
if (service.status === 'Dead') {
steps.push(generateStep('remove', { current: service }));
}
}
const volumeChanges = this.compareComponents(
this.volumes,
target.volumes,
@ -124,15 +132,6 @@ export class App {
true,
);
let steps: CompositionStep[] = [];
// Any services which have died get a remove step
for (const service of this.services) {
if (service.status === 'Dead') {
steps.push(generateStep('remove', { current: service }));
}
}
const { removePairs, installPairs, updatePairs } = this.compareServices(
this.services,
target.services,
@ -571,7 +570,12 @@ export class App {
current.isEqualConfig(target, context.containerIds)
) {
// Update service metadata or start/stop a service
return this.generateContainerStep(current, target, context.locksTaken);
return this.generateContainerStep(
current,
target,
context.locksTaken,
context.force,
);
}
let strategy: string;
@ -664,6 +668,7 @@ export class App {
current: Service,
target: Service,
locksTaken: LocksTakenMap,
force: boolean,
) {
// Update container metadata if service release has changed
if (current.commit !== target.commit) {
@ -679,6 +684,7 @@ export class App {
return generateStep('takeLock', {
appId: target.appId,
services: [target.serviceName],
force,
});
} else if (target.config.running !== current.config.running) {
if (target.config.running) {

View File

@ -17,7 +17,7 @@ import {
ContractViolationError,
InternalInconsistencyError,
} from '../lib/errors';
import { getServicesLockedByAppId, lock } from '../lib/update-lock';
import { getServicesLockedByAppId } from '../lib/update-lock';
import { checkTruthy } from '../lib/validation';
import App from './app';
@ -64,7 +64,6 @@ export function resetTimeSpentFetching(value: number = 0) {
}
const actionExecutors = getExecutors({
lockFn: lock,
callbacks: {
fetchStart: () => {
fetchesInProgress += 1;
@ -120,6 +119,7 @@ export async function getRequiredSteps(
targetApps: InstancedAppState,
keepImages?: boolean,
keepVolumes?: boolean,
force: boolean = false,
): Promise<CompositionStep[]> {
// get some required data
const [downloading, availableImages, { localMode, delta }] =
@ -146,6 +146,7 @@ export async function getRequiredSteps(
// Volumes are not removed when stopping an app when going to local mode
keepVolumes,
delta,
force,
downloading,
availableImages,
containerIdsByAppId,
@ -161,6 +162,7 @@ export async function inferNextSteps(
keepImages = false,
keepVolumes = false,
delta = true,
force = false,
downloading = [] as UpdateState['downloading'],
availableImages = [] as UpdateState['availableImages'],
containerIdsByAppId = {} as {
@ -219,6 +221,7 @@ export async function inferNextSteps(
containerIds: containerIdsByAppId[id],
downloading,
locksTaken,
force,
},
targetApps[id],
),
@ -233,6 +236,7 @@ export async function inferNextSteps(
downloading,
containerIds: containerIdsByAppId[id],
locksTaken,
force,
}),
);
}
@ -257,6 +261,7 @@ export async function inferNextSteps(
containerIds: containerIdsByAppId[id] ?? {},
downloading,
locksTaken,
force,
},
targetApps[id],
),
@ -301,17 +306,6 @@ export async function inferNextSteps(
return steps;
}
export async function stopAll({ force = false, skipLock = false } = {}) {
const services = await serviceManager.getAll();
await Promise.all(
services.map(async (s) => {
return lock(s.appId, { force, skipLock }, async () => {
await serviceManager.kill(s, { removeContainer: false, wait: true });
});
}),
);
}
// The following two function may look pretty odd, but after the move to uuids,
// there's a chance that the current running apps don't have a uuid set. We
// still need to be able to work on these and perform various state changes. To
@ -501,7 +495,7 @@ function killServicesUsingApi(current: InstancedAppState): CompositionStep[] {
// intermediate targets to perform changes
export async function executeStep(
step: CompositionStep,
{ force = false, skipLock = false } = {},
{ force = false } = {},
): Promise<void> {
if (!validActions.includes(step.action)) {
return Promise.reject(
@ -515,7 +509,6 @@ export async function executeStep(
await actionExecutors[step.action]({
...step,
force,
skipLock,
} as any);
}

View File

@ -1,5 +1,3 @@
import _ from 'lodash';
import * as config from '../config';
import type { Image } from './images';
import * as images from './images';
@ -13,33 +11,22 @@ import * as commitStore from './commit';
import * as updateLock from '../lib/update-lock';
import type { DeviceLegacyReport } from '../types/state';
interface BaseCompositionStepArgs {
force?: boolean;
skipLock?: boolean;
}
// FIXME: Most of the steps take the
// BaseCompositionStepArgs, but some also take an options
// structure which includes some of the same fields. It
// would be nice to remove the need for this
interface CompositionStepArgs {
stop: {
current: Service;
options?: {
skipLock?: boolean;
wait?: boolean;
};
} & BaseCompositionStepArgs;
};
kill: {
current: Service;
options?: {
skipLock?: boolean;
wait?: boolean;
};
} & BaseCompositionStepArgs;
};
remove: {
current: Service;
} & BaseCompositionStepArgs;
};
updateMetadata: {
current: Service;
target: Service;
@ -47,13 +34,10 @@ interface CompositionStepArgs {
restart: {
current: Service;
target: Service;
options?: {
skipLock?: boolean;
};
} & BaseCompositionStepArgs;
};
start: {
target: Service;
} & BaseCompositionStepArgs;
};
updateCommit: {
target: string;
appId: number;
@ -62,10 +46,9 @@ interface CompositionStepArgs {
current: Service;
target: Service;
options?: {
skipLock?: boolean;
timeout?: number;
};
} & BaseCompositionStepArgs;
};
fetch: {
image: Image;
serviceName: string;
@ -94,6 +77,7 @@ interface CompositionStepArgs {
takeLock: {
appId: number;
services: string[];
force: boolean;
};
releaseLock: {
appId: number;
@ -119,13 +103,6 @@ export function generateStep<T extends CompositionStepAction>(
type Executors<T extends CompositionStepAction> = {
[key in T]: (step: CompositionStepT<key>) => Promise<unknown>;
};
type LockingFn = (
// TODO: Once the entire codebase is typescript, change
// this to number
app: number | number[] | null,
args: BaseCompositionStepArgs,
fn: () => Promise<unknown>,
) => Promise<unknown>;
interface CompositionCallbacks {
// TODO: Once the entire codebase is typescript, change
@ -137,38 +114,20 @@ interface CompositionCallbacks {
bestDeltaSource: (image: Image, available: Image[]) => string | null;
}
export function getExecutors(app: {
lockFn: LockingFn;
callbacks: CompositionCallbacks;
}) {
export function getExecutors(app: { callbacks: CompositionCallbacks }) {
const executors: Executors<CompositionStepAction> = {
stop: (step) => {
return app.lockFn(
step.current.appId,
{
force: step.force,
skipLock: step.skipLock || _.get(step, ['options', 'skipLock']),
},
async () => {
const wait = _.get(step, ['options', 'wait'], false);
await serviceManager.kill(step.current, {
removeContainer: false,
wait,
});
},
);
stop: async (step) => {
// Should always be preceded by a takeLock step,
// so the call is executed assuming that the lock is taken.
await serviceManager.kill(step.current, {
removeContainer: false,
wait: step.options?.wait || false,
});
},
kill: (step) => {
return app.lockFn(
step.current.appId,
{
force: step.force,
skipLock: step.skipLock || _.get(step, ['options', 'skipLock']),
},
async () => {
await serviceManager.kill(step.current);
},
);
kill: async (step) => {
// Should always be preceded by a takeLock step,
// so the call is executed assuming that the lock is taken.
await serviceManager.kill(step.current);
},
remove: async (step) => {
// Only called for dead containers, so no need to
@ -180,18 +139,11 @@ export function getExecutors(app: {
// so the call is executed assuming that the lock is taken.
await serviceManager.updateMetadata(step.current, step.target);
},
restart: (step) => {
return app.lockFn(
step.current.appId,
{
force: step.force,
skipLock: step.skipLock || _.get(step, ['options', 'skipLock']),
},
async () => {
await serviceManager.kill(step.current, { wait: true });
await serviceManager.start(step.target);
},
);
restart: async (step) => {
// Should always be preceded by a takeLock step,
// so the call is executed assuming that the lock is taken.
await serviceManager.kill(step.current, { wait: true });
await serviceManager.start(step.target);
},
start: async (step) => {
await serviceManager.start(step.target);
@ -199,17 +151,10 @@ export function getExecutors(app: {
updateCommit: async (step) => {
await commitStore.upsertCommitForApp(step.appId, step.target);
},
handover: (step) => {
return app.lockFn(
step.current.appId,
{
force: step.force,
skipLock: step.skipLock || _.get(step, ['options', 'skipLock']),
},
async () => {
await serviceManager.handover(step.current, step.target);
},
);
handover: async (step) => {
// Should always be preceded by a takeLock step,
// so the call is executed assuming that the lock is taken.
await serviceManager.handover(step.current, step.target);
},
fetch: async (step) => {
const startTime = process.hrtime();
@ -271,7 +216,7 @@ export function getExecutors(app: {
/* async noop */
},
takeLock: async (step) => {
await updateLock.takeLock(step.appId, step.services);
await updateLock.takeLock(step.appId, step.services, step.force);
},
releaseLock: async (step) => {
await updateLock.releaseLock(step.appId);

View File

@ -11,11 +11,11 @@ import * as applicationManager from '../compose/application-manager';
import type { CompositionStepAction } from '../compose/composition-steps';
import { generateStep } from '../compose/composition-steps';
import * as commitStore from '../compose/commit';
import type Service from '../compose/service';
import { getApp } from '../device-state/db-format';
import * as TargetState from '../device-state/target-state';
import log from '../lib/supervisor-console';
import blink = require('../lib/blink');
import { lock } from '../lib/update-lock';
import * as constants from '../lib/constants';
import {
InternalInconsistencyError,
@ -88,32 +88,29 @@ export const regenerateKey = async (oldKey: string) => {
export const doRestart = async (appId: number, force: boolean = false) => {
await deviceState.initialized();
return await lock(appId, { force }, async () => {
const currentState = await deviceState.getCurrentState();
if (currentState.local.apps?.[appId] == null) {
throw new InternalInconsistencyError(
`Application with ID ${appId} is not in the current state`,
);
}
const app = currentState.local.apps[appId];
const services = app.services;
app.services = [];
const currentState = await deviceState.getCurrentState();
if (currentState.local.apps?.[appId] == null) {
throw new InternalInconsistencyError(
`Application with ID ${appId} is not in the current state`,
);
}
return deviceState
.applyIntermediateTarget(currentState, {
skipLock: true,
})
.then(() => {
app.services = services;
return deviceState.applyIntermediateTarget(currentState, {
skipLock: true,
keepVolumes: false,
});
})
.finally(() => {
deviceState.triggerApplyTarget();
});
});
const app = currentState.local.apps[appId];
const services = app.services;
try {
// Set target so that services get deleted
app.services = [];
await deviceState.applyIntermediateTarget(currentState, { force });
// Restore services
app.services = services;
return deviceState.applyIntermediateTarget(currentState, {
keepVolumes: false,
force,
});
} finally {
deviceState.triggerApplyTarget();
}
};
/**
@ -131,47 +128,37 @@ export const doPurge = async (appId: number, force: boolean = false) => {
'Purge data',
);
return await lock(appId, { force }, async () => {
const currentState = await deviceState.getCurrentState();
if (currentState.local.apps?.[appId] == null) {
throw new InternalInconsistencyError(
`Application with ID ${appId} is not in the current state`,
);
}
const currentState = await deviceState.getCurrentState();
if (currentState.local.apps?.[appId] == null) {
throw new InternalInconsistencyError(
`Application with ID ${appId} is not in the current state`,
);
}
// Save & delete the app from the current state
const app = currentState.local.apps[appId];
delete currentState.local.apps[appId];
const app = currentState.local.apps[appId];
// Delete the app from the current state
delete currentState.local.apps[appId];
return deviceState
.applyIntermediateTarget(currentState, {
skipLock: true,
// Purposely tell the apply function to delete volumes so they can get
// deleted even in local mode
keepVolumes: false,
})
.then(() => {
currentState.local.apps[appId] = app;
return deviceState.applyIntermediateTarget(currentState, {
skipLock: true,
});
})
.finally(() => {
deviceState.triggerApplyTarget();
});
})
.then(() =>
logger.logSystemMessage('Purged data', { appId }, 'Purge data success'),
)
.catch((err) => {
logger.logSystemMessage(
`Error purging data: ${err}`,
{ appId, error: err },
'Purge data error',
);
throw err;
try {
// Purposely tell the apply function to delete volumes so
// they can get deleted even in local mode
await deviceState.applyIntermediateTarget(currentState, {
keepVolumes: false,
force,
});
// Restore user app after purge
currentState.local.apps[appId] = app;
await deviceState.applyIntermediateTarget(currentState);
logger.logSystemMessage('Purged data', { appId }, 'Purge data success');
} catch (err: any) {
logger.logSystemMessage(
`Error purging data: ${err}`,
{ appId, error: err?.message ?? err },
'Purge data error',
);
throw err;
} finally {
deviceState.triggerApplyTarget();
}
};
type ClientError = BadRequestError | NotFoundError;
@ -224,6 +211,57 @@ export const executeDeviceAction = async (
});
};
/**
* Used internally by executeServiceAction to handle locks
* around execution of a service action.
*/
const executeDeviceActionWithLock = async ({
action,
appId,
currentService,
targetService,
force = false,
}: {
action: CompositionStepAction;
appId: number;
currentService?: Service;
targetService?: Service;
force: boolean;
}) => {
try {
if (currentService) {
// Take lock for current service to be modified / stopped
await executeDeviceAction(
generateStep('takeLock', {
appId,
services: [currentService.serviceName],
force,
}),
// 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
force,
);
}
// 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,
}),
);
}
};
/**
* Executes a composition step action on a service.
* isLegacy indicates that the action is being called from a legacy (v1) endpoint,
@ -279,15 +317,23 @@ export const executeServiceAction = async ({
throw new NotFoundError(messages.targetServiceNotFound);
}
// Execute action on service
return await executeDeviceAction(
generateStep(action, {
current: currentService,
target: targetService,
wait: true,
}),
force,
);
// A single service start action doesn't require locks
if (action === 'start') {
// Execute action on service
await executeDeviceAction(
generateStep(action, {
target: targetService,
}),
);
} else {
await executeDeviceActionWithLock({
action,
appId,
currentService,
targetService,
force,
});
}
};
/**

View File

@ -574,11 +574,7 @@ export async function shutdown({
// should happen via intermediate targets
export async function executeStepAction(
step: DeviceStateStep<PossibleStepTargets>,
{
force,
initial,
skipLock,
}: { force?: boolean; initial?: boolean; skipLock?: boolean },
{ force, initial }: { force?: boolean; initial?: boolean },
) {
if (deviceConfig.isValidAction(step.action)) {
await deviceConfig.executeStepAction(step as deviceConfig.ConfigStep, {
@ -587,7 +583,6 @@ export async function executeStepAction(
} else if (applicationManager.validActions.includes(step.action)) {
return applicationManager.executeStep(step as any, {
force,
skipLock,
});
} else {
switch (step.action) {
@ -613,11 +608,9 @@ export async function applyStep(
{
force,
initial,
skipLock,
}: {
force?: boolean;
initial?: boolean;
skipLock?: boolean;
},
) {
if (shuttingDown) {
@ -627,7 +620,6 @@ export async function applyStep(
await executeStepAction(step, {
force,
initial,
skipLock,
});
emitAsync('step-completed', null, step);
} catch (e: any) {
@ -685,7 +677,6 @@ export const applyTarget = async ({
force = false,
initial = false,
intermediate = false,
skipLock = false,
nextDelay = 200,
retryCount = 0,
keepVolumes = undefined as boolean | undefined,
@ -724,6 +715,7 @@ export const applyTarget = async ({
// the value
intermediate || undefined,
keepVolumes,
force,
);
if (_.isEmpty(appSteps)) {
@ -769,16 +761,13 @@ export const applyTarget = async ({
}
try {
await Promise.all(
steps.map((s) => applyStep(s, { force, initial, skipLock })),
);
await Promise.all(steps.map((s) => applyStep(s, { force, initial })));
await setTimeout(nextDelay);
await applyTarget({
force,
initial,
intermediate,
skipLock,
nextDelay,
retryCount,
keepVolumes,
@ -884,11 +873,7 @@ export function triggerApplyTarget({
export async function applyIntermediateTarget(
intermediate: InstancedDeviceState,
{
force = false,
skipLock = false,
keepVolumes = undefined as boolean | undefined,
} = {},
{ force = false, keepVolumes = undefined as boolean | undefined } = {},
) {
return pausingApply(async () => {
// TODO: Make sure we don't accidentally overwrite this
@ -897,7 +882,6 @@ export async function applyIntermediateTarget(
return applyTarget({
intermediate: true,
force,
skipLock,
keepVolumes,
}).then(() => {
intermediateTarget = null;

View File

@ -81,7 +81,11 @@ async function dispose(
* 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[]) {
export async function takeLock(
appId: number,
services: string[],
force: boolean = false,
) {
const release = await takeGlobalLockRW(appId);
try {
const actuallyLocked: string[] = [];
@ -93,7 +97,7 @@ export async function takeLock(appId: number, services: string[]) {
);
for (const service of servicesWithoutLock) {
await mkdirp(pathOnRoot(lockPath(appId, service)));
await lockService(appId, service);
await lockService(appId, service, force);
actuallyLocked.push(service);
}
return actuallyLocked;
@ -193,17 +197,14 @@ export function getServicesLockedByAppId(): LocksTakenMap {
* all existing lockfiles before performing the operation
*
* TODO: convert to native Promises and async/await. May require native implementation of Bluebird's dispose / using
*
* TODO: Remove skipLock as it's not a good interface. If lock is called it should try to take the lock
* without an option to skip.
*/
export async function lock<T>(
appId: number | number[],
{ force = false, skipLock = false }: { force: boolean; skipLock?: boolean },
{ force = false }: { force: boolean },
fn: () => Resolvable<T>,
): Promise<T> {
const appIdsToLock = Array.isArray(appId) ? appId : [appId];
if (skipLock || !appId || !appIdsToLock.length) {
if (!appId || !appIdsToLock.length) {
return fn();
}

View File

@ -996,7 +996,7 @@ describe('compose/application-manager', () => {
},
);
// A start step should happen for the depended service first
// A start step should happen for the dependant service first
expect(startStep).to.have.property('action').that.equals('start');
expect(startStep)
.to.have.property('target')

View File

@ -4,6 +4,7 @@ import { stub } from 'sinon';
import Docker from 'dockerode';
import request from 'supertest';
import { setTimeout } from 'timers/promises';
import { testfs } from 'mocha-pod';
import * as deviceState from '~/src/device-state';
import * as config from '~/src/config';
@ -11,10 +12,12 @@ import * as hostConfig from '~/src/host-config';
import * as deviceApi from '~/src/device-api';
import * as actions from '~/src/device-api/actions';
import * as TargetState from '~/src/device-state/target-state';
import * as updateLock from '~/lib/update-lock';
import { pathOnRoot } from '~/lib/host-utils';
import { exec } from '~/lib/fs-utils';
import * as lockfile from '~/lib/lockfile';
import { cleanupDocker } from '~/test-lib/docker-helper';
import { exec } from '~/src/lib/fs-utils';
export async function dbusSend(
dest: string,
path: string,
@ -79,6 +82,7 @@ describe('manages application lifecycle', () => {
const BALENA_SUPERVISOR_ADDRESS =
process.env.BALENA_SUPERVISOR_ADDRESS || 'http://balena-supervisor:48484';
const APP_ID = 1;
const lockdir = pathOnRoot(updateLock.BASE_LOCK_DIR);
const docker = new Docker();
const getSupervisorTarget = async () =>
@ -218,6 +222,11 @@ describe('manages application lifecycle', () => {
ctns.every(({ State }) => !startedAt.includes(State.StartedAt));
};
const mockFs = testfs(
{ [`${lockdir}/${APP_ID}`]: {} },
{ cleanup: [`${lockdir}/${APP_ID}/**/*.lock`] },
);
before(async () => {
// Images are ignored in local mode so we need to pull the base image
await docker.pull(BASE_IMAGE);
@ -251,10 +260,16 @@ describe('manages application lifecycle', () => {
});
beforeEach(async () => {
await mockFs.enable();
// Create a single-container application in local mode
await setSupervisorTarget(targetState);
});
afterEach(async () => {
await mockFs.restore();
});
// Make sure the app is running and correct before testing more assertions
it('should setup a single container app (sanity check)', async () => {
containers = await waitForSetup(targetState);
@ -292,6 +307,69 @@ describe('manages application lifecycle', () => {
);
});
it('should not restart an application when user locks are present', async () => {
containers = await waitForSetup(targetState);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post(`/v1/restart`)
.set('Content-Type', 'application/json')
.send(JSON.stringify({ appId: APP_ID }))
.expect(423);
// Containers should not have been restarted
const containersAfterRestart = await waitForSetup(targetState);
expect(
containersAfterRestart.map((ctn) => ctn.State.StartedAt),
).to.deep.include.members(containers.map((ctn) => ctn.State.StartedAt));
// Remove the lock
await lockfile.unlock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
});
it('should restart an application when user locks are present if force is specified', async () => {
containers = await waitForSetup(targetState);
const isRestartSuccessful = startTimesChanged(
containers.map((ctn) => ctn.State.StartedAt),
);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post(`/v1/restart`)
.set('Content-Type', 'application/json')
.send(JSON.stringify({ appId: APP_ID, force: true }));
const restartedContainers = await waitForSetup(
targetState,
isRestartSuccessful,
);
// Technically the wait function above should already verify that the two
// containers have been restarted, but verify explcitly with an assertion
expect(isRestartSuccessful(restartedContainers)).to.be.true;
// Containers should have different Ids since they're recreated
expect(restartedContainers.map(({ Id }) => Id)).to.not.have.members(
containers.map((ctn) => ctn.Id),
);
// Wait briefly for state to settle which includes releasing locks
await setTimeout(1000);
// User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]);
});
it('should restart service by removing and recreating corresponding container', async () => {
containers = await waitForSetup(targetState);
const isRestartSuccessful = startTimesChanged(
@ -321,6 +399,73 @@ describe('manages application lifecycle', () => {
);
});
// Since restart-service follows the same code paths as start|stop-service,
// these lock test cases should be sufficient to cover all three service actions.
it('should not restart service when user locks are present', async () => {
containers = await waitForSetup(targetState);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post('/v2/applications/1/restart-service')
.set('Content-Type', 'application/json')
.send(JSON.stringify({ serviceName: serviceNames[0] }))
.expect(423);
// Containers should not have been restarted
const containersAfterRestart = await waitForSetup(targetState);
expect(
containersAfterRestart.map((ctn) => ctn.State.StartedAt),
).to.deep.include.members(containers.map((ctn) => ctn.State.StartedAt));
// Remove the lock
await lockfile.unlock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
});
it('should restart service when user locks are present if force is specified', async () => {
containers = await waitForSetup(targetState);
const isRestartSuccessful = startTimesChanged(
containers
.filter((ctn) => ctn.Name.includes(serviceNames[0]))
.map((ctn) => ctn.State.StartedAt),
);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post('/v2/applications/1/restart-service')
.set('Content-Type', 'application/json')
.send(JSON.stringify({ serviceName: serviceNames[0], force: true }));
const restartedContainers = await waitForSetup(
targetState,
isRestartSuccessful,
);
// Technically the wait function above should already verify that the two
// containers have been restarted, but verify explcitly with an assertion
expect(isRestartSuccessful(restartedContainers)).to.be.true;
// Containers should have different Ids since they're recreated
expect(restartedContainers.map(({ Id }) => Id)).to.not.have.members(
containers.map((ctn) => ctn.Id),
);
// Wait briefly for state to settle which includes releasing locks
await setTimeout(1000);
// User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]);
});
it('should stop a running service', async () => {
containers = await waitForSetup(targetState);
@ -520,10 +665,15 @@ describe('manages application lifecycle', () => {
});
beforeEach(async () => {
await mockFs.enable();
// Create a multi-container application in local mode
await setSupervisorTarget(targetState);
});
afterEach(async () => {
await mockFs.restore();
});
// Make sure the app is running and correct before testing more assertions
it('should setup a multi-container app (sanity check)', async () => {
containers = await waitForSetup(targetState);
@ -560,6 +710,69 @@ describe('manages application lifecycle', () => {
);
});
it('should not restart an application when user locks are present', async () => {
containers = await waitForSetup(targetState);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post(`/v1/restart`)
.set('Content-Type', 'application/json')
.send(JSON.stringify({ appId: APP_ID }))
.expect(423);
// Containers should not have been restarted
const containersAfterRestart = await waitForSetup(targetState);
expect(
containersAfterRestart.map((ctn) => ctn.State.StartedAt),
).to.deep.include.members(containers.map((ctn) => ctn.State.StartedAt));
// Remove the lock
await lockfile.unlock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
});
it('should restart an application when user locks are present if force is specified', async () => {
containers = await waitForSetup(targetState);
const isRestartSuccessful = startTimesChanged(
containers.map((ctn) => ctn.State.StartedAt),
);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post(`/v1/restart`)
.set('Content-Type', 'application/json')
.send(JSON.stringify({ appId: APP_ID, force: true }));
const restartedContainers = await waitForSetup(
targetState,
isRestartSuccessful,
);
// Technically the wait function above should already verify that the two
// containers have been restarted, but verify explcitly with an assertion
expect(isRestartSuccessful(restartedContainers)).to.be.true;
// Containers should have different Ids since they're recreated
expect(restartedContainers.map(({ Id }) => Id)).to.not.have.members(
containers.map((ctn) => ctn.Id),
);
// Wait briefly for state to settle which includes releasing locks
await setTimeout(500);
// User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]);
});
it('should restart service by removing and recreating corresponding container', async () => {
containers = await waitForSetup(targetState);
const serviceName = serviceNames[0];
@ -601,6 +814,73 @@ describe('manages application lifecycle', () => {
expect(sharedIds.length).to.equal(1);
});
// Since restart-service follows the same code paths as start|stop-service,
// these lock test cases should be sufficient to cover all three service actions.
it('should not restart service when user locks are present', async () => {
containers = await waitForSetup(targetState);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post('/v2/applications/1/restart-service')
.set('Content-Type', 'application/json')
.send(JSON.stringify({ serviceName: serviceNames[0] }))
.expect(423);
// Containers should not have been restarted
const containersAfterRestart = await waitForSetup(targetState);
expect(
containersAfterRestart.map((ctn) => ctn.State.StartedAt),
).to.deep.include.members(containers.map((ctn) => ctn.State.StartedAt));
// Remove the lock
await lockfile.unlock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
});
it('should restart service when user locks are present if force is specified', async () => {
containers = await waitForSetup(targetState);
const isRestartSuccessful = startTimesChanged(
containers
.filter((ctn) => ctn.Name.includes(serviceNames[0]))
.map((ctn) => ctn.State.StartedAt),
);
// Create a lock
await lockfile.lock(
`${lockdir}/${APP_ID}/${serviceNames[0]}/updates.lock`,
);
await request(BALENA_SUPERVISOR_ADDRESS)
.post('/v2/applications/1/restart-service')
.set('Content-Type', 'application/json')
.send(JSON.stringify({ serviceName: serviceNames[0], force: true }));
const restartedContainers = await waitForSetup(
targetState,
isRestartSuccessful,
);
// Technically the wait function above should already verify that the two
// containers have been restarted, but verify explcitly with an assertion
expect(isRestartSuccessful(restartedContainers)).to.be.true;
// Containers should have different Ids since they're recreated
expect(restartedContainers.map(({ Id }) => Id)).to.not.have.members(
containers.map((ctn) => ctn.Id),
);
// Wait briefly for state to settle which includes releasing locks
await setTimeout(500);
// User lock should be overridden
expect(await updateLock.getLocksTaken()).to.deep.equal([]);
});
it('should stop a running service', async () => {
containers = await waitForSetup(targetState);

View File

@ -15,6 +15,7 @@ import {
const defaultContext = {
keepVolumes: false,
force: false,
availableImages: [] as Image[],
containerIds: {},
downloading: [] as string[],
@ -151,6 +152,7 @@ describe('compose/app', () => {
}),
],
volumes: [Volume.fromComposeObject('test-volume', 1, 'deadbeef')],
networks: [DEFAULT_NETWORK],
});
const target = createApp({
services: [
@ -164,6 +166,7 @@ describe('compose/app', () => {
labels: { test: 'test' },
}),
],
networks: [DEFAULT_NETWORK],
isTarget: true,
});
@ -453,9 +456,10 @@ describe('compose/app', () => {
const steps = current.nextStepsForAppUpdate(defaultContext, target);
const [removeNetworkStep] = expectSteps('kill', steps);
const [killStep] = expectSteps('kill', steps);
console.log(killStep);
expect(removeNetworkStep).to.have.property('current').that.deep.includes({
expect(killStep).to.have.property('current').that.deep.includes({
serviceName: 'test',
});
});
@ -576,7 +580,7 @@ describe('compose/app', () => {
services: [
await createService({
serviceName: 'test',
composition: { networks: ['test-network'] },
composition: { networks: { 'test-network': {} } },
}),
],
networks: [Network.fromComposeObject('test-network', 1, 'appuuid', {})],
@ -977,7 +981,6 @@ describe('compose/app', () => {
// Take lock before updating metadata
const steps = current.nextStepsForAppUpdate(defaultContext, target);
const [takeLockStep] = expectSteps('takeLock', steps);
expect(takeLockStep).to.have.property('appId').that.equals(1);
expect(takeLockStep)
.to.have.property('services')
.that.deep.equals(['main']);
@ -1080,7 +1083,7 @@ describe('compose/app', () => {
.to.have.property('current')
.that.deep.includes({ serviceName: 'main' });
// assume the intermediate step has already removed the app
// Assume the intermediate step has already removed the app
const intermediate = createApp({
services: [],
// Default network was already created
@ -1092,7 +1095,6 @@ describe('compose/app', () => {
contextWithImages,
target,
);
const [startStep] = expectSteps('start', stepsToTarget);
expect(startStep)
.to.have.property('target')
@ -1173,7 +1175,6 @@ describe('compose/app', () => {
{ ...contextWithImages, ...{ containerIds: { dep: 'dep-id' } } },
target,
);
const [startMainStep] = expectSteps('start', stepsToTarget);
expect(startMainStep)
.to.have.property('target')
@ -1470,6 +1471,7 @@ describe('compose/app', () => {
commit: 'old-release',
}),
],
networks: [DEFAULT_NETWORK],
});
const target = createApp({
services: [
@ -1479,6 +1481,7 @@ describe('compose/app', () => {
commit: 'new-release',
}),
],
networks: [DEFAULT_NETWORK],
isTarget: true,
});