[Security Solution] redirect old isolate / release routes (#134536)

* [Security Solution] redirect old isolate / release routes

deprecate old isolate / release routes and 308 to newer routes

* add header check to tests
This commit is contained in:
Joey F. Poon 2022-06-21 11:03:29 -05:00 committed by GitHub
parent 10a5f9ac83
commit 02bc0e97bb
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
6 changed files with 74 additions and 876 deletions

View file

@ -58,7 +58,7 @@ const BASE_ENDPOINT_ACTION_ROUTE = `${BASE_ENDPOINT_ROUTE}/action`;
/** Action Response Routes */
export const ISOLATE_HOST_ROUTE_V2 = `${BASE_ENDPOINT_ACTION_ROUTE}/isolate`;
export const RELEASE_HOST_ROUTE = `${BASE_ENDPOINT_ACTION_ROUTE}/release`;
export const UNISOLATE_HOST_ROUTE_V2 = `${BASE_ENDPOINT_ACTION_ROUTE}/unisolate`;
export const GET_RUNNING_PROCESSES_ROUTE = `${BASE_ENDPOINT_ACTION_ROUTE}/running_procs`;
export const KILL_PROCESS_ROUTE = `${BASE_ENDPOINT_ACTION_ROUTE}/kill_process`;
export const SUSPEND_PROCESS_ROUTE = `${BASE_ENDPOINT_ACTION_ROUTE}/suspend_process`;

View file

@ -8,21 +8,17 @@
import { registerActionDetailsRoutes } from './details';
import { SecuritySolutionPluginRouter } from '../../../types';
import { EndpointAppContext } from '../../types';
import { registerHostIsolationRoutes } from './isolation';
import { registerActionStatusRoutes } from './status';
import { registerActionAuditLogRoutes } from './audit_log';
import { registerActionListRoutes } from './list';
import { registerResponseActionRoutes } from './response_actions';
export * from './isolation';
// wrap route registration
export function registerActionRoutes(
router: SecuritySolutionPluginRouter,
endpointContext: EndpointAppContext
) {
registerHostIsolationRoutes(router, endpointContext);
registerActionStatusRoutes(router, endpointContext);
registerActionAuditLogRoutes(router, endpointContext);
registerActionListRoutes(router, endpointContext);

View file

@ -1,568 +0,0 @@
/*
* 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.
*/
/* eslint-disable @typescript-eslint/no-explicit-any */
import * as estypes from '@elastic/elasticsearch/lib/api/typesWithBodyKey';
import { AwaitedProperties } from '@kbn/utility-types';
import {
KibanaRequest,
KibanaResponseFactory,
RequestHandler,
RouteConfig,
} from '@kbn/core/server';
import {
elasticsearchServiceMock,
httpServerMock,
httpServiceMock,
loggingSystemMock,
savedObjectsClientMock,
} from '@kbn/core/server/mocks';
import { parseExperimentalConfigValue } from '../../../../common/experimental_features';
import { EndpointAppContextService } from '../../endpoint_app_context_services';
import {
createMockEndpointAppContextServiceSetupContract,
createMockEndpointAppContextServiceStartContract,
createRouteHandlerContext,
} from '../../mocks';
import { NoParametersRequestSchema } from '../../../../common/endpoint/schema/actions';
import { registerHostIsolationRoutes } from './isolation';
import { createMockConfig } from '../../../lib/detection_engine/routes/__mocks__';
import { LicenseService } from '../../../../common/license';
import { Subject } from 'rxjs';
import { ILicense } from '@kbn/licensing-plugin/common/types';
import { licenseMock } from '@kbn/licensing-plugin/common/licensing.mock';
import { License } from '@kbn/licensing-plugin/common/license';
import {
ISOLATE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE,
metadataTransformPrefix,
ENDPOINT_ACTIONS_INDEX,
} from '../../../../common/endpoint/constants';
import {
EndpointAction,
HostIsolationRequestBody,
HostIsolationResponse,
HostMetadata,
LogsEndpointAction,
} from '../../../../common/endpoint/types';
import { EndpointDocGenerator } from '../../../../common/endpoint/generate_data';
import { legacyMetadataSearchResponseMock } from '../metadata/support/test_support';
import { AGENT_ACTIONS_INDEX, ElasticsearchAssetType } from '@kbn/fleet-plugin/common';
import { CasesClientMock } from '@kbn/cases-plugin/server/client/mocks';
import { EndpointAuthz } from '../../../../common/endpoint/types/authz';
import type { PackageClient } from '@kbn/fleet-plugin/server';
import { createMockPackageService } from '@kbn/fleet-plugin/server/mocks';
import { SecuritySolutionRequestHandlerContextMock } from '../../../lib/detection_engine/routes/__mocks__/request_context';
interface CallRouteInterface {
body?: HostIsolationRequestBody;
idxResponse?: any;
searchResponse?: HostMetadata;
mockUser?: any;
license?: License;
authz?: Partial<EndpointAuthz>;
}
const Platinum = licenseMock.createLicense({ license: { type: 'platinum', mode: 'platinum' } });
const Gold = licenseMock.createLicense({ license: { type: 'gold', mode: 'gold' } });
describe('Host Isolation', () => {
describe('schema', () => {
it('should require at least 1 Endpoint ID', () => {
expect(() => {
NoParametersRequestSchema.body.validate({});
}).toThrow();
});
it('should accept an Endpoint ID as the only required field', () => {
expect(() => {
NoParametersRequestSchema.body.validate({
endpoint_ids: ['ABC-XYZ-000'],
});
}).not.toThrow();
});
it('should accept a comment', () => {
expect(() => {
NoParametersRequestSchema.body.validate({
endpoint_ids: ['ABC-XYZ-000'],
comment: 'a user comment',
});
}).not.toThrow();
});
it('should accept alert IDs', () => {
expect(() => {
NoParametersRequestSchema.body.validate({
endpoint_ids: ['ABC-XYZ-000'],
alert_ids: ['0000000-000-00'],
});
}).not.toThrow();
});
it('should accept case IDs', () => {
expect(() => {
NoParametersRequestSchema.body.validate({
endpoint_ids: ['ABC-XYZ-000'],
case_ids: ['000000000-000-000'],
});
}).not.toThrow();
});
});
describe('handler', () => {
let endpointAppContextService: EndpointAppContextService;
let mockResponse: jest.Mocked<KibanaResponseFactory>;
let licenseService: LicenseService;
let licenseEmitter: Subject<ILicense>;
let callRoute: (
routePrefix: string,
opts: CallRouteInterface,
indexExists?: { endpointDsExists: boolean }
) => Promise<AwaitedProperties<SecuritySolutionRequestHandlerContextMock>>;
const superUser = {
username: 'superuser',
roles: ['superuser'],
};
const docGen = new EndpointDocGenerator();
beforeEach(() => {
// instantiate... everything
const mockScopedClient = elasticsearchServiceMock.createScopedClusterClient();
const mockClusterClient = elasticsearchServiceMock.createClusterClient();
mockClusterClient.asScoped.mockReturnValue(mockScopedClient);
const routerMock = httpServiceMock.createRouter();
mockResponse = httpServerMock.createResponseFactory();
const startContract = createMockEndpointAppContextServiceStartContract();
endpointAppContextService = new EndpointAppContextService();
const mockSavedObjectClient = savedObjectsClientMock.create();
const mockPackageService = createMockPackageService();
const mockedPackageClient = mockPackageService.asInternalUser as jest.Mocked<PackageClient>;
mockedPackageClient.getInstallation.mockResolvedValue({
installed_kibana: [],
package_assets: [],
es_index_patterns: {},
name: '',
version: '',
install_status: 'installed',
install_version: '',
install_started_at: '',
install_source: 'registry',
installed_es: [
{
id: 'logs-endpoint.events.security',
type: ElasticsearchAssetType.indexTemplate,
},
{
id: `${metadataTransformPrefix}-0.16.0-dev.0`,
type: ElasticsearchAssetType.transform,
},
],
keep_policies_up_to_date: false,
});
licenseEmitter = new Subject();
licenseService = new LicenseService();
licenseService.start(licenseEmitter);
endpointAppContextService.setup(createMockEndpointAppContextServiceSetupContract());
endpointAppContextService.start({
...startContract,
licenseService,
packageService: mockPackageService,
});
// add the host isolation route handlers to routerMock
registerHostIsolationRoutes(routerMock, {
logFactory: loggingSystemMock.create(),
service: endpointAppContextService,
config: () => Promise.resolve(createMockConfig()),
experimentalFeatures: parseExperimentalConfigValue(createMockConfig().enableExperimental),
});
// define a convenience function to execute an API call for a given route, body, and mocked response from ES
// it returns the requestContext mock used in the call, to assert internal calls (e.g. the indexed document)
callRoute = async (
routePrefix: string,
{ body, idxResponse, searchResponse, mockUser, license, authz = {} }: CallRouteInterface,
indexExists?: { endpointDsExists: boolean }
): Promise<AwaitedProperties<SecuritySolutionRequestHandlerContextMock>> => {
const asUser = mockUser ? mockUser : superUser;
(startContract.security.authc.getCurrentUser as jest.Mock).mockImplementationOnce(
() => asUser
);
const ctx = createRouteHandlerContext(mockScopedClient, mockSavedObjectClient);
ctx.securitySolution.endpointAuthz = {
...ctx.securitySolution.endpointAuthz,
...authz,
};
// mock _index_template
ctx.core.elasticsearch.client.asInternalUser.indices.existsIndexTemplate.mockResponseImplementationOnce(
() => {
if (indexExists) {
return {
body: true,
statusCode: 200,
};
}
return {
body: false,
statusCode: 404,
};
}
);
const withIdxResp = idxResponse ? idxResponse : { statusCode: 201 };
ctx.core.elasticsearch.client.asInternalUser.index.mockResponseImplementation(
() => withIdxResp
);
ctx.core.elasticsearch.client.asCurrentUser.search.mockResponseImplementation(() => {
return {
body: legacyMetadataSearchResponseMock(searchResponse),
};
});
const withLicense = license ? license : Platinum;
licenseEmitter.next(withLicense);
const mockRequest = httpServerMock.createKibanaRequest({ body });
const [, routeHandler]: [
RouteConfig<any, any, any, any>,
RequestHandler<any, any, any, any>
] = routerMock.post.mock.calls.find(([{ path }]) => path.startsWith(routePrefix))!;
await routeHandler(ctx, mockRequest, mockResponse);
return ctx;
};
});
afterEach(() => {
endpointAppContextService.stop();
licenseService.stop();
licenseEmitter.complete();
});
it('succeeds when an endpoint ID is provided', async () => {
await callRoute(ISOLATE_HOST_ROUTE, { body: { endpoint_ids: ['XYZ'] } });
expect(mockResponse.ok).toBeCalled();
});
it('reports elasticsearch errors creating an action', async () => {
const ErrMessage = 'something went wrong?';
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
idxResponse: {
statusCode: 500,
body: {
result: ErrMessage,
},
},
});
expect(mockResponse.ok).not.toBeCalled();
const response = mockResponse.customError.mock.calls[0][0];
expect(response.statusCode).toEqual(500);
expect((response.body as Error).message).toEqual(ErrMessage);
});
it('accepts a comment field', async () => {
await callRoute(ISOLATE_HOST_ROUTE, { body: { endpoint_ids: ['XYZ'], comment: 'XYZ' } });
expect(mockResponse.ok).toBeCalled();
});
it('sends the action to the requested agent', async () => {
const metadataResponse = docGen.generateHostMetadata();
const AgentID = metadataResponse.elastic.agent.id;
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['ABC-XYZ-000'] },
searchResponse: metadataResponse,
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.agents).toContain(AgentID);
});
it('records the user who performed the action to the action record', async () => {
const testU = { username: 'testuser', roles: ['superuser'] };
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
mockUser: testU,
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.user_id).toEqual(testU.username);
});
it('records the comment in the action payload', async () => {
const CommentText = "I am isolating this because it's Friday";
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'], comment: CommentText },
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.data.comment).toEqual(CommentText);
});
it('creates an action and returns its ID', async () => {
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'], comment: 'XYZ' },
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
const actionID = actionDoc.action_id;
expect(mockResponse.ok).toBeCalled();
expect((mockResponse.ok.mock.calls[0][0]?.body as HostIsolationResponse).action).toEqual(
actionID
);
});
it('records the timeout in the action payload', async () => {
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.timeout).toEqual(300);
});
it('sends the action to the correct agent when endpoint ID is given', async () => {
const doc = docGen.generateHostMetadata();
const AgentID = doc.elastic.agent.id;
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
searchResponse: doc,
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.agents).toContain(AgentID);
});
it('sends the isolate command payload from the isolate route', async () => {
const ctx = await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.data.command).toEqual('isolate');
});
it('sends the unisolate command payload from the unisolate route', async () => {
const ctx = await callRoute(UNISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
});
const actionDoc: EndpointAction = (
ctx.core.elasticsearch.client.asInternalUser.index.mock
.calls[0][0] as estypes.IndexRequest<EndpointAction>
).body!;
expect(actionDoc.data.command).toEqual('unisolate');
});
describe('With endpoint data streams', () => {
it('handles unisolation', async () => {
const ctx = await callRoute(
UNISOLATE_HOST_ROUTE,
{
body: { endpoint_ids: ['XYZ'] },
},
{ endpointDsExists: true }
);
const indexDoc = ctx.core.elasticsearch.client.asInternalUser.index;
const actionDocs: [
{ index: string; body?: LogsEndpointAction },
{ index: string; body?: EndpointAction }
] = [
indexDoc.mock.calls[0][0] as estypes.IndexRequest<LogsEndpointAction>,
indexDoc.mock.calls[1][0] as estypes.IndexRequest<EndpointAction>,
];
expect(actionDocs[0].index).toEqual(ENDPOINT_ACTIONS_INDEX);
expect(actionDocs[1].index).toEqual(AGENT_ACTIONS_INDEX);
expect(actionDocs[0].body!.EndpointActions.data.command).toEqual('unisolate');
expect(actionDocs[1].body!.data.command).toEqual('unisolate');
});
it('handles isolation', async () => {
const ctx = await callRoute(
ISOLATE_HOST_ROUTE,
{
body: { endpoint_ids: ['XYZ'] },
},
{ endpointDsExists: true }
);
const indexDoc = ctx.core.elasticsearch.client.asInternalUser.index;
const actionDocs: [
{ index: string; body?: LogsEndpointAction },
{ index: string; body?: EndpointAction }
] = [
indexDoc.mock.calls[0][0] as estypes.IndexRequest<LogsEndpointAction>,
indexDoc.mock.calls[1][0] as estypes.IndexRequest<EndpointAction>,
];
expect(actionDocs[0].index).toEqual(ENDPOINT_ACTIONS_INDEX);
expect(actionDocs[1].index).toEqual(AGENT_ACTIONS_INDEX);
expect(actionDocs[0].body!.EndpointActions.data.command).toEqual('isolate');
expect(actionDocs[1].body!.data.command).toEqual('isolate');
});
it('handles errors', async () => {
const ErrMessage = 'Uh oh!';
await callRoute(
UNISOLATE_HOST_ROUTE,
{
body: { endpoint_ids: ['XYZ'] },
idxResponse: {
statusCode: 500,
body: {
result: ErrMessage,
},
},
},
{ endpointDsExists: true }
);
expect(mockResponse.ok).not.toBeCalled();
const response = mockResponse.customError.mock.calls[0][0];
expect(response.statusCode).toEqual(500);
expect((response.body as Error).message).toEqual(ErrMessage);
});
});
describe('License Level', () => {
it('allows platinum license levels to isolate hosts', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
license: Platinum,
});
expect(mockResponse.ok).toBeCalled();
});
it('prohibits isolating hosts if no authz for it', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
authz: { canIsolateHost: false },
license: Gold,
});
expect(mockResponse.forbidden).toBeCalled();
});
it('allows any license level to unisolate', async () => {
licenseEmitter.next(Gold);
await callRoute(UNISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
license: Gold,
});
expect(mockResponse.ok).toBeCalled();
});
});
describe('User Authorization Level', () => {
it('allows user to perform isolation when canIsolateHost is true', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
});
expect(mockResponse.ok).toBeCalled();
});
it('allows user to perform unisolation when canUnIsolateHost is true', async () => {
await callRoute(UNISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
});
expect(mockResponse.ok).toBeCalled();
});
it('prohibits user from performing isolation if canIsolateHost is false', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
authz: { canIsolateHost: false },
});
expect(mockResponse.forbidden).toBeCalled();
});
it('prohibits user from performing un-isolation if canUnIsolateHost is false', async () => {
await callRoute(UNISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'] },
authz: { canUnIsolateHost: false },
});
expect(mockResponse.forbidden).toBeCalled();
});
});
describe('Cases', () => {
let casesClient: CasesClientMock;
const getCaseIdsFromAttachmentAddService = () => {
return casesClient.attachments.add.mock.calls.map(([addArgs]) => addArgs.caseId);
};
beforeEach(async () => {
casesClient = (await endpointAppContextService.getCasesClient(
{} as KibanaRequest
)) as CasesClientMock;
let counter = 1;
casesClient.cases.getCasesByAlertID.mockImplementation(async () => {
return [
{
id: `case-${counter++}`,
title: 'case',
},
];
});
});
it('logs a comment to the provided cases', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'], case_ids: ['one', 'two'] },
});
expect(casesClient.attachments.add).toHaveBeenCalledTimes(2);
expect(getCaseIdsFromAttachmentAddService()).toEqual(
expect.arrayContaining(['one', 'two'])
);
});
it('logs a comment to any cases associated with the given alerts', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
body: { endpoint_ids: ['XYZ'], alert_ids: ['one', 'two'] },
});
expect(getCaseIdsFromAttachmentAddService()).toEqual(
expect.arrayContaining(['case-1', 'case-2'])
);
});
it('logs a comment to any cases provided on input along with cases associated with the given alerts', async () => {
await callRoute(ISOLATE_HOST_ROUTE, {
// 'case-1` provided on `case_ids` should be dedupped
body: {
endpoint_ids: ['XYZ'],
case_ids: ['ONE', 'TWO', 'case-1'],
alert_ids: ['one', 'two'],
},
});
expect(casesClient.attachments.add).toHaveBeenCalledTimes(4);
expect(getCaseIdsFromAttachmentAddService()).toEqual(
expect.arrayContaining(['ONE', 'TWO', 'case-1', 'case-2'])
);
});
});
});
});

