[Fleet] Prevent concurrent runs of Fleet setup (#183636)

Closes https://github.com/elastic/ingest-dev/issues/3346

- [x] Unit and integration tests are created or updated
- [x] Turn down info logging

The linked issue seems to be caused by multiple kibana instances running
Fleet setup at the same time, trying to create the preconfigured cloud
policy concurrently, and in case of failures, the agent policy is left
with a revision with no inputs, this way preventing fleet-server to
start properly.

See the concurrent errors in the logs:
https://platform-logging.kb.us-west2.gcp.elastic-cloud.com/app/r/s/tUpMP

This fix introduces a `fleet-setup-lock` SO type, which is used to
create a document as a lock by Fleet setup, and is deleted when the
setup is completed. Concurrent calls to Fleet setup will return early if
this doc exists.

To verify:
Run the test `./run_fleet_setup_parallel.sh` from local kibana, and
verify the generated logs that only one of them ran Fleet setup.

---------

Co-authored-by: kibanamachine <42973632+kibanamachine@users.noreply.github.com>
This commit is contained in:
Julia Bardi 2024-05-31 16:38:51 +02:00 committed by GitHub
parent 3701edc6a2
commit 464f797a73
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
24 changed files with 565 additions and 16 deletions

View file

@ -457,6 +457,11 @@
"proxy_headers",
"url"
],
"fleet-setup-lock": [
"started_at",
"status",
"uuid"
],
"fleet-uninstall-tokens": [
"policy_id",
"token_plain"

View file

@ -1546,6 +1546,19 @@
}
}
},
"fleet-setup-lock": {
"properties": {
"started_at": {
"type": "date"
},
"status": {
"type": "keyword"
},
"uuid": {
"type": "text"
}
}
},
"fleet-uninstall-tokens": {
"dynamic": false,
"properties": {

7
run_fleet_setup_parallel.sh Executable file
View file

@ -0,0 +1,7 @@
node scripts/jest_integration.js x-pack/plugins/fleet/server/integration_tests/es.test.ts &
sleep 5
node scripts/jest_integration.js x-pack/plugins/fleet/server/integration_tests/fleet_setup.test.ts &
node scripts/jest_integration.js x-pack/plugins/fleet/server/integration_tests/fleet_setup.test.ts &
node scripts/jest_integration.js x-pack/plugins/fleet/server/integration_tests/fleet_setup.test.ts &
exit 0

View file

@ -101,6 +101,7 @@ describe('checking migration metadata changes on all registered SO types', () =>
"fleet-message-signing-keys": "93421f43fed2526b59092a4e3c65d64bc2266c0f",
"fleet-preconfiguration-deletion-record": "c52ea1e13c919afe8a5e8e3adbb7080980ecc08e",
"fleet-proxy": "6cb688f0d2dd856400c1dbc998b28704ff70363d",
"fleet-setup-lock": "0dc784792c79b5af5a6e6b5dcac06b0dbaa90bde",
"fleet-uninstall-tokens": "ed8aa37e3cdd69e4360709e64944bb81cae0c025",
"graph-workspace": "5cc6bb1455b078fd848c37324672163f09b5e376",
"guided-onboarding-guide-state": "d338972ed887ac480c09a1a7fbf582d6a3827c91",

View file

@ -69,6 +69,7 @@ const previouslyRegisteredTypes = [
'fleet-preconfiguration-deletion-record',
'fleet-proxy',
'fleet-uninstall-tokens',
'fleet-setup-lock',
'graph-workspace',
'guided-setup-state',
'guided-onboarding-guide-state',

View file

@ -221,6 +221,7 @@ describe('split .kibana index into multiple system indices', () => {
"fleet-message-signing-keys",
"fleet-preconfiguration-deletion-record",
"fleet-proxy",
"fleet-setup-lock",
"fleet-uninstall-tokens",
"graph-workspace",
"guided-onboarding-guide-state",

View file

@ -6,7 +6,7 @@
*/
export { INTEGRATIONS_PLUGIN_ID, PLUGIN_ID } from './plugin';
export { INGEST_SAVED_OBJECT_INDEX } from './saved_objects';
export { INGEST_SAVED_OBJECT_INDEX, FLEET_SETUP_LOCK_TYPE } from './saved_objects';
export * from './routes';
export * from './agent';
export * from './agent_policy';

View file

@ -6,3 +6,5 @@
*/
export const INGEST_SAVED_OBJECT_INDEX = '.kibana_ingest';
export const FLEET_SETUP_LOCK_TYPE = 'fleet-setup-lock';

View file

@ -62,6 +62,7 @@ export interface AgentPolicy extends Omit<NewAgentPolicy, 'id'> {
agents?: number;
unprivileged_agents?: number;
is_protected: boolean;
version?: string;
}
export interface FullAgentPolicyInputStream {

View file

@ -0,0 +1,12 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
export interface FleetSetupLock {
status: string;
uuid: string;
started_at: string;
}

View file

@ -20,3 +20,4 @@ export * from './fleet_server_policy_config';
export * from './fleet_proxy';
export * from './secret';
export * from './setup_technology';
export * from './fleet_setup_lock';

View file

@ -90,6 +90,7 @@ export {
OUTPUT_SECRETS_MINIMUM_FLEET_SERVER_VERSION,
// outputs
OUTPUT_HEALTH_DATA_STREAM,
FLEET_SETUP_LOCK_TYPE,
type PrivilegeMapObject,
} from '../../common/constants';

View file

@ -0,0 +1,50 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
import type { TestElasticsearchUtils } from '@kbn/core-test-helpers-kbn-server';
import { createTestServers } from '@kbn/core-test-helpers-kbn-server';
/**
* Verifies that multiple Kibana instances running in parallel will not create duplicate preconfiguration objects.
*/
describe.skip('Fleet setup preconfiguration with multiple instances Kibana', () => {
let esServer: TestElasticsearchUtils;
const startServers = async () => {
const { startES } = createTestServers({
adjustTimeout: (t) => jest.setTimeout(t),
settings: {
es: {
license: 'trial',
},
},
});
esServer = await startES();
};
const stopServers = async () => {
if (esServer) {
await esServer.stop();
}
await new Promise((res) => setTimeout(res, 10000));
};
beforeEach(async () => {
await startServers();
});
afterEach(async () => {
await stopServers();
});
describe('startES', () => {
it('start es', async () => {
await new Promise((resolve) => setTimeout(resolve, 60000));
});
});
});

View file

@ -0,0 +1,278 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
import Path from 'path';
import { range } from 'lodash';
import type { ISavedObjectsRepository } from '@kbn/core/server';
import type { TestElasticsearchUtils, createRoot } from '@kbn/core-test-helpers-kbn-server';
import { getSupertest, createRootWithCorePlugins } from '@kbn/core-test-helpers-kbn-server';
import type {
AgentPolicySOAttributes,
Installation,
OutputSOAttributes,
PackagePolicySOAttributes,
} from '../types';
type Root = ReturnType<typeof createRoot>;
const startAndWaitForFleetSetup = async (root: Root) => {
const start = await root.start();
const isFleetSetupRunning = async () => {
const statusApi = getSupertest(root, 'get', '/api/status');
const resp: any = await statusApi.send();
const fleetStatus = resp.body?.status?.plugins?.fleet;
if (fleetStatus?.meta?.error) {
throw new Error(`Setup failed: ${JSON.stringify(fleetStatus)}`);
}
const isRunning = !fleetStatus || fleetStatus?.summary === 'Fleet is setting up';
return isRunning;
};
while (await isFleetSetupRunning()) {
await new Promise((resolve) => setTimeout(resolve, 2000));
}
return start;
};
const createAndSetupRoot = async (config?: object, index?: number) => {
const root = createRootWithCorePlugins(
{
xpack: {
fleet: config,
},
logging: {
appenders: {
file: {
type: 'file',
fileName: Path.join(__dirname, `logs_${Math.floor(Math.random() * 100)}.log`),
layout: {
type: 'json',
},
},
},
loggers: [
{
name: 'root',
appenders: ['file'],
},
{
name: 'plugins.fleet',
appenders: ['file'],
level: 'info',
},
],
},
},
{ oss: false }
);
await root.preboot();
await root.setup();
return root;
};
/**
* Verifies that multiple Kibana instances running in parallel will not create duplicate preconfiguration objects.
*/
describe.skip('Fleet setup preconfiguration with multiple instances Kibana', () => {
let esServer: TestElasticsearchUtils;
let roots: Root[] = [];
const registryUrl = 'https://epr.elastic.co/';
const addRoots = async (n: number) => {
const newRoots = await Promise.all(
range(n).map((val, index) => createAndSetupRoot(preconfiguration, index))
);
newRoots.forEach((r) => roots.push(r));
return newRoots;
};
const startRoots = async () => {
return await Promise.all(roots.map(startAndWaitForFleetSetup));
};
const stopServers = async () => {
for (const root of roots) {
await root.shutdown();
}
roots = [];
if (esServer) {
await esServer.stop();
}
await new Promise((res) => setTimeout(res, 10000));
};
afterEach(async () => {
await stopServers();
});
describe('preconfiguration setup', () => {
it('sets up Fleet correctly', async () => {
await addRoots(1);
const [root1Start] = await startRoots();
const soClient = root1Start.savedObjects.createInternalRepository();
const esClient = root1Start.elasticsearch.client.asInternalUser;
await new Promise((res) => setTimeout(res, 1000));
try {
const res = await esClient.search({
index: '.fleet-policies',
q: 'policy_id:policy-elastic-agent-on-cloud',
sort: 'revision_idx:desc',
_source: ['revision_idx', '@timestamp'],
});
// eslint-disable-next-line no-console
console.log(JSON.stringify(res, null, 2));
expect(res.hits.hits.length).toBeGreaterThanOrEqual(1);
expect((res.hits.hits[0]._source as any)?.data?.inputs).not.toEqual([]);
} catch (err) {
if (err.statusCode === 404) {
return;
}
throw err;
}
await expectFleetSetupState(soClient);
});
});
const preconfiguration = {
registryUrl,
packages: [
{
name: 'fleet_server',
version: 'latest',
},
{
name: 'apm',
version: 'latest',
},
{
name: 'endpoint',
version: 'latest',
},
{
name: 'log',
version: 'latest',
},
],
outputs: [
{
name: 'Preconfigured output',
id: 'preconfigured-output',
type: 'elasticsearch',
hosts: ['http://127.0.0.1:9200'],
},
],
fleetServerHosts: [
{
id: 'fleet-server',
name: 'Fleet Server',
is_default: true,
host_urls: ['https://192.168.178.216:8220'],
},
],
agentPolicies: [
{
name: 'Elastic Cloud agent policy',
id: 'policy-elastic-agent-on-cloud',
data_output_id: 'preconfigured-output',
monitoring_output_id: 'preconfigured-output',
is_managed: true,
is_default_fleet_server: true,
package_policies: [
{
name: 'elastic-cloud-fleet-server',
package: {
name: 'fleet_server',
},
inputs: [
{
type: 'fleet-server',
keep_enabled: true,
vars: [{ name: 'host', value: '127.0.0.1:8220', frozen: true }],
},
],
},
],
},
],
};
async function expectFleetSetupState(soClient: ISavedObjectsRepository) {
// Assert setup state
const agentPolicies = await soClient.find<AgentPolicySOAttributes>({
type: 'ingest-agent-policies',
perPage: 10000,
});
expect(agentPolicies.saved_objects).toHaveLength(1);
expect(agentPolicies.saved_objects.map((ap) => ap.attributes)).toEqual(
expect.arrayContaining([
expect.objectContaining({
name: 'Elastic Cloud agent policy',
is_managed: true,
is_default_fleet_server: true,
data_output_id: 'preconfigured-output',
}),
])
);
const packagePolicies = await soClient.find<PackagePolicySOAttributes>({
type: 'ingest-package-policies',
perPage: 10000,
});
expect(packagePolicies.saved_objects.length).toBeGreaterThanOrEqual(1);
expect(packagePolicies.saved_objects.map((pp) => pp.attributes.name)).toEqual(
expect.arrayContaining(['elastic-cloud-fleet-server'])
);
const outputs = await soClient.find<OutputSOAttributes>({
type: 'ingest-outputs',
perPage: 10000,
});
expect(outputs.saved_objects).toHaveLength(2);
expect(outputs.saved_objects.map((o) => o.attributes)).toEqual(
expect.arrayContaining([
expect.objectContaining({
name: 'default',
is_default: true,
is_default_monitoring: true,
type: 'elasticsearch',
output_id: 'fleet-default-output',
hosts: ['http://localhost:9200'],
}),
expect.objectContaining({
name: 'Preconfigured output',
is_default: false,
is_default_monitoring: false,
type: 'elasticsearch',
output_id: 'preconfigured-output',
hosts: ['http://127.0.0.1:9200'],
}),
])
);
const packages = await soClient.find<Installation>({
type: 'epm-packages',
perPage: 10000,
});
expect(packages.saved_objects.length).toBeGreaterThanOrEqual(1);
expect(packages.saved_objects.map((p) => p.attributes.name)).toEqual(
expect.arrayContaining(['fleet_server'])
);
}
});

View file

@ -656,18 +656,24 @@ export class FleetPlugin
)
.toPromise();
const randomIntFromInterval = (min: number, max: number) => {
return Math.floor(Math.random() * (max - min + 1) + min);
};
// Retry Fleet setup w/ backoff
await backOff(
async () => {
await setupFleet(
new SavedObjectsClient(core.savedObjects.createInternalRepository()),
core.elasticsearch.client.asInternalUser
core.elasticsearch.client.asInternalUser,
{ useLock: true }
);
},
{
numOfAttempts: setupAttempts,
delayFirstAttempt: true,
// 1s initial backoff
startingDelay: 1000,
startingDelay: randomIntFromInterval(100, 1000),
// 5m max backoff
maxDelay: 60000 * 5,
timeMultiple: 2,

View file

@ -23,6 +23,7 @@ import {
MESSAGE_SIGNING_KEYS_SAVED_OBJECT_TYPE,
INGEST_SAVED_OBJECT_INDEX,
UNINSTALL_TOKENS_SAVED_OBJECT_TYPE,
FLEET_SETUP_LOCK_TYPE,
} from '../constants';
import { migrateSyntheticsPackagePolicyToV8120 } from './migrations/synthetics/to_v8_12_0';
@ -100,6 +101,22 @@ export const getSavedObjectTypes = (
const { useSpaceAwareness } = options;
return {
[FLEET_SETUP_LOCK_TYPE]: {
name: FLEET_SETUP_LOCK_TYPE,
indexPattern: INGEST_SAVED_OBJECT_INDEX,
hidden: false,
namespaceType: 'agnostic',
management: {
importableAndExportable: false,
},
mappings: {
properties: {
status: { type: 'keyword' },
uuid: { type: 'text' },
started_at: { type: 'date' },
},
},
},
// Deprecated
[GLOBAL_SETTINGS_SAVED_OBJECT_TYPE]: {
name: GLOBAL_SETTINGS_SAVED_OBJECT_TYPE,

View file

@ -183,7 +183,11 @@ class AgentPolicyService {
if (options.bumpRevision || options.removeProtection) {
await this.triggerAgentPolicyUpdatedEvent(soClient, esClient, 'updated', id);
}
logger.debug(`Agent policy ${id} update completed`);
logger.debug(
`Agent policy ${id} update completed, revision: ${
options.bumpRevision ? existingAgentPolicy.revision + 1 : existingAgentPolicy.revision
}`
);
return (await this.get(soClient, id)) as AgentPolicy;
}
@ -389,6 +393,7 @@ class AgentPolicyService {
const agentPolicy = {
id: agentPolicySO.id,
version: agentPolicySO.version,
...agentPolicySO.attributes,
};
@ -1041,6 +1046,14 @@ class AgentPolicyService {
return acc;
}, [] as FleetServerPolicy[]);
appContextService
.getLogger()
.debug(
`Deploying policies: ${fleetServerPolicies
.map((pol) => `${pol.policy_id}:${pol.revision_idx}`)
.join(', ')}`
);
const fleetServerPoliciesBulkBody = fleetServerPolicies.flatMap((fleetServerPolicy) => [
{
index: {

View file

@ -103,7 +103,9 @@ export async function _installPackage({
const hasExceededTimeout =
Date.now() - Date.parse(installedPkg.attributes.install_started_at) <
MAX_TIME_COMPLETE_INSTALL;
logger.debug(`Package install - Install status ${installedPkg.attributes.install_status}`);
logger.debug(
`Package install - Install status ${pkgName}-${pkgVersion}: ${installedPkg.attributes.install_status}`
);
// if the installation is currently running, don't try to install
// instead, only return already installed assets
@ -142,7 +144,7 @@ export async function _installPackage({
});
}
} else {
logger.debug(`Package install - Create installation`);
logger.debug(`Package install - Create installation ${pkgName}-${pkgVersion}`);
await createInstallation({
savedObjectsClient,
packageInfo,

View file

@ -294,7 +294,14 @@ export async function ensurePreconfiguredPackagesAndPolicies(
packagePolicy.name === installablePackagePolicy.packagePolicy.name
);
});
logger.debug(`Adding preconfigured package policies ${packagePoliciesToAdd}`);
logger.debug(
`Adding preconfigured package policies ${JSON.stringify(
packagePoliciesToAdd.map((pol) => ({
name: pol.packagePolicy.name,
package: pol.installedPackage.name,
}))
)}`
);
const s = apm.startSpan('Add preconfigured package policies', 'preconfiguration');
await addPreconfiguredPolicyPackages(
esClient,

View file

@ -34,7 +34,6 @@ export async function resetPreconfiguredAgentPolicies(
await _deleteExistingData(soClient, esClient, logger, agentPolicyId);
await _deleteGhostPackagePolicies(soClient, esClient, logger);
await _deletePreconfigurationDeleteRecord(soClient, logger, agentPolicyId);
await setupFleet(soClient, esClient);
}

View file

@ -68,6 +68,8 @@ describe('setupFleet', () => {
soClient.get.mockResolvedValue({ attributes: {} } as any);
soClient.find.mockResolvedValue({ saved_objects: [] } as any);
soClient.bulkGet.mockResolvedValue({ saved_objects: [] } as any);
soClient.create.mockResolvedValue({ attributes: {} } as any);
soClient.delete.mockResolvedValue({});
});
afterEach(async () => {
@ -134,4 +136,59 @@ describe('setupFleet', () => {
],
});
});
it('should create and delete lock if not exists', async () => {
soClient.get.mockRejectedValue({ isBoom: true, output: { statusCode: 404 } } as any);
const result = await setupFleet(soClient, esClient, { useLock: true });
expect(result).toEqual({
isInitialized: true,
nonFatalErrors: [],
});
expect(soClient.create).toHaveBeenCalledWith('fleet-setup-lock', expect.anything(), {
id: 'fleet-setup-lock',
});
expect(soClient.delete).toHaveBeenCalledWith('fleet-setup-lock', 'fleet-setup-lock', {
refresh: true,
});
});
it('should return not initialized if lock exists', async () => {
const result = await setupFleet(soClient, esClient, { useLock: true });
expect(result).toEqual({
isInitialized: false,
nonFatalErrors: [],
});
expect(soClient.create).not.toHaveBeenCalled();
expect(soClient.delete).not.toHaveBeenCalled();
});
it('should return not initialized if lock could not be created', async () => {
soClient.get.mockRejectedValue({ isBoom: true, output: { statusCode: 404 } } as any);
soClient.create.mockRejectedValue({ isBoom: true, output: { statusCode: 409 } } as any);
const result = await setupFleet(soClient, esClient, { useLock: true });
expect(result).toEqual({
isInitialized: false,
nonFatalErrors: [],
});
expect(soClient.delete).not.toHaveBeenCalled();
});
it('should delete previous lock if created more than 1 hour ago', async () => {
soClient.get.mockResolvedValue({
attributes: { started_at: new Date(Date.now() - 60 * 60 * 1000 - 1000).toISOString() },
} as any);
const result = await setupFleet(soClient, esClient, { useLock: true });
expect(result).toEqual({
isInitialized: true,
nonFatalErrors: [],
});
expect(soClient.create).toHaveBeenCalled();
expect(soClient.delete).toHaveBeenCalledTimes(2);
});
});

View file

@ -11,14 +11,15 @@ import apm from 'elastic-apm-node';
import { compact } from 'lodash';
import pMap from 'p-map';
import { v4 as uuidv4 } from 'uuid';
import type { ElasticsearchClient, SavedObjectsClientContract } from '@kbn/core/server';
import { DEFAULT_SPACE_ID } from '@kbn/spaces-plugin/common/constants';
import { MessageSigningError } from '../../common/errors';
import { AUTO_UPDATE_PACKAGES } from '../../common/constants';
import { AUTO_UPDATE_PACKAGES, FLEET_SETUP_LOCK_TYPE } from '../../common/constants';
import type { PreconfigurationError } from '../../common/constants';
import type { DefaultPackagesInstallationError } from '../../common/types';
import type { DefaultPackagesInstallationError, FleetSetupLock } from '../../common/types';
import { appContextService } from './app_context';
import { ensurePreconfiguredPackagesAndPolicies } from './preconfiguration';
@ -65,11 +66,19 @@ export interface SetupStatus {
export async function setupFleet(
soClient: SavedObjectsClientContract,
esClient: ElasticsearchClient
esClient: ElasticsearchClient,
options: {
useLock: boolean;
} = { useLock: false }
): Promise<SetupStatus> {
const t = apm.startTransaction('fleet-setup', 'fleet');
let created = false;
try {
if (options.useLock) {
const { created: isCreated, toReturn } = await createLock(soClient);
created = isCreated;
if (toReturn) return toReturn;
}
return await awaitIfPending(async () => createSetupSideEffects(soClient, esClient));
} catch (error) {
apm.captureError(error);
@ -77,6 +86,71 @@ export async function setupFleet(
throw error;
} finally {
t.end();
// only delete lock if it was created by this instance
if (options.useLock && created) {
await deleteLock(soClient);
}
}
}
async function createLock(
soClient: SavedObjectsClientContract
): Promise<{ created: boolean; toReturn?: SetupStatus }> {
const logger = appContextService.getLogger();
let created;
try {
// check if fleet setup is already started
const fleetSetupLock = await soClient.get<FleetSetupLock>(
FLEET_SETUP_LOCK_TYPE,
FLEET_SETUP_LOCK_TYPE
);
const LOCK_TIMEOUT = 60 * 60 * 1000; // 1 hour
// started more than 1 hour ago, delete previous lock
if (
fleetSetupLock.attributes.started_at &&
new Date(fleetSetupLock.attributes.started_at).getTime() < Date.now() - LOCK_TIMEOUT
) {
await deleteLock(soClient);
} else {
logger.info('Fleet setup already in progress, abort setup');
return { created: false, toReturn: { isInitialized: false, nonFatalErrors: [] } };
}
} catch (error) {
if (error.isBoom && error.output.statusCode === 404) {
logger.debug('Fleet setup lock does not exist, continue setup');
}
}
try {
created = await soClient.create<FleetSetupLock>(
FLEET_SETUP_LOCK_TYPE,
{
status: 'in_progress',
uuid: uuidv4(),
started_at: new Date().toISOString(),
},
{ id: FLEET_SETUP_LOCK_TYPE }
);
logger.debug(`Fleet setup lock created: ${JSON.stringify(created)}`);
} catch (error) {
logger.info(`Could not create fleet setup lock, abort setup: ${error}`);
return { created: false, toReturn: { isInitialized: false, nonFatalErrors: [] } };
}
return { created: !!created };
}
async function deleteLock(soClient: SavedObjectsClientContract) {
const logger = appContextService.getLogger();
try {
await soClient.delete(FLEET_SETUP_LOCK_TYPE, FLEET_SETUP_LOCK_TYPE, { refresh: true });
logger.debug(`Fleet setup lock deleted`);
} catch (error) {
// ignore 404 errors
if (error.statusCode !== 404) {
logger.error('Could not delete fleet setup lock', error);
}
}
}

View file

@ -65,6 +65,7 @@ export interface AgentPolicySOAttributes {
agents?: number;
overrides?: any | null;
global_data_tags?: Array<{ name: string; value: string | number }>;
version?: string;
}
export interface AgentSOAttributes {

View file

@ -517,7 +517,7 @@ export default function (providerContext: FtrProviderContext) {
})
.expect(200);
// eslint-disable-next-line @typescript-eslint/naming-convention
const { id, updated_at, ...newPolicy } = item;
const { id, updated_at, version, ...newPolicy } = item;
expect(newPolicy).to.eql({
name: 'Copied policy',
@ -947,7 +947,7 @@ export default function (providerContext: FtrProviderContext) {
.expect(200);
createdPolicyIds.push(updatedPolicy.id);
// eslint-disable-next-line @typescript-eslint/naming-convention
const { id, updated_at, ...newPolicy } = updatedPolicy;
const { id, updated_at, version, ...newPolicy } = updatedPolicy;
expect(newPolicy).to.eql({
status: 'active',
@ -1108,7 +1108,7 @@ export default function (providerContext: FtrProviderContext) {
})
.expect(200);
// eslint-disable-next-line @typescript-eslint/naming-convention
const { id, updated_at, ...newPolicy } = updatedPolicy;
const { id, updated_at, version, ...newPolicy } = updatedPolicy;
createdPolicyIds.push(updatedPolicy.id);
expect(newPolicy).to.eql({
@ -1168,7 +1168,7 @@ export default function (providerContext: FtrProviderContext) {
.expect(200);
// eslint-disable-next-line @typescript-eslint/naming-convention
const { id, updated_at, ...newPolicy } = updatedPolicy;
const { id, updated_at, version, ...newPolicy } = updatedPolicy;
expect(newPolicy).to.eql({
status: 'active',