View file

@ -1,294 +0,0 @@
/*
* 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 moment from 'moment';
import { RequestHandler, Logger } from '@kbn/core/server';
import uuid from 'uuid';
import { TypeOf } from '@kbn/config-schema';
import { CommentType } from '@kbn/cases-plugin/common';
import { CasesByAlertId } from '@kbn/cases-plugin/common/api/cases/case';
import { AGENT_ACTIONS_INDEX } from '@kbn/fleet-plugin/common';
import { NoParametersRequestSchema } from '../../../../common/endpoint/schema/actions';
import {
ENDPOINT_ACTIONS_DS,
ENDPOINT_ACTION_RESPONSES_DS,
ISOLATE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE,
failedFleetActionErrorCode,
} from '../../../../common/endpoint/constants';
import {
EndpointAction,
HostMetadata,
LogsEndpointAction,
LogsEndpointActionResponse,
} from '../../../../common/endpoint/types';
import {
SecuritySolutionPluginRouter,
SecuritySolutionRequestHandlerContext,
} from '../../../types';
import { getMetadataForEndpoints } from '../../services';
import { EndpointAppContext } from '../../types';
import { APP_ID } from '../../../../common/constants';
import { doLogsEndpointActionDsExists } from '../../utils';
import { withEndpointAuthz } from '../with_endpoint_authz';
/**
* Registers the Host-(un-)isolation routes
*/
export function registerHostIsolationRoutes(
router: SecuritySolutionPluginRouter,
endpointContext: EndpointAppContext
) {
const logger = endpointContext.logFactory.get('hostIsolation');
// perform isolation
router.post(
{
path: ISOLATE_HOST_ROUTE,
validate: NoParametersRequestSchema,
options: { authRequired: true, tags: ['access:securitySolution'] },
},
withEndpointAuthz(
{ all: ['canIsolateHost'] },
logger,
isolationRequestHandler(endpointContext, true)
)
);
// perform UN-isolate
router.post(
{
path: UNISOLATE_HOST_ROUTE,
validate: NoParametersRequestSchema,
options: { authRequired: true, tags: ['access:securitySolution'] },
},
withEndpointAuthz(
{ all: ['canUnIsolateHost'] },
logger,
isolationRequestHandler(endpointContext, false)
)
);
}
const createFailedActionResponseEntry = async ({
context,
doc,
logger,
}: {
context: SecuritySolutionRequestHandlerContext;
doc: LogsEndpointActionResponse;
logger: Logger;
}): Promise<void> => {
// 8.0+ requires internal user to write to system indices
const esClient = (await context.core).elasticsearch.client.asInternalUser;
try {
await esClient.index<LogsEndpointActionResponse>({
index: `${ENDPOINT_ACTION_RESPONSES_DS}-default`,
body: {
...doc,
error: {
code: failedFleetActionErrorCode,
message: 'Failed to deliver action request to fleet',
},
},
});
} catch (e) {
logger.error(e);
}
};
export const isolationRequestHandler = function (
endpointContext: EndpointAppContext,
isolate: boolean
): RequestHandler<
unknown,
unknown,
TypeOf<typeof NoParametersRequestSchema.body>,
SecuritySolutionRequestHandlerContext
> {
return async (context, req, res) => {
endpointContext.service.getFeatureUsageService().notifyUsage('HOST_ISOLATION');
const user = endpointContext.service.security?.authc.getCurrentUser(req);
// fetch the Agent IDs to send the commands to
const endpointIDs = [...new Set(req.body.endpoint_ids)]; // dedupe
const endpointData = await getMetadataForEndpoints(endpointIDs, context);
const casesClient = await endpointContext.service.getCasesClient(req);
// convert any alert IDs into cases
let caseIDs: string[] = req.body.case_ids?.slice() || [];
if (req.body.alert_ids && req.body.alert_ids.length > 0) {
const newIDs: string[][] = await Promise.all(
req.body.alert_ids.map(async (a: string) => {
const cases: CasesByAlertId = await casesClient.cases.getCasesByAlertID({
alertID: a,
options: { owner: APP_ID },
});
return cases.map((caseInfo): string => {
return caseInfo.id;
});
})
);
caseIDs = caseIDs.concat(...newIDs);
}
caseIDs = [...new Set(caseIDs)];
// create an Action ID and dispatch it to ES & Fleet Server
const actionID = uuid.v4();
let fleetActionIndexResult;
let logsEndpointActionsResult;
const agents = endpointData.map((endpoint: HostMetadata) => endpoint.elastic.agent.id);
const doc = {
'@timestamp': moment().toISOString(),
agent: {
id: agents,
},
EndpointActions: {
action_id: actionID,
expiration: moment().add(2, 'weeks').toISOString(),
type: 'INPUT_ACTION',
input_type: 'endpoint',
data: {
command: isolate ? 'isolate' : 'unisolate',
comment: req.body.comment ?? undefined,
},
} as Omit<EndpointAction, 'agents' | 'user_id'>,
user: {
// eslint-disable-next-line @typescript-eslint/no-non-null-assertion
id: user!.username,
},
};
// if .logs-endpoint.actions data stream exists
// try to create action request record in .logs-endpoint.actions DS as the current user
// (from >= v7.16, use this check to ensure the current user has privileges to write to the new index)
// and allow only users with superuser privileges to write to fleet indices
const logger = endpointContext.logFactory.get('host-isolation');
const doesLogsEndpointActionsDsExist = await doLogsEndpointActionDsExists({
context,
logger,
dataStreamName: ENDPOINT_ACTIONS_DS,
});
// 8.0+ requires internal user to write to system indices
const esClient = (await context.core).elasticsearch.client.asInternalUser;
// if the new endpoint indices/data streams exists
// write the action request to the new endpoint index
if (doesLogsEndpointActionsDsExist) {
try {
logsEndpointActionsResult = await esClient.index<LogsEndpointAction>(
{
index: `${ENDPOINT_ACTIONS_DS}-default`,
body: {
...doc,
},
refresh: 'wait_for',
},
{ meta: true }
);
if (logsEndpointActionsResult.statusCode !== 201) {
return res.customError({
statusCode: 500,
body: {
message: logsEndpointActionsResult.body.result,
},
});
}
} catch (e) {
return res.customError({
statusCode: 500,
body: { message: e },
});
}
}
// write actions to .fleet-actions index
try {
fleetActionIndexResult = await esClient.index<EndpointAction>(
{
index: AGENT_ACTIONS_INDEX,
body: {
...doc.EndpointActions,
'@timestamp': doc['@timestamp'],
agents,
timeout: 300, // 5 minutes
user_id: doc.user.id,
},
refresh: 'wait_for',
},
{ meta: true }
);
if (fleetActionIndexResult.statusCode !== 201) {
return res.customError({
statusCode: 500,
body: {
message: fleetActionIndexResult.body.result,
},
});
}
} catch (e) {
// create entry in .logs-endpoint.action.responses-default data stream
// when writing to .fleet-actions fails
if (doesLogsEndpointActionsDsExist) {
await createFailedActionResponseEntry({
context,
doc: {
'@timestamp': moment().toISOString(),
agent: doc.agent,
EndpointActions: {
action_id: doc.EndpointActions.action_id,
completed_at: moment().toISOString(),
started_at: moment().toISOString(),
data: doc.EndpointActions.data,
},
},
logger,
});
}
return res.customError({
statusCode: 500,
body: { message: e },
});
}
// Update all cases with a comment
if (caseIDs.length > 0) {
const targets = endpointData.map((endpt: HostMetadata) => ({
hostname: endpt.host.hostname,
endpointId: endpt.agent.id,
}));
await Promise.all(
caseIDs.map((caseId) =>
casesClient.attachments.add({
caseId,
comment: {
type: CommentType.actions,
comment: req.body.comment || '',
actions: {
targets,
type: isolate ? 'isolate' : 'unisolate',
},
owner: APP_ID,
},
})
)
);
}
return res.ok({
body: {
action: actionID,
},
});
};
};

View file

@ -35,12 +35,14 @@ import { parseExperimentalConfigValue } from '../../../../common/experimental_fe
import { LicenseService } from '../../../../common/license';
import {
ISOLATE_HOST_ROUTE_V2,
RELEASE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE_V2,
metadataTransformPrefix,
ENDPOINT_ACTIONS_INDEX,
KILL_PROCESS_ROUTE,
SUSPEND_PROCESS_ROUTE,
GET_RUNNING_PROCESSES_ROUTE,
ISOLATE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE,
} from '../../../../common/endpoint/constants';
import {
ActionDetails,
@ -221,6 +223,22 @@ describe('Response actions', () => {
getActionDetailsByIdSpy.mockClear();
});
it('correctly redirects legacy isolate to new route', async () => {
await callRoute(ISOLATE_HOST_ROUTE, { body: { endpoint_ids: ['XYZ'] } });
expect(mockResponse.custom).toBeCalled();
const response = mockResponse.custom.mock.calls[0][0];
expect(response.statusCode).toEqual(308);
expect(response.headers?.location).toEqual(ISOLATE_HOST_ROUTE_V2);
});
it('correctly redirects legacy release to new route', async () => {
await callRoute(UNISOLATE_HOST_ROUTE, { body: { endpoint_ids: ['XYZ'] } });
expect(mockResponse.custom).toBeCalled();
const response = mockResponse.custom.mock.calls[0][0];
expect(response.statusCode).toEqual(308);
expect(response.headers?.location).toEqual(UNISOLATE_HOST_ROUTE_V2);
});
it('succeeds when an endpoint ID is provided', async () => {
await callRoute(ISOLATE_HOST_ROUTE_V2, { body: { endpoint_ids: ['XYZ'] } });
expect(mockResponse.ok).toBeCalled();
@ -350,7 +368,7 @@ describe('Response actions', () => {
});
it('sends the unisolate command payload from the unisolate route', async () => {
const ctx = await callRoute(RELEASE_HOST_ROUTE, {
const ctx = await callRoute(UNISOLATE_HOST_ROUTE_V2, {
body: { endpoint_ids: ['XYZ'] },
});
const actionDoc: EndpointAction = (
@ -396,7 +414,7 @@ describe('Response actions', () => {
describe('With endpoint data streams', () => {
it('handles unisolation', async () => {
const ctx = await callRoute(
RELEASE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE_V2,
{
body: { endpoint_ids: ['XYZ'] },
},
@ -537,7 +555,7 @@ describe('Response actions', () => {
it('handles errors', async () => {
const ErrMessage = 'Uh oh!';
await callRoute(
RELEASE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE_V2,
{
body: { endpoint_ids: ['XYZ'] },
idxResponse: {
@ -578,7 +596,7 @@ describe('Response actions', () => {
it('allows any license level to unisolate', async () => {
licenseEmitter.next(Gold);
await callRoute(RELEASE_HOST_ROUTE, {
await callRoute(UNISOLATE_HOST_ROUTE_V2, {
body: { endpoint_ids: ['XYZ'] },
license: Gold,
});
@ -595,7 +613,7 @@ describe('Response actions', () => {
});
it('allows user to perform unisolation when canUnIsolateHost is true', async () => {
await callRoute(RELEASE_HOST_ROUTE, {
await callRoute(UNISOLATE_HOST_ROUTE_V2, {
body: { endpoint_ids: ['XYZ'] },
});
expect(mockResponse.ok).toBeCalled();
@ -610,7 +628,7 @@ describe('Response actions', () => {
});
it('prohibits user from performing un-isolation if canUnIsolateHost is false', async () => {
await callRoute(RELEASE_HOST_ROUTE, {
await callRoute(UNISOLATE_HOST_ROUTE_V2, {
body: { endpoint_ids: ['XYZ'] },
authz: { canUnIsolateHost: false },
});

View file

@ -22,13 +22,15 @@ import {
import { APP_ID } from '../../../../common/constants';
import {
ISOLATE_HOST_ROUTE_V2,
RELEASE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE_V2,
ENDPOINT_ACTIONS_DS,
ENDPOINT_ACTION_RESPONSES_DS,
failedFleetActionErrorCode,
KILL_PROCESS_ROUTE,
SUSPEND_PROCESS_ROUTE,
GET_RUNNING_PROCESSES_ROUTE,
ISOLATE_HOST_ROUTE,
UNISOLATE_HOST_ROUTE,
} from '../../../../common/endpoint/constants';
import type {
EndpointAction,
@ -56,6 +58,34 @@ export function registerResponseActionRoutes(
) {
const logger = endpointContext.logFactory.get('hostIsolation');
/**
* @deprecated use ISOLATE_HOST_ROUTE_V2 instead
*/
router.post(
{
path: ISOLATE_HOST_ROUTE,
validate: NoParametersRequestSchema,
options: { authRequired: true, tags: ['access:securitySolution'] },
},
withEndpointAuthz({ all: ['canIsolateHost'] }, logger, redirectHandler(ISOLATE_HOST_ROUTE_V2))
);
/**
* @deprecated use RELEASE_HOST_ROUTE instead
*/
router.post(
{
path: UNISOLATE_HOST_ROUTE,
validate: NoParametersRequestSchema,
options: { authRequired: true, tags: ['access:securitySolution'] },
},
withEndpointAuthz(
{ all: ['canUnIsolateHost'] },
logger,
redirectHandler(UNISOLATE_HOST_ROUTE_V2)
)
);
router.post(
{
path: ISOLATE_HOST_ROUTE_V2,
@ -71,7 +101,7 @@ export function registerResponseActionRoutes(
router.post(
{
path: RELEASE_HOST_ROUTE,
path: UNISOLATE_HOST_ROUTE_V2,
validate: NoParametersRequestSchema,
options: { authRequired: true, tags: ['access:securitySolution'] },
},
@ -364,3 +394,19 @@ const createFailedActionResponseEntry = async ({
logger.error(e);
}
};
function redirectHandler(
location: string
): RequestHandler<
unknown,
unknown,
TypeOf<typeof NoParametersRequestSchema.body>,
SecuritySolutionRequestHandlerContext
> {
return async (_context, _req, res) => {
return res.custom({
statusCode: 308,
headers: { location },
});
};
}