From 171cd2ef85a27c8aef4b643fad3140580a1d20cd Mon Sep 17 00:00:00 2001 From: Surya Sashank Nistala Date: Fri, 26 May 2023 14:30:57 -0700 Subject: [PATCH] Adds transport layer actions for CRUD workflows (#934) (#938) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (cherry picked from commit e0b7a5a7905b977e58d80e3b9134b14893d122b0) * remove unneeded import --------- * Stashed user together with it's roles --------- * Added workflow execution logic (#850) * Added workflow execution logic * Adjusted code according to comments * Updated version of the findings json * Updating the workflow metadata in the case of updating flag set to false while the metadata alerady exist * Added logging for workflow metadata update * Added Rest Execute Workflow action * Extended workflow context with workflowMetadataId. Adjusted the doc level monitor findings * Updated conditions for unstashing the context when indexing and deleting the workflow --------- * Added fix when executing the workflow and when chained findings index… (#890) * Fixed deleting monitor workflow metadata (#882) * Fixed deleting monitor metadata and workflow metadata. * fix monitor metadata error from conflict resolution * remove unused import * remove rest execute workflow action * increment schema version for findings mapping json --------- Signed-off-by: Stevan Buzejic Signed-off-by: Angie Zhang Signed-off-by: Ashish Agrawal Signed-off-by: Surya Sashank Nistala Co-authored-by: Stevan Buzejic Co-authored-by: Angie Zhang Co-authored-by: opensearch-trigger-bot[bot] <98922864+opensearch-trigger-bot[bot]@users.noreply.github.com> Co-authored-by: Petar Dzepina Co-authored-by: Ashish Agrawal --- alerting/build.gradle | 2 + .../org/opensearch/alerting/AlertingPlugin.kt | 44 +- .../alerting/BucketLevelMonitorRunner.kt | 27 +- .../alerting/DocumentLevelMonitorRunner.kt | 52 +- .../org/opensearch/alerting/InputService.kt | 49 +- .../alerting/MonitorMetadataService.kt | 50 +- .../org/opensearch/alerting/MonitorRunner.kt | 4 +- .../alerting/MonitorRunnerExecutionContext.kt | 1 + .../alerting/QueryLevelMonitorRunner.kt | 6 +- .../alerting/WorkflowMetadataService.kt | 174 +++ .../opensearch/alerting/WorkflowService.kt | 130 ++ .../alerting/action/ExecuteWorkflowAction.kt | 15 + .../alerting/action/ExecuteWorkflowRequest.kt | 70 + .../action/ExecuteWorkflowResponse.kt | 39 + .../alerting/alerts/AlertIndices.kt | 25 +- .../alerting/model/MonitorMetadata.kt | 13 +- .../alerting/model/WorkflowMetadata.kt | 105 ++ .../alerting/model/WorkflowRunResult.kt | 53 + .../alerting/service/DeleteMonitorService.kt | 186 +++ .../transport/TransportDeleteMonitorAction.kt | 151 +- .../TransportDeleteWorkflowAction.kt | 330 +++++ .../TransportExecuteWorkflowAction.kt | 128 ++ .../transport/TransportGetWorkflowAction.kt | 148 ++ .../transport/TransportIndexMonitorAction.kt | 8 +- .../transport/TransportIndexWorkflowAction.kt | 722 ++++++++++ .../opensearch/alerting/util/AlertingUtils.kt | 2 + .../workflow/CompositeWorkflowRunner.kt | 180 +++ .../alerting/workflow/WorkflowRunContext.kt | 15 + .../alerting/workflow/WorkflowRunner.kt | 21 + .../workflow/WorkflowRunnerService.kt | 252 ++++ .../alerting/alerts/finding_mapping.json | 5 +- .../org/opensearch/alerting/TestHelpers.kt | 69 + .../opensearch/alerting/WorkflowMonitorIT.kt | 1269 +++++++++++++++++ .../opensearch/alerting/WorkflowRunnerIT.kt | 999 +++++++++++++ .../alerting/resthandler/MonitorRestApiIT.kt | 4 +- .../transport/AlertingSingleNodeTestCase.kt | 96 +- .../transport/WorkflowSingleNodeTestCase.kt | 161 +++ .../resources/mappings/scheduled-jobs.json | 170 ++- 38 files changed, 5575 insertions(+), 200 deletions(-) create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/WorkflowMetadataService.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/WorkflowService.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowAction.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowRequest.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowResponse.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowMetadata.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowRunResult.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/service/DeleteMonitorService.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteWorkflowAction.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteWorkflowAction.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportGetWorkflowAction.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexWorkflowAction.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunContext.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunner.kt create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunnerService.kt create mode 100644 alerting/src/test/kotlin/org/opensearch/alerting/WorkflowMonitorIT.kt create mode 100644 alerting/src/test/kotlin/org/opensearch/alerting/WorkflowRunnerIT.kt create mode 100644 alerting/src/test/kotlin/org/opensearch/alerting/transport/WorkflowSingleNodeTestCase.kt diff --git a/alerting/build.gradle b/alerting/build.gradle index ba2189b54..2edc37b80 100644 --- a/alerting/build.gradle +++ b/alerting/build.gradle @@ -114,6 +114,8 @@ dependencies { testImplementation "org.mockito:mockito-core:${versions.mockito}" testImplementation "org.opensearch.plugin:reindex-client:${opensearch_version}" testImplementation "org.opensearch.plugin:parent-join-client:${opensearch_version}" + testImplementation "org.opensearch.plugin:lang-painless:${opensearch_version}" + testImplementation "org.opensearch.plugin:lang-mustache-client:${opensearch_version}" } javadoc.enabled = false // turn off javadoc as it barfs on Kotlin code diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt index 9a9c75f32..babd25938 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt @@ -8,6 +8,7 @@ package org.opensearch.alerting import org.opensearch.action.ActionRequest import org.opensearch.action.ActionResponse import org.opensearch.alerting.action.ExecuteMonitorAction +import org.opensearch.alerting.action.ExecuteWorkflowAction import org.opensearch.alerting.action.GetDestinationsAction import org.opensearch.alerting.action.GetEmailAccountAction import org.opensearch.alerting.action.GetEmailGroupAction @@ -38,25 +39,31 @@ import org.opensearch.alerting.resthandler.RestSearchEmailAccountAction import org.opensearch.alerting.resthandler.RestSearchEmailGroupAction import org.opensearch.alerting.resthandler.RestSearchMonitorAction import org.opensearch.alerting.script.TriggerScript +import org.opensearch.alerting.service.DeleteMonitorService import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.alerting.settings.DestinationSettings import org.opensearch.alerting.settings.LegacyOpenDistroAlertingSettings import org.opensearch.alerting.settings.LegacyOpenDistroDestinationSettings import org.opensearch.alerting.transport.TransportAcknowledgeAlertAction import org.opensearch.alerting.transport.TransportDeleteMonitorAction +import org.opensearch.alerting.transport.TransportDeleteWorkflowAction import org.opensearch.alerting.transport.TransportExecuteMonitorAction +import org.opensearch.alerting.transport.TransportExecuteWorkflowAction import org.opensearch.alerting.transport.TransportGetAlertsAction import org.opensearch.alerting.transport.TransportGetDestinationsAction import org.opensearch.alerting.transport.TransportGetEmailAccountAction import org.opensearch.alerting.transport.TransportGetEmailGroupAction import org.opensearch.alerting.transport.TransportGetFindingsSearchAction import org.opensearch.alerting.transport.TransportGetMonitorAction +import org.opensearch.alerting.transport.TransportGetWorkflowAction import org.opensearch.alerting.transport.TransportIndexMonitorAction +import org.opensearch.alerting.transport.TransportIndexWorkflowAction import org.opensearch.alerting.transport.TransportSearchEmailAccountAction import org.opensearch.alerting.transport.TransportSearchEmailGroupAction import org.opensearch.alerting.transport.TransportSearchMonitorAction import org.opensearch.alerting.util.DocLevelMonitorQueries import org.opensearch.alerting.util.destinationmigration.DestinationMigrationCoordinator +import org.opensearch.alerting.workflow.WorkflowRunnerService import org.opensearch.client.Client import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.node.DiscoveryNodes @@ -78,6 +85,7 @@ import org.opensearch.commons.alerting.model.Monitor import org.opensearch.commons.alerting.model.QueryLevelTrigger import org.opensearch.commons.alerting.model.ScheduledJob import org.opensearch.commons.alerting.model.SearchInput +import org.opensearch.commons.alerting.model.Workflow import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.core.xcontent.XContentParser import org.opensearch.env.Environment @@ -140,6 +148,7 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R } lateinit var runner: MonitorRunnerService + lateinit var workflowRunner: WorkflowRunnerService lateinit var scheduler: JobScheduler lateinit var sweeper: JobSweeper lateinit var scheduledJobIndices: ScheduledJobIndices @@ -191,8 +200,11 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R ActionPlugin.ActionHandler(SearchEmailGroupAction.INSTANCE, TransportSearchEmailGroupAction::class.java), ActionPlugin.ActionHandler(GetDestinationsAction.INSTANCE, TransportGetDestinationsAction::class.java), ActionPlugin.ActionHandler(AlertingActions.GET_ALERTS_ACTION_TYPE, TransportGetAlertsAction::class.java), - ActionPlugin.ActionHandler(AlertingActions.GET_FINDINGS_ACTION_TYPE, TransportGetFindingsSearchAction::class.java) - + ActionPlugin.ActionHandler(AlertingActions.GET_FINDINGS_ACTION_TYPE, TransportGetFindingsSearchAction::class.java), + ActionPlugin.ActionHandler(AlertingActions.INDEX_WORKFLOW_ACTION_TYPE, TransportIndexWorkflowAction::class.java), + ActionPlugin.ActionHandler(AlertingActions.GET_WORKFLOW_ACTION_TYPE, TransportGetWorkflowAction::class.java), + ActionPlugin.ActionHandler(AlertingActions.DELETE_WORKFLOW_ACTION_TYPE, TransportDeleteWorkflowAction::class.java), + ActionPlugin.ActionHandler(ExecuteWorkflowAction.INSTANCE, TransportExecuteWorkflowAction::class.java) ) } @@ -204,7 +216,8 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R QueryLevelTrigger.XCONTENT_REGISTRY, BucketLevelTrigger.XCONTENT_REGISTRY, ClusterMetricsInput.XCONTENT_REGISTRY, - DocumentLevelTrigger.XCONTENT_REGISTRY + DocumentLevelTrigger.XCONTENT_REGISTRY, + Workflow.XCONTENT_REGISTRY ) } @@ -239,6 +252,22 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R .registerDocLevelMonitorQueries(DocLevelMonitorQueries(client, clusterService)) .registerConsumers() .registerDestinationSettings() + workflowRunner = WorkflowRunnerService + .registerClusterService(clusterService) + .registerClient(client) + .registerNamedXContentRegistry(xContentRegistry) + .registerScriptService(scriptService) + .registerIndexNameExpressionResolver(indexNameExpressionResolver) + .registerSettings(settings) + .registerThreadPool(threadPool) + .registerAlertIndices(alertIndices) + .registerInputService(InputService(client, scriptService, namedWriteableRegistry, xContentRegistry)) + .registerTriggerService(TriggerService(scriptService)) + .registerAlertService(AlertService(client, xContentRegistry, alertIndices)) + .registerDocLevelMonitorQueries(DocLevelMonitorQueries(client, clusterService)) + .registerWorkflowService(WorkflowService(client, xContentRegistry)) + .registerConsumers() + .registerDestinationSettings() scheduledJobIndices = ScheduledJobIndices(client.admin(), clusterService) docLevelMonitorQueries = DocLevelMonitorQueries(client, clusterService) scheduler = JobScheduler(threadPool, runner) @@ -254,6 +283,15 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R settings ) + WorkflowMetadataService.initialize( + client, + clusterService, + xContentRegistry, + settings + ) + + DeleteMonitorService.initialize(client) + return listOf(sweeper, scheduler, runner, scheduledJobIndices, docLevelMonitorQueries, destinationMigrationCoordinator) } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/BucketLevelMonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/BucketLevelMonitorRunner.kt index 0e4183b4e..93989c7aa 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/BucketLevelMonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/BucketLevelMonitorRunner.kt @@ -25,6 +25,7 @@ import org.opensearch.alerting.util.defaultToPerExecutionAction import org.opensearch.alerting.util.getActionExecutionPolicy import org.opensearch.alerting.util.getBucketKeysHash import org.opensearch.alerting.util.getCombinedTriggerRunResult +import org.opensearch.alerting.workflow.WorkflowRunContext import org.opensearch.common.xcontent.LoggingDeprecationHandler import org.opensearch.common.xcontent.XContentType import org.opensearch.commons.alerting.model.Alert @@ -59,7 +60,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, periodStart: Instant, periodEnd: Instant, - dryrun: Boolean + dryrun: Boolean, + workflowRunContext: WorkflowRunContext? ): MonitorRunResult { val roles = MonitorRunnerService.getRolesForMonitor(monitor) logger.debug("Running monitor: ${monitor.name} with roles: $roles Thread: ${Thread.currentThread().name}") @@ -118,7 +120,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { monitor, periodStart, periodEnd, - monitorResult.inputResults + monitorResult.inputResults, + workflowRunContext ) if (firstIteration) { firstPageOfInputResults = inputResults @@ -154,7 +157,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { monitorCtx, periodStart, periodEnd, - !dryrun && monitor.id != Monitor.NO_ID + !dryrun && monitor.id != Monitor.NO_ID, + workflowRunContext ) } else { emptyList() @@ -350,7 +354,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, periodStart: Instant, periodEnd: Instant, - shouldCreateFinding: Boolean + shouldCreateFinding: Boolean, + workflowRunContext: WorkflowRunContext? = null ): List { monitor.inputs.forEach { input -> if (input is SearchInput) { @@ -361,14 +366,14 @@ object BucketLevelMonitorRunner : MonitorRunner() { for (aggFactory in (query.aggregations() as AggregatorFactories.Builder).aggregatorFactories) { when (aggFactory) { is CompositeAggregationBuilder -> { - var grouByFields = 0 // if number of fields used to group by > 1 we won't calculate findings + var groupByFields = 0 // if number of fields used to group by > 1 we won't calculate findings val sources = aggFactory.sources() for (source in sources) { - if (grouByFields > 0) { + if (groupByFields > 0) { logger.error("grouByFields > 0. not generating findings for bucket level monitor ${monitor.id}") return listOf() } - grouByFields++ + groupByFields++ fieldName = source.field() } } @@ -409,7 +414,7 @@ object BucketLevelMonitorRunner : MonitorRunner() { sr.source().query(queryBuilder) } val searchResponse: SearchResponse = monitorCtx.client!!.suspendUntil { monitorCtx.client!!.search(sr, it) } - return createFindingPerIndex(searchResponse, monitor, monitorCtx, shouldCreateFinding) + return createFindingPerIndex(searchResponse, monitor, monitorCtx, shouldCreateFinding, workflowRunContext?.executionId) } else { logger.error("Couldn't resolve groupBy field. Not generating bucket level monitor findings for monitor %${monitor.id}") } @@ -422,7 +427,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { searchResponse: SearchResponse, monitor: Monitor, monitorCtx: MonitorRunnerExecutionContext, - shouldCreateFinding: Boolean + shouldCreateFinding: Boolean, + workflowExecutionId: String? = null ): List { val docIdsByIndexName: MutableMap> = mutableMapOf() for (hit in searchResponse.hits.hits) { @@ -441,7 +447,8 @@ object BucketLevelMonitorRunner : MonitorRunner() { monitorName = monitor.name, index = it.key, timestamp = Instant.now(), - docLevelQueries = listOf() + docLevelQueries = listOf(), + executionId = workflowExecutionId ) val findingStr = finding.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS).string() diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt index b4c43924a..bb2a04e8a 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt @@ -27,6 +27,7 @@ import org.opensearch.alerting.util.AlertingException import org.opensearch.alerting.util.IndexUtils import org.opensearch.alerting.util.defaultToPerExecutionAction import org.opensearch.alerting.util.getActionExecutionPolicy +import org.opensearch.alerting.workflow.WorkflowRunContext import org.opensearch.client.Client import org.opensearch.client.node.NodeClient import org.opensearch.cluster.metadata.IndexMetadata @@ -70,7 +71,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, periodStart: Instant, periodEnd: Instant, - dryrun: Boolean + dryrun: Boolean, + workflowRunContext: WorkflowRunContext? ): MonitorRunResult { logger.debug("Document-level-monitor is running ...") val isTempMonitor = dryrun || monitor.id == Monitor.NO_ID @@ -96,7 +98,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { var (monitorMetadata, _) = MonitorMetadataService.getOrCreateMetadata( monitor = monitor, createWithRunContext = false, - skipIndex = isTempMonitor + skipIndex = isTempMonitor, + workflowRunContext?.workflowMetadataId ) val docLevelMonitorInput = monitor.inputs[0] as DocLevelMonitorInput @@ -135,6 +138,9 @@ object DocumentLevelMonitorRunner : MonitorRunner() { } } + // Map of document ids per index when monitor is workflow delegate and has chained findings + val matchingDocIdsPerIndex = workflowRunContext?.matchingDocIdsPerIndex + indices.forEach { indexName -> // Prepare lastRunContext for each index val indexLastRunContext = lastRunContext.getOrPut(indexName) { @@ -169,7 +175,13 @@ object DocumentLevelMonitorRunner : MonitorRunner() { // Prepare DocumentExecutionContext for each index val docExecutionContext = DocumentExecutionContext(queries, indexLastRunContext, indexUpdatedRunContext) - val matchingDocs = getMatchingDocs(monitor, monitorCtx, docExecutionContext, indexName) + val matchingDocs = getMatchingDocs( + monitor, + monitorCtx, + docExecutionContext, + indexName, + matchingDocIdsPerIndex?.get(indexName) + ) if (matchingDocs.isNotEmpty()) { val matchedQueriesForDocs = getMatchedQueries( @@ -226,7 +238,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { idQueryMap, docsToQueries, queryToDocIds, - dryrun + dryrun, + workflowRunContext?.executionId ) } } @@ -298,7 +311,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { idQueryMap: Map, docsToQueries: Map>, queryToDocIds: Map>, - dryrun: Boolean + dryrun: Boolean, + workflowExecutionId: String? = null ): DocumentLevelTriggerRunResult { val triggerCtx = DocumentLevelTriggerExecutionContext(monitor, trigger) val triggerResult = monitorCtx.triggerService!!.runDocLevelTrigger(monitor, trigger, queryToDocIds) @@ -309,7 +323,14 @@ object DocumentLevelMonitorRunner : MonitorRunner() { // TODO: Implement throttling for findings docsToQueries.forEach { val triggeredQueries = it.value.map { queryId -> idQueryMap[queryId]!! } - val findingId = createFindings(monitor, monitorCtx, triggeredQueries, it.key, !dryrun && monitor.id != Monitor.NO_ID) + val findingId = createFindings( + monitor, + monitorCtx, + triggeredQueries, + it.key, + !dryrun && monitor.id != Monitor.NO_ID, + workflowExecutionId + ) findings.add(findingId) if (triggerResult.triggeredDocs.contains(it.key)) { @@ -379,7 +400,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, docLevelQueries: List, matchingDocId: String, - shouldCreateFinding: Boolean + shouldCreateFinding: Boolean, + workflowExecutionId: String? = null, ): String { // Before the "|" is the doc id and after the "|" is the index val docIndex = matchingDocId.split("|") @@ -392,7 +414,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { monitorName = monitor.name, index = docIndex[1], docLevelQueries = docLevelQueries, - timestamp = Instant.now() + timestamp = Instant.now(), + executionId = workflowExecutionId ) val findingStr = finding.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS).string() @@ -514,7 +537,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { monitor: Monitor, monitorCtx: MonitorRunnerExecutionContext, docExecutionCtx: DocumentExecutionContext, - index: String + index: String, + docIds: List? = null ): List> { val count: Int = docExecutionCtx.updatedLastRunContext["shards_count"] as Int val matchingDocs = mutableListOf>() @@ -530,7 +554,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { shard, prevSeqNo, maxSeqNo, - null + null, + docIds ) if (hits.hits.isNotEmpty()) { @@ -549,7 +574,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { shard: String, prevSeqNo: Long?, maxSeqNo: Long, - query: String? + query: String?, + docIds: List? = null ): SearchHits { if (prevSeqNo?.equals(maxSeqNo) == true && maxSeqNo != 0L) { return SearchHits.empty() @@ -561,6 +587,10 @@ object DocumentLevelMonitorRunner : MonitorRunner() { boolQueryBuilder.must(QueryBuilders.queryStringQuery(query)) } + if (!docIds.isNullOrEmpty()) { + boolQueryBuilder.filter(QueryBuilders.termsQuery("_id", docIds)) + } + val request: SearchRequest = SearchRequest() .indices(index) .preference("_shards:$shard") diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/InputService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/InputService.kt index e2b464064..5013b1eff 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/InputService.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/InputService.kt @@ -16,6 +16,7 @@ import org.opensearch.alerting.util.AggregationQueryRewriter import org.opensearch.alerting.util.addUserBackendRolesFilter import org.opensearch.alerting.util.executeTransportAction import org.opensearch.alerting.util.toMap +import org.opensearch.alerting.workflow.WorkflowRunContext import org.opensearch.client.Client import org.opensearch.common.io.stream.BytesStreamOutput import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput @@ -26,6 +27,11 @@ import org.opensearch.commons.alerting.model.ClusterMetricsInput import org.opensearch.commons.alerting.model.Monitor import org.opensearch.commons.alerting.model.SearchInput import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.index.query.BoolQueryBuilder +import org.opensearch.index.query.MatchQueryBuilder +import org.opensearch.index.query.QueryBuilder +import org.opensearch.index.query.QueryBuilders +import org.opensearch.index.query.TermsQueryBuilder import org.opensearch.script.Script import org.opensearch.script.ScriptService import org.opensearch.script.ScriptType @@ -47,12 +53,16 @@ class InputService( monitor: Monitor, periodStart: Instant, periodEnd: Instant, - prevResult: InputRunResults? = null + prevResult: InputRunResults? = null, + workflowRunContext: WorkflowRunContext? = null ): InputRunResults { return try { val results = mutableListOf>() val aggTriggerAfterKey: MutableMap = mutableMapOf() + // If monitor execution is triggered from a workflow + val matchingDocIdsPerIndex = workflowRunContext?.matchingDocIdsPerIndex + // TODO: If/when multiple input queries are supported for Bucket-Level Monitor execution, aggTriggerAfterKeys will // need to be updated to account for it monitor.inputs.forEach { input -> @@ -63,9 +73,17 @@ class InputService( "period_start" to periodStart.toEpochMilli(), "period_end" to periodEnd.toEpochMilli() ) + // Deep copying query before passing it to rewriteQuery since otherwise, the monitor.input is modified directly // which causes a strange bug where the rewritten query persists on the Monitor across executions val rewrittenQuery = AggregationQueryRewriter.rewriteQuery(deepCopyQuery(input.query), prevResult, monitor.triggers) + + // Rewrite query to consider the doc ids per given index + if (chainedFindingExist(matchingDocIdsPerIndex) && rewrittenQuery.query() != null) { + val updatedSourceQuery = updateInputQueryWithFindingDocIds(rewrittenQuery.query(), matchingDocIdsPerIndex!!) + rewrittenQuery.query(updatedSourceQuery) + } + val searchSource = scriptService.compile( Script( ScriptType.INLINE, @@ -107,6 +125,35 @@ class InputService( } } + /** + * Extends the given query builder with query that filters the given indices with the given doc ids per index + * Used whenever we want to select the documents that were found in chained delegate execution of the current workflow run + * + * @param query Original bucket monitor query + * @param matchingDocIdsPerIndex Map of finding doc ids grouped by index + */ + private fun updateInputQueryWithFindingDocIds( + query: QueryBuilder, + matchingDocIdsPerIndex: Map>, + ): QueryBuilder { + val queryBuilder = QueryBuilders.boolQuery().must(query) + val shouldQuery = QueryBuilders.boolQuery() + + matchingDocIdsPerIndex.forEach { entry -> + shouldQuery + .should() + .add( + BoolQueryBuilder() + .must(MatchQueryBuilder("_index", entry.key)) + .must(TermsQueryBuilder("_id", entry.value)) + ) + } + return queryBuilder.must(shouldQuery) + } + + private fun chainedFindingExist(indexToDocIds: Map>?) = + !indexToDocIds.isNullOrEmpty() + private fun deepCopyQuery(query: SearchSourceBuilder): SearchSourceBuilder { val out = BytesStreamOutput() query.writeTo(out) diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt index 2a6a57642..58153a5f7 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt @@ -57,13 +57,14 @@ object MonitorMetadataService : private lateinit var clusterService: ClusterService private lateinit var settings: Settings - @Volatile private lateinit var indexTimeout: TimeValue + @Volatile + private lateinit var indexTimeout: TimeValue fun initialize( client: Client, clusterService: ClusterService, xContentRegistry: NamedXContentRegistry, - settings: Settings + settings: Settings, ) { this.clusterService = clusterService this.client = client @@ -110,18 +111,24 @@ object MonitorMetadataService : } } + /** + * Document monitors are keeping the context of the last run. + * Since one monitor can be part of multiple workflows we need to be sure that execution of the current workflow + * doesn't interfere with the other workflows that are dependent on the given monitor + */ suspend fun getOrCreateMetadata( monitor: Monitor, createWithRunContext: Boolean = true, - skipIndex: Boolean = false + skipIndex: Boolean = false, + workflowMetadataId: String? = null, ): Pair { try { val created = true - val metadata = getMetadata(monitor) + val metadata = getMetadata(monitor, workflowMetadataId) return if (metadata != null) { metadata to !created } else { - val newMetadata = createNewMetadata(monitor, createWithRunContext = createWithRunContext) + val newMetadata = createNewMetadata(monitor, createWithRunContext = createWithRunContext, workflowMetadataId) if (skipIndex) { newMetadata to created } else { @@ -133,9 +140,9 @@ object MonitorMetadataService : } } - suspend fun getMetadata(monitor: Monitor): MonitorMetadata? { + suspend fun getMetadata(monitor: Monitor, workflowMetadataId: String? = null): MonitorMetadata? { try { - val metadataId = MonitorMetadata.getId(monitor) + val metadataId = MonitorMetadata.getId(monitor, workflowMetadataId) val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, metadataId).routing(monitor.id) val getResponse: GetResponse = client.suspendUntil { get(getRequest, it) } @@ -168,28 +175,31 @@ object MonitorMetadataService : val runContext = if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { createFullRunContext(monitorIndex, metadata.lastRunContext as MutableMap>) } else null - if (runContext != null) { - return metadata.copy( + return if (runContext != null) { + metadata.copy( lastRunContext = runContext ) } else { - return metadata + metadata } } catch (e: Exception) { throw AlertingException.wrap(e) } } - private suspend fun createNewMetadata(monitor: Monitor, createWithRunContext: Boolean): MonitorMetadata { - val monitorIndex = if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { + private suspend fun createNewMetadata( + monitor: Monitor, + createWithRunContext: Boolean, + workflowMetadataId: String? = null, + ): MonitorMetadata { + val monitorIndex = if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) (monitor.inputs[0] as DocLevelMonitorInput).indices[0] - } else null - val runContext = - if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR && createWithRunContext) { - createFullRunContext(monitorIndex) - } else emptyMap() + else null + val runContext = if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR && createWithRunContext) + createFullRunContext(monitorIndex) + else emptyMap() return MonitorMetadata( - id = "${monitor.id}-metadata", + id = MonitorMetadata.getId(monitor, workflowMetadataId), seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO, primaryTerm = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, monitorId = monitor.id, @@ -199,9 +209,9 @@ object MonitorMetadataService : ) } - private suspend fun createFullRunContext( + suspend fun createFullRunContext( index: String?, - existingRunContext: MutableMap>? = null + existingRunContext: MutableMap>? = null, ): MutableMap> { val lastRunContext = existingRunContext?.toMutableMap() ?: mutableMapOf() try { diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt index 8d9aca954..419962578 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt @@ -24,6 +24,7 @@ import org.opensearch.alerting.util.destinationmigration.publishLegacyNotificati import org.opensearch.alerting.util.destinationmigration.sendNotification import org.opensearch.alerting.util.isAllowed import org.opensearch.alerting.util.isTestAction +import org.opensearch.alerting.workflow.WorkflowRunContext import org.opensearch.client.node.NodeClient import org.opensearch.common.Strings import org.opensearch.commons.alerting.model.Monitor @@ -39,7 +40,8 @@ abstract class MonitorRunner { monitorCtx: MonitorRunnerExecutionContext, periodStart: Instant, periodEnd: Instant, - dryRun: Boolean + dryRun: Boolean, + workflowRunContext: WorkflowRunContext? = null ): MonitorRunResult<*> suspend fun runAction( diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt index 67fa29877..41a26bb79 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt @@ -35,6 +35,7 @@ data class MonitorRunnerExecutionContext( var triggerService: TriggerService? = null, var alertService: AlertService? = null, var docLevelMonitorQueries: DocLevelMonitorQueries? = null, + var workflowService: WorkflowService? = null, @Volatile var retryPolicy: BackoffPolicy? = null, @Volatile var moveAlertsRetryPolicy: BackoffPolicy? = null, diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/QueryLevelMonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/QueryLevelMonitorRunner.kt index ea5e173a0..884ad372d 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/QueryLevelMonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/QueryLevelMonitorRunner.kt @@ -12,6 +12,7 @@ import org.opensearch.alerting.opensearchapi.InjectorContextElement import org.opensearch.alerting.opensearchapi.withClosableContext import org.opensearch.alerting.script.QueryLevelTriggerExecutionContext import org.opensearch.alerting.util.isADMonitor +import org.opensearch.alerting.workflow.WorkflowRunContext import org.opensearch.commons.alerting.model.Alert import org.opensearch.commons.alerting.model.Monitor import org.opensearch.commons.alerting.model.QueryLevelTrigger @@ -25,7 +26,8 @@ object QueryLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, periodStart: Instant, periodEnd: Instant, - dryrun: Boolean + dryrun: Boolean, + workflowRunContext: WorkflowRunContext? ): MonitorRunResult { val roles = MonitorRunnerService.getRolesForMonitor(monitor) logger.debug("Running monitor: ${monitor.name} with roles: $roles Thread: ${Thread.currentThread().name}") @@ -48,7 +50,7 @@ object QueryLevelMonitorRunner : MonitorRunner() { if (!isADMonitor(monitor)) { withClosableContext(InjectorContextElement(monitor.id, monitorCtx.settings!!, monitorCtx.threadPool!!.threadContext, roles)) { monitorResult = monitorResult.copy( - inputResults = monitorCtx.inputService!!.collectInputResults(monitor, periodStart, periodEnd) + inputResults = monitorCtx.inputService!!.collectInputResults(monitor, periodStart, periodEnd, null, workflowRunContext) ) } } else { diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowMetadataService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowMetadataService.kt new file mode 100644 index 000000000..7020732fd --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowMetadataService.kt @@ -0,0 +1,174 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting + +import kotlinx.coroutines.CoroutineName +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.SupervisorJob +import org.apache.logging.log4j.LogManager +import org.opensearch.OpenSearchException +import org.opensearch.action.DocWriteRequest +import org.opensearch.action.DocWriteResponse +import org.opensearch.action.get.GetRequest +import org.opensearch.action.get.GetResponse +import org.opensearch.action.index.IndexRequest +import org.opensearch.action.index.IndexResponse +import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.model.WorkflowMetadata +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.settings.AlertingSettings +import org.opensearch.alerting.util.AlertingException +import org.opensearch.client.Client +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.settings.Settings +import org.opensearch.common.unit.TimeValue +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentFactory +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentParserUtils +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.model.CompositeInput +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentParser +import org.opensearch.rest.RestStatus +import java.time.Instant +import java.time.LocalDateTime +import java.time.ZoneOffset +import java.util.UUID + +object WorkflowMetadataService : + CoroutineScope by CoroutineScope(SupervisorJob() + Dispatchers.Default + CoroutineName("WorkflowMetadataService")) { + private val log = LogManager.getLogger(this::class.java) + + private lateinit var client: Client + private lateinit var xContentRegistry: NamedXContentRegistry + private lateinit var clusterService: ClusterService + private lateinit var settings: Settings + + @Volatile private lateinit var indexTimeout: TimeValue + + fun initialize( + client: Client, + clusterService: ClusterService, + xContentRegistry: NamedXContentRegistry, + settings: Settings + ) { + this.clusterService = clusterService + this.client = client + this.xContentRegistry = xContentRegistry + this.settings = settings + this.indexTimeout = AlertingSettings.INDEX_TIMEOUT.get(settings) + this.clusterService.clusterSettings.addSettingsUpdateConsumer(AlertingSettings.INDEX_TIMEOUT) { indexTimeout = it } + } + + @Suppress("ComplexMethod", "ReturnCount") + suspend fun upsertWorkflowMetadata(metadata: WorkflowMetadata, updating: Boolean): WorkflowMetadata { + try { + val indexRequest = IndexRequest(ScheduledJob.SCHEDULED_JOBS_INDEX) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .source(metadata.toXContent(XContentFactory.jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) + .id(metadata.id) + .routing(metadata.workflowId) + .timeout(indexTimeout) + + if (updating) { + indexRequest.id(metadata.id) + } else { + indexRequest.opType(DocWriteRequest.OpType.CREATE) + } + val response: IndexResponse = client.suspendUntil { index(indexRequest, it) } + when (response.result) { + DocWriteResponse.Result.DELETED, DocWriteResponse.Result.NOOP, DocWriteResponse.Result.NOT_FOUND, null -> { + val failureReason = "The upsert metadata call failed with a ${response.result?.lowercase} result" + log.error(failureReason) + throw AlertingException(failureReason, RestStatus.INTERNAL_SERVER_ERROR, IllegalStateException(failureReason)) + } + DocWriteResponse.Result.CREATED, DocWriteResponse.Result.UPDATED -> { + log.debug("Successfully upserted WorkflowMetadata:${metadata.id} ") + } + } + return metadata + } catch (e: Exception) { + // If the update is set to false and id is set conflict exception will be thrown + if (e is OpenSearchException && e.status() == RestStatus.CONFLICT && !updating) { + log.debug( + "Metadata with ${metadata.id} for workflow ${metadata.workflowId} already exist." + + " Instead of creating new, updating existing metadata will be performed" + ) + return upsertWorkflowMetadata(metadata, true) + } + log.error("Error saving metadata", e) + throw AlertingException.wrap(e) + } + } + + suspend fun getOrCreateWorkflowMetadata( + workflow: Workflow, + skipIndex: Boolean = false, + executionId: String + ): Pair { + try { + val created = true + val metadata = getWorkflowMetadata(workflow) + return if (metadata != null) { + metadata to !created + } else { + val newMetadata = createNewWorkflowMetadata(workflow, executionId, skipIndex) + if (skipIndex) { + newMetadata to created + } else { + upsertWorkflowMetadata(newMetadata, updating = false) to created + } + } + } catch (e: Exception) { + throw AlertingException.wrap(e) + } + } + + private suspend fun getWorkflowMetadata(workflow: Workflow): WorkflowMetadata? { + try { + val metadataId = WorkflowMetadata.getId(workflow.id) + val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, metadataId).routing(workflow.id) + + val getResponse: GetResponse = client.suspendUntil { get(getRequest, it) } + return if (getResponse.isExists) { + val xcp = XContentHelper.createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, + getResponse.sourceAsBytesRef, + XContentType.JSON + ) + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.nextToken(), xcp) + WorkflowMetadata.parse(xcp) + } else { + null + } + } catch (e: Exception) { + if (e.message?.contains("no such index") == true) { + return null + } else { + throw AlertingException.wrap(e) + } + } + } + + private fun createNewWorkflowMetadata(workflow: Workflow, executionId: String, isTempWorkflow: Boolean): WorkflowMetadata { + // In the case of temp workflow (ie. workflow is in dry-run) use timestampWithUUID-metadata format + // In the case of regular workflow execution, use the workflowId-metadata format + val id = if (isTempWorkflow) "${LocalDateTime.now(ZoneOffset.UTC)}${UUID.randomUUID()}" else workflow.id + return WorkflowMetadata( + id = WorkflowMetadata.getId(id), + workflowId = workflow.id, + monitorIds = (workflow.inputs[0] as CompositeInput).getMonitorIds(), + latestRunTime = Instant.now(), + latestExecutionId = executionId + ) + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowService.kt new file mode 100644 index 000000000..0fa93587f --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/WorkflowService.kt @@ -0,0 +1,130 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting + +import org.apache.logging.log4j.LogManager +import org.opensearch.OpenSearchException +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.util.AlertingException +import org.opensearch.client.Client +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentFactory +import org.opensearch.common.xcontent.XContentParserUtils +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.model.Finding +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.XContentParser +import org.opensearch.index.query.QueryBuilders +import org.opensearch.search.builder.SearchSourceBuilder + +private val log = LogManager.getLogger(WorkflowService::class.java) + +/** + * Contains util methods used in workflow execution + */ +class WorkflowService( + val client: Client, + val xContentRegistry: NamedXContentRegistry, +) { + /** + * Returns finding doc ids per index for the given workflow execution + * Used for pre-filtering the dataset in the case of creating a workflow with chained findings + * + * @param chainedMonitor Monitor that is previously executed + * @param workflowExecutionId Execution id of the current workflow + */ + suspend fun getFindingDocIdsByExecutionId(chainedMonitor: Monitor, workflowExecutionId: String): Map> { + try { + // Search findings index per monitor and workflow execution id + val bqb = QueryBuilders.boolQuery().filter(QueryBuilders.termQuery(Finding.MONITOR_ID_FIELD, chainedMonitor.id)) + .filter(QueryBuilders.termQuery(Finding.EXECUTION_ID_FIELD, workflowExecutionId)) + val searchRequest = SearchRequest() + .source( + SearchSourceBuilder() + .query(bqb) + .version(true) + .seqNoAndPrimaryTerm(true) + ) + .indices(chainedMonitor.dataSources.findingsIndex) + val searchResponse: SearchResponse = client.suspendUntil { client.search(searchRequest, it) } + + // Get the findings docs + val findings = mutableListOf() + for (hit in searchResponse.hits) { + val xcp = XContentFactory.xContent(XContentType.JSON) + .createParser(xContentRegistry, LoggingDeprecationHandler.INSTANCE, hit.sourceAsString) + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.nextToken(), xcp) + val finding = Finding.parse(xcp) + findings.add(finding) + } + // Based on the findings get the document ids + val indexToRelatedDocIdsMap = mutableMapOf>() + for (finding in findings) { + indexToRelatedDocIdsMap.getOrPut(finding.index) { mutableListOf() }.addAll(finding.relatedDocIds) + } + return indexToRelatedDocIdsMap + } catch (t: Exception) { + log.error("Error getting finding doc ids: ${t.message}", t) + throw AlertingException.wrap(t) + } + } + + /** + * Returns the list of monitors for the given ids + * Used in workflow execution in order to figure out the monitor type + * + * @param monitors List of monitor ids + * @param size Expected number of monitors + */ + suspend fun getMonitorsById(monitors: List, size: Int): List { + try { + val bqb = QueryBuilders.boolQuery().filter(QueryBuilders.termsQuery("_id", monitors)) + + val searchRequest = SearchRequest() + .source( + SearchSourceBuilder() + .query(bqb) + .version(true) + .seqNoAndPrimaryTerm(true) + .size(size) + ) + .indices(ScheduledJob.SCHEDULED_JOBS_INDEX) + + val searchResponse: SearchResponse = client.suspendUntil { client.search(searchRequest, it) } + return parseMonitors(searchResponse) + } catch (e: Exception) { + log.error("Error getting monitors: ${e.message}", e) + throw AlertingException.wrap(e) + } + } + + private fun parseMonitors(response: SearchResponse): List { + if (response.isTimedOut) { + log.error("Request for getting monitors timeout") + throw OpenSearchException("Cannot determine that the ${ScheduledJob.SCHEDULED_JOBS_INDEX} index is healthy") + } + val monitors = mutableListOf() + try { + for (hit in response.hits) { + XContentType.JSON.xContent().createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, hit.sourceAsString + ).use { hitsParser -> + val monitor = ScheduledJob.parse(hitsParser, hit.id, hit.version) as Monitor + monitors.add(monitor) + } + } + } catch (e: Exception) { + log.error("Error parsing monitors: ${e.message}", e) + throw AlertingException.wrap(e) + } + return monitors + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowAction.kt new file mode 100644 index 000000000..efed1087d --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowAction.kt @@ -0,0 +1,15 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.action + +import org.opensearch.action.ActionType + +class ExecuteWorkflowAction private constructor() : ActionType(NAME, ::ExecuteWorkflowResponse) { + companion object { + val INSTANCE = ExecuteWorkflowAction() + const val NAME = "cluster:admin/opensearch/alerting/workflow/execute" + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowRequest.kt b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowRequest.kt new file mode 100644 index 000000000..2d97bbdcc --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowRequest.kt @@ -0,0 +1,70 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.action + +import org.opensearch.action.ActionRequest +import org.opensearch.action.ActionRequestValidationException +import org.opensearch.action.ValidateActions +import org.opensearch.common.io.stream.StreamInput +import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.common.unit.TimeValue +import org.opensearch.commons.alerting.model.Workflow +import java.io.IOException + +/** + * A class containing workflow details. + */ +class ExecuteWorkflowRequest : ActionRequest { + val dryrun: Boolean + val requestEnd: TimeValue + val workflowId: String? + val workflow: Workflow? + + constructor( + dryrun: Boolean, + requestEnd: TimeValue, + workflowId: String?, + workflow: Workflow?, + ) : super() { + this.dryrun = dryrun + this.requestEnd = requestEnd + this.workflowId = workflowId + this.workflow = workflow + } + + @Throws(IOException::class) + constructor(sin: StreamInput) : this( + sin.readBoolean(), + sin.readTimeValue(), + sin.readOptionalString(), + if (sin.readBoolean()) { + Workflow.readFrom(sin) + } else null + ) + + override fun validate(): ActionRequestValidationException? { + var validationException: ActionRequestValidationException? = null + if (workflowId == null && workflow == null) { + validationException = ValidateActions.addValidationError( + "Both workflow and workflow id are missing", validationException + ) + } + return validationException + } + + @Throws(IOException::class) + override fun writeTo(out: StreamOutput) { + out.writeBoolean(dryrun) + out.writeTimeValue(requestEnd) + out.writeOptionalString(workflowId) + if (workflow != null) { + out.writeBoolean(true) + workflow.writeTo(out) + } else { + out.writeBoolean(false) + } + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowResponse.kt b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowResponse.kt new file mode 100644 index 000000000..300f15f75 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/action/ExecuteWorkflowResponse.kt @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.action + +import org.opensearch.action.ActionResponse +import org.opensearch.alerting.model.WorkflowRunResult +import org.opensearch.common.io.stream.StreamInput +import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder +import java.io.IOException + +class ExecuteWorkflowResponse : ActionResponse, ToXContentObject { + val workflowRunResult: WorkflowRunResult + constructor( + workflowRunResult: WorkflowRunResult + ) : super() { + this.workflowRunResult = workflowRunResult + } + + @Throws(IOException::class) + constructor(sin: StreamInput) : this( + WorkflowRunResult(sin) + ) + + @Throws(IOException::class) + override fun writeTo(out: StreamOutput) { + workflowRunResult.writeTo(out) + } + + @Throws(IOException::class) + override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { + return workflowRunResult.toXContent(builder, params) + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt b/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt index 1492c7681..25ef422e8 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt @@ -137,18 +137,23 @@ class AlertIndices( } @Volatile private var alertHistoryEnabled = AlertingSettings.ALERT_HISTORY_ENABLED.get(settings) + @Volatile private var findingHistoryEnabled = AlertingSettings.FINDING_HISTORY_ENABLED.get(settings) @Volatile private var alertHistoryMaxDocs = AlertingSettings.ALERT_HISTORY_MAX_DOCS.get(settings) + @Volatile private var findingHistoryMaxDocs = AlertingSettings.FINDING_HISTORY_MAX_DOCS.get(settings) @Volatile private var alertHistoryMaxAge = AlertingSettings.ALERT_HISTORY_INDEX_MAX_AGE.get(settings) + @Volatile private var findingHistoryMaxAge = AlertingSettings.FINDING_HISTORY_INDEX_MAX_AGE.get(settings) @Volatile private var alertHistoryRolloverPeriod = AlertingSettings.ALERT_HISTORY_ROLLOVER_PERIOD.get(settings) + @Volatile private var findingHistoryRolloverPeriod = AlertingSettings.FINDING_HISTORY_ROLLOVER_PERIOD.get(settings) @Volatile private var alertHistoryRetentionPeriod = AlertingSettings.ALERT_HISTORY_RETENTION_PERIOD.get(settings) + @Volatile private var findingHistoryRetentionPeriod = AlertingSettings.FINDING_HISTORY_RETENTION_PERIOD.get(settings) @Volatile private var requestTimeout = AlertingSettings.REQUEST_TIMEOUT.get(settings) @@ -449,17 +454,25 @@ class AlertIndices( private fun rolloverAlertHistoryIndex() { rolloverIndex( - alertHistoryIndexInitialized, ALERT_HISTORY_WRITE_INDEX, - ALERT_HISTORY_INDEX_PATTERN, alertMapping(), - alertHistoryMaxDocs, alertHistoryMaxAge, ALERT_HISTORY_WRITE_INDEX + alertHistoryIndexInitialized, + ALERT_HISTORY_WRITE_INDEX, + ALERT_HISTORY_INDEX_PATTERN, + alertMapping(), + alertHistoryMaxDocs, + alertHistoryMaxAge, + ALERT_HISTORY_WRITE_INDEX ) } private fun rolloverFindingHistoryIndex() { rolloverIndex( - findingHistoryIndexInitialized, FINDING_HISTORY_WRITE_INDEX, - FINDING_HISTORY_INDEX_PATTERN, findingMapping(), - findingHistoryMaxDocs, findingHistoryMaxAge, FINDING_HISTORY_WRITE_INDEX + findingHistoryIndexInitialized, + FINDING_HISTORY_WRITE_INDEX, + FINDING_HISTORY_INDEX_PATTERN, + findingMapping(), + findingHistoryMaxDocs, + findingHistoryMaxAge, + FINDING_HISTORY_WRITE_INDEX ) } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt b/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt index 30a7f742c..565e5f9d6 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt @@ -120,8 +120,17 @@ data class MonitorMetadata( return MonitorMetadata(sin) } - fun getId(monitor: Monitor): String { - return monitor.id + "-metadata" + /** workflowMetadataId is used as key for monitor metadata in the case when the workflow execution happens + so the monitor lastRunContext (in the case of doc level monitor) is not interfering with the monitor execution + WorkflowMetadataId will be either workflowId-metadata (when executing the workflow as it is scheduled) + or timestampWithUUID-metadata (when a workflow is executed in a dry-run mode) + In the case of temp workflow, doc level monitors must have lastRunContext created from scratch + That's why we are using workflowMetadataId - in order to ensure that the doc level monitor metadata is created from scratch + **/ + fun getId(monitor: Monitor, workflowMetadataId: String? = null): String { + return if (workflowMetadataId.isNullOrEmpty()) "${monitor.id}-metadata" + // WorkflowMetadataId already contains -metadata suffix + else "$workflowMetadataId-${monitor.id}-metadata" } } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowMetadata.kt b/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowMetadata.kt new file mode 100644 index 000000000..ba4cf1201 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowMetadata.kt @@ -0,0 +1,105 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.model + +import org.opensearch.common.io.stream.StreamInput +import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.common.io.stream.Writeable +import org.opensearch.common.xcontent.XContentParserUtils +import org.opensearch.commons.alerting.util.instant +import org.opensearch.commons.alerting.util.optionalTimeField +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser +import java.io.IOException +import java.time.Instant + +data class WorkflowMetadata( + val id: String, + val workflowId: String, + val monitorIds: List, + val latestRunTime: Instant, + val latestExecutionId: String +) : Writeable, ToXContent { + + @Throws(IOException::class) + constructor(sin: StreamInput) : this( + id = sin.readString(), + workflowId = sin.readString(), + monitorIds = sin.readStringList(), + latestRunTime = sin.readInstant(), + latestExecutionId = sin.readString() + ) + + override fun writeTo(out: StreamOutput) { + out.writeString(id) + out.writeString(workflowId) + out.writeStringCollection(monitorIds) + out.writeInstant(latestRunTime) + out.writeString(latestExecutionId) + } + + override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { + builder.startObject() + if (params.paramAsBoolean("with_type", false)) builder.startObject(METADATA) + builder.field(WORKFLOW_ID_FIELD, workflowId) + .field(MONITOR_IDS_FIELD, monitorIds) + .optionalTimeField(LATEST_RUN_TIME, latestRunTime) + .field(LATEST_EXECUTION_ID, latestExecutionId) + if (params.paramAsBoolean("with_type", false)) builder.endObject() + return builder.endObject() + } + + companion object { + const val METADATA = "workflow_metadata" + const val WORKFLOW_ID_FIELD = "workflow_id" + const val MONITOR_IDS_FIELD = "monitor_ids" + const val LATEST_RUN_TIME = "latest_run_time" + const val LATEST_EXECUTION_ID = "latest_execution_id" + + @JvmStatic @JvmOverloads + @Throws(IOException::class) + fun parse(xcp: XContentParser): WorkflowMetadata { + lateinit var workflowId: String + var monitorIds = mutableListOf() + lateinit var latestRunTime: Instant + lateinit var latestExecutionId: String + + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.currentToken(), xcp) + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + val fieldName = xcp.currentName() + xcp.nextToken() + + when (fieldName) { + WORKFLOW_ID_FIELD -> workflowId = xcp.text() + MONITOR_IDS_FIELD -> { + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_ARRAY, xcp.currentToken(), xcp) + while (xcp.nextToken() != XContentParser.Token.END_ARRAY) { + monitorIds.add(xcp.text()) + } + } + LATEST_RUN_TIME -> latestRunTime = xcp.instant()!! + LATEST_EXECUTION_ID -> latestExecutionId = xcp.text() + } + } + return WorkflowMetadata( + id = "$workflowId-metadata", + workflowId = workflowId, + monitorIds = monitorIds, + latestRunTime = latestRunTime, + latestExecutionId = latestExecutionId + ) + } + + @JvmStatic + @Throws(IOException::class) + fun readFrom(sin: StreamInput): WorkflowMetadata { + return WorkflowMetadata(sin) + } + + fun getId(workflowId: String? = null) = "$workflowId-metadata" + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowRunResult.kt b/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowRunResult.kt new file mode 100644 index 000000000..9ef7a3536 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/model/WorkflowRunResult.kt @@ -0,0 +1,53 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.model + +import org.opensearch.common.io.stream.StreamInput +import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import java.io.IOException +import java.lang.Exception +import java.time.Instant + +data class WorkflowRunResult( + val workflowRunResult: List> = mutableListOf(), + val executionStartTime: Instant, + val executionEndTime: Instant? = null, + val executionId: String, + val error: Exception? = null +) : Writeable, ToXContent { + + @Throws(IOException::class) + @Suppress("UNCHECKED_CAST") + constructor(sin: StreamInput) : this( + sin.readList> { s: StreamInput -> MonitorRunResult.readFrom(s) }, + sin.readInstant(), + sin.readInstant(), + sin.readString(), + sin.readException() + ) + + override fun writeTo(out: StreamOutput) { + out.writeList(workflowRunResult) + out.writeInstant(executionStartTime) + out.writeInstant(executionEndTime) + out.writeString(executionId) + out.writeException(error) + } + + override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { + builder.startObject().startArray("workflow_run_result") + for (monitorResult in workflowRunResult) { + monitorResult.toXContent(builder, ToXContent.EMPTY_PARAMS) + } + builder.endArray().field("execution_start_time", executionStartTime) + .field("execution_end_time", executionEndTime) + .field("error", error?.message).endObject() + return builder + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/service/DeleteMonitorService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/service/DeleteMonitorService.kt new file mode 100644 index 000000000..a21ec379e --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/service/DeleteMonitorService.kt @@ -0,0 +1,186 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.service + +import kotlinx.coroutines.CoroutineName +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.SupervisorJob +import org.apache.logging.log4j.LogManager +import org.apache.lucene.search.join.ScoreMode +import org.opensearch.action.ActionListener +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsRequest +import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse +import org.opensearch.action.delete.DeleteRequest +import org.opensearch.action.delete.DeleteResponse +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +import org.opensearch.action.support.IndicesOptions +import org.opensearch.action.support.WriteRequest.RefreshPolicy +import org.opensearch.action.support.master.AcknowledgedResponse +import org.opensearch.alerting.MonitorMetadataService +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.transport.TransportDeleteWorkflowAction.Companion.WORKFLOW_DELEGATE_PATH +import org.opensearch.alerting.transport.TransportDeleteWorkflowAction.Companion.WORKFLOW_MONITOR_PATH +import org.opensearch.alerting.util.AlertingException +import org.opensearch.client.Client +import org.opensearch.commons.alerting.action.DeleteMonitorResponse +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.index.query.QueryBuilders +import org.opensearch.index.reindex.BulkByScrollResponse +import org.opensearch.index.reindex.DeleteByQueryAction +import org.opensearch.index.reindex.DeleteByQueryRequestBuilder +import org.opensearch.search.builder.SearchSourceBuilder +import kotlin.coroutines.resume +import kotlin.coroutines.resumeWithException +import kotlin.coroutines.suspendCoroutine + +/** + * Component used when deleting the monitors + */ +object DeleteMonitorService : + CoroutineScope by CoroutineScope(SupervisorJob() + Dispatchers.Default + CoroutineName("WorkflowMetadataService")) { + private val log = LogManager.getLogger(this.javaClass) + + private lateinit var client: Client + + fun initialize( + client: Client, + ) { + DeleteMonitorService.client = client + } + + /** + * Deletes the monitor, docLevelQueries and monitor metadata + * @param monitor monitor to be deleted + * @param refreshPolicy + */ + suspend fun deleteMonitor(monitor: Monitor, refreshPolicy: RefreshPolicy): DeleteMonitorResponse { + val deleteResponse = deleteMonitor(monitor.id, refreshPolicy) + deleteDocLevelMonitorQueriesAndIndices(monitor) + deleteMetadata(monitor) + return DeleteMonitorResponse(deleteResponse.id, deleteResponse.version) + } + + private suspend fun deleteMonitor(monitorId: String, refreshPolicy: RefreshPolicy): DeleteResponse { + val deleteMonitorRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, monitorId) + .setRefreshPolicy(refreshPolicy) + return client.suspendUntil { delete(deleteMonitorRequest, it) } + } + + private suspend fun deleteMetadata(monitor: Monitor) { + val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, "${monitor.id}-metadata") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + try { + val deleteResponse: DeleteResponse = client.suspendUntil { delete(deleteRequest, it) } + log.debug("Monitor metadata: ${deleteResponse.id} deletion result: ${deleteResponse.result}") + } catch (e: Exception) { + // we only log the error and don't fail the request because if monitor document has been deleted, + // we cannot retry based on this failure + log.error("Failed to delete monitor metadata ${deleteRequest.id()}.", e) + } + } + + private suspend fun deleteDocLevelMonitorQueriesAndIndices(monitor: Monitor) { + try { + val metadata = MonitorMetadataService.getMetadata(monitor) + metadata?.sourceToQueryIndexMapping?.forEach { (_, queryIndex) -> + + val indicesExistsResponse: IndicesExistsResponse = + client.suspendUntil { + client.admin().indices().exists(IndicesExistsRequest(queryIndex), it) + } + if (indicesExistsResponse.isExists == false) { + return + } + // Check if there's any queries from other monitors in this queryIndex, + // to avoid unnecessary doc deletion, if we could just delete index completely + val searchResponse: SearchResponse = client.suspendUntil { + search( + SearchRequest(queryIndex).source( + SearchSourceBuilder() + .size(0) + .query( + QueryBuilders.boolQuery().mustNot( + QueryBuilders.matchQuery("monitor_id", monitor.id) + ) + ) + ).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN), + it + ) + } + if (searchResponse.hits.totalHits.value == 0L) { + val ack: AcknowledgedResponse = client.suspendUntil { + client.admin().indices().delete( + DeleteIndexRequest(queryIndex).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN), + it + ) + } + if (ack.isAcknowledged == false) { + log.error("Deletion of concrete queryIndex:$queryIndex is not ack'd!") + } + } else { + // Delete all queries added by this monitor + val response: BulkByScrollResponse = suspendCoroutine { cont -> + DeleteByQueryRequestBuilder(client, DeleteByQueryAction.INSTANCE) + .source(queryIndex) + .filter(QueryBuilders.matchQuery("monitor_id", monitor.id)) + .refresh(true) + .execute( + object : ActionListener { + override fun onResponse(response: BulkByScrollResponse) = cont.resume(response) + override fun onFailure(t: Exception) = cont.resumeWithException(t) + } + ) + } + } + } + } catch (e: Exception) { + // we only log the error and don't fail the request because if monitor document has been deleted successfully, + // we cannot retry based on this failure + log.error("Failed to delete doc level queries from query index.", e) + } + } + + /** + * Checks if the monitor is part of the workflow + * + * @param monitorId id of monitor that is checked if it is a workflow delegate + */ + suspend fun monitorIsWorkflowDelegate(monitorId: String): Boolean { + val queryBuilder = QueryBuilders.nestedQuery( + WORKFLOW_DELEGATE_PATH, + QueryBuilders.boolQuery().must( + QueryBuilders.matchQuery( + WORKFLOW_MONITOR_PATH, + monitorId + ) + ), + ScoreMode.None + ) + try { + val searchRequest = SearchRequest() + .indices(ScheduledJob.SCHEDULED_JOBS_INDEX) + .source(SearchSourceBuilder().query(queryBuilder)) + + client.threadPool().threadContext.stashContext().use { + val searchResponse: SearchResponse = client.suspendUntil { search(searchRequest, it) } + if (searchResponse.hits.totalHits?.value == 0L) { + return false + } + + val workflowIds = searchResponse.hits.hits.map { it.id }.joinToString() + log.info("Monitor $monitorId can't be deleted since it belongs to $workflowIds") + return true + } + } catch (ex: Exception) { + log.error("Error getting the monitor workflows", ex) + throw AlertingException.wrap(ex) + } + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt index 885986ee3..cc0b7bb51 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt @@ -12,22 +12,13 @@ import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchStatusException import org.opensearch.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.admin.indices.delete.DeleteIndexRequest -import org.opensearch.action.admin.indices.exists.indices.IndicesExistsRequest -import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse -import org.opensearch.action.delete.DeleteRequest -import org.opensearch.action.delete.DeleteResponse import org.opensearch.action.get.GetRequest import org.opensearch.action.get.GetResponse -import org.opensearch.action.search.SearchRequest -import org.opensearch.action.search.SearchResponse import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction -import org.opensearch.action.support.IndicesOptions -import org.opensearch.action.support.WriteRequest -import org.opensearch.action.support.master.AcknowledgedResponse -import org.opensearch.alerting.MonitorMetadataService +import org.opensearch.action.support.WriteRequest.RefreshPolicy import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.service.DeleteMonitorService import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.alerting.util.AlertingException import org.opensearch.client.Client @@ -45,17 +36,9 @@ import org.opensearch.commons.alerting.model.ScheduledJob import org.opensearch.commons.authuser.User import org.opensearch.commons.utils.recreateObject import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.index.query.QueryBuilders -import org.opensearch.index.reindex.BulkByScrollResponse -import org.opensearch.index.reindex.DeleteByQueryAction -import org.opensearch.index.reindex.DeleteByQueryRequestBuilder import org.opensearch.rest.RestStatus -import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.tasks.Task import org.opensearch.transport.TransportService -import kotlin.coroutines.resume -import kotlin.coroutines.resumeWithException -import kotlin.coroutines.suspendCoroutine private val scope: CoroutineScope = CoroutineScope(Dispatchers.IO) private val log = LogManager.getLogger(TransportDeleteMonitorAction::class.java) @@ -85,42 +68,52 @@ class TransportDeleteMonitorAction @Inject constructor( val transformedRequest = request as? DeleteMonitorRequest ?: recreateObject(request) { DeleteMonitorRequest(it) } val user = readUserFromThreadContext(client) - val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, transformedRequest.monitorId) - .setRefreshPolicy(transformedRequest.refreshPolicy) if (!validateUserBackendRoles(user, actionListener)) { return } scope.launch { - DeleteMonitorHandler(client, actionListener, deleteRequest, user, transformedRequest.monitorId).resolveUserAndStart() + DeleteMonitorHandler( + client, + actionListener, + user, + transformedRequest.monitorId + ).resolveUserAndStart(transformedRequest.refreshPolicy) } } inner class DeleteMonitorHandler( private val client: Client, private val actionListener: ActionListener, - private val deleteRequest: DeleteRequest, private val user: User?, private val monitorId: String ) { - suspend fun resolveUserAndStart() { + suspend fun resolveUserAndStart(refreshPolicy: RefreshPolicy) { try { val monitor = getMonitor() - val canDelete = user == null || - !doFilterForUser(user) || + val canDelete = user == null || !doFilterForUser(user) || checkUserPermissionsWithResource(user, monitor.user, actionListener, "monitor", monitorId) - if (canDelete) { - val deleteResponse = deleteAllResourcesForMonitor(client, monitor, deleteRequest, monitorId) - actionListener.onResponse(DeleteMonitorResponse(deleteResponse.id, deleteResponse.version)) + if (DeleteMonitorService.monitorIsWorkflowDelegate(monitor.id)) { + actionListener.onFailure( + AlertingException( + "Monitor can't be deleted because it is a part of workflow(s)", + RestStatus.FORBIDDEN, + IllegalStateException() + ) + ) + } else if (canDelete) { + actionListener.onResponse( + DeleteMonitorService.deleteMonitor(monitor, refreshPolicy) + ) } else { actionListener.onFailure( AlertingException("Not allowed to delete this monitor!", RestStatus.FORBIDDEN, IllegalStateException()) ) } } catch (t: Exception) { - log.error("Failed to delete monitor ${deleteRequest.id()}", t) + log.error("Failed to delete monitor $monitorId", t) actionListener.onFailure(AlertingException.wrap(t)) } } @@ -145,102 +138,4 @@ class TransportDeleteMonitorAction @Inject constructor( return ScheduledJob.parse(xcp, getResponse.id, getResponse.version) as Monitor } } - - companion object { - @JvmStatic - suspend fun deleteAllResourcesForMonitor( - client: Client, - monitor: Monitor, - deleteRequest: DeleteRequest, - monitorId: String, - ): DeleteResponse { - val deleteResponse = deleteMonitorDocument(client, deleteRequest) - deleteDocLevelMonitorQueriesAndIndices(client, monitor, monitorId) - deleteMetadata(client, monitor) - return deleteResponse - } - - private suspend fun deleteMonitorDocument(client: Client, deleteRequest: DeleteRequest): DeleteResponse { - return client.suspendUntil { delete(deleteRequest, it) } - } - - suspend fun deleteMetadata(client: Client, monitor: Monitor) { - val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, "${monitor.id}-metadata") - .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) - try { - val deleteResponse: DeleteResponse = client.suspendUntil { delete(deleteRequest, it) } - log.debug("Monitor metadata: ${deleteResponse.id} deletion result: ${deleteResponse.result}") - } catch (e: Exception) { - // we only log the error and don't fail the request because if monitor document has been deleted, - // we cannot retry based on this failure - log.error("Failed to delete monitor metadata ${deleteRequest.id()}.", e) - } - } - - suspend fun deleteDocLevelMonitorQueriesAndIndices( - client: Client, - monitor: Monitor, - monitorId: String, - ) { - try { - val metadata = MonitorMetadataService.getMetadata(monitor) - metadata?.sourceToQueryIndexMapping?.forEach { (_, queryIndex) -> - - val indicesExistsResponse: IndicesExistsResponse = - client.suspendUntil { - client.admin().indices().exists(IndicesExistsRequest(queryIndex), it) - } - if (indicesExistsResponse.isExists == false) { - return - } - // Check if there's any queries from other monitors in this queryIndex, - // to avoid unnecessary doc deletion, if we could just delete index completely - val searchResponse: SearchResponse = client.suspendUntil { - search( - SearchRequest(queryIndex).source( - SearchSourceBuilder() - .size(0) - .query( - QueryBuilders.boolQuery().mustNot( - QueryBuilders.matchQuery("monitor_id", monitorId) - ) - ) - ).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN), - it - ) - } - if (searchResponse.hits.totalHits.value == 0L) { - val ack: AcknowledgedResponse = client.suspendUntil { - client.admin().indices().delete( - DeleteIndexRequest(queryIndex).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN), it - ) - } - if (ack.isAcknowledged == false) { - log.error("Deletion of concrete queryIndex:$queryIndex is not ack'd!") - } - } else { - // Delete all queries added by this monitor - val response: BulkByScrollResponse = suspendCoroutine { cont -> - DeleteByQueryRequestBuilder(client, DeleteByQueryAction.INSTANCE) - .source(queryIndex) - .filter(QueryBuilders.matchQuery("monitor_id", monitorId)) - .refresh(true) - .execute( - object : ActionListener { - override fun onResponse(response: BulkByScrollResponse) = cont.resume(response) - override fun onFailure(t: Exception) { - cont.resumeWithException(t) - } - } - ) - } - } - } - } catch (e: Exception) { - // we only log the error and don't fail the request because if monitor document has been deleted successfully, - // we cannot retry based on this failure - log.error("Failed to delete doc level queries from query index.", e) - } - } - } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteWorkflowAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteWorkflowAction.kt new file mode 100644 index 000000000..79fd817ca --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteWorkflowAction.kt @@ -0,0 +1,330 @@ + +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.transport + +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.launch +import org.apache.logging.log4j.LogManager +import org.apache.lucene.search.join.ScoreMode +import org.opensearch.OpenSearchException +import org.opensearch.OpenSearchStatusException +import org.opensearch.action.ActionListener +import org.opensearch.action.ActionRequest +import org.opensearch.action.delete.DeleteRequest +import org.opensearch.action.delete.DeleteResponse +import org.opensearch.action.get.GetRequest +import org.opensearch.action.get.GetResponse +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +import org.opensearch.action.support.ActionFilters +import org.opensearch.action.support.HandledTransportAction +import org.opensearch.action.support.WriteRequest.RefreshPolicy +import org.opensearch.alerting.model.MonitorMetadata +import org.opensearch.alerting.model.WorkflowMetadata +import org.opensearch.alerting.opensearchapi.addFilter +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.service.DeleteMonitorService +import org.opensearch.alerting.settings.AlertingSettings +import org.opensearch.alerting.util.AlertingException +import org.opensearch.client.Client +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.inject.Inject +import org.opensearch.common.settings.Settings +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.action.AlertingActions +import org.opensearch.commons.alerting.action.DeleteWorkflowRequest +import org.opensearch.commons.alerting.action.DeleteWorkflowResponse +import org.opensearch.commons.alerting.model.CompositeInput +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.authuser.User +import org.opensearch.commons.utils.recreateObject +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.XContentParser +import org.opensearch.index.IndexNotFoundException +import org.opensearch.index.query.QueryBuilders +import org.opensearch.index.reindex.BulkByScrollResponse +import org.opensearch.index.reindex.DeleteByQueryAction +import org.opensearch.index.reindex.DeleteByQueryRequestBuilder +import org.opensearch.rest.RestStatus +import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.tasks.Task +import org.opensearch.transport.TransportService + +private val scope: CoroutineScope = CoroutineScope(Dispatchers.IO) +/** + * Transport class that deletes the workflow. + * If the deleteDelegateMonitor flag is set to true, deletes the workflow delegates that are not part of another workflow + */ +class TransportDeleteWorkflowAction @Inject constructor( + transportService: TransportService, + val client: Client, + actionFilters: ActionFilters, + val clusterService: ClusterService, + val settings: Settings, + val xContentRegistry: NamedXContentRegistry, +) : HandledTransportAction( + AlertingActions.DELETE_WORKFLOW_ACTION_NAME, transportService, actionFilters, ::DeleteWorkflowRequest +), + SecureTransportAction { + private val log = LogManager.getLogger(javaClass) + + @Volatile override var filterByEnabled = AlertingSettings.FILTER_BY_BACKEND_ROLES.get(settings) + + init { + listenFilterBySettingChange(clusterService) + } + + override fun doExecute(task: Task, request: ActionRequest, actionListener: ActionListener) { + val transformedRequest = request as? DeleteWorkflowRequest + ?: recreateObject(request) { DeleteWorkflowRequest(it) } + + val user = readUserFromThreadContext(client) + val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, transformedRequest.workflowId) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + + if (!validateUserBackendRoles(user, actionListener)) { + return + } + + scope.launch { + DeleteWorkflowHandler( + client, + actionListener, + deleteRequest, + transformedRequest.deleteDelegateMonitors, + user, + transformedRequest.workflowId + ).resolveUserAndStart() + } + } + + inner class DeleteWorkflowHandler( + private val client: Client, + private val actionListener: ActionListener, + private val deleteRequest: DeleteRequest, + private val deleteDelegateMonitors: Boolean?, + private val user: User?, + private val workflowId: String, + ) { + suspend fun resolveUserAndStart() { + try { + val workflow = getWorkflow() + + val canDelete = user == null || + !doFilterForUser(user) || + checkUserPermissionsWithResource( + user, + workflow.user, + actionListener, + "workflow", + workflowId + ) + + if (canDelete) { + val delegateMonitorIds = (workflow.inputs[0] as CompositeInput).getMonitorIds() + var deletableMonitors = listOf() + // User can only delete the delegate monitors only in the case if all monitors can be deleted + // if there are monitors in this workflow that are referenced in other workflows, we cannot delete the monitors. + // We will not partially delete monitors. we delete them all or fail the request. + if (deleteDelegateMonitors == true) { + deletableMonitors = getDeletableDelegates(workflowId, delegateMonitorIds, user) + val monitorsDiff = delegateMonitorIds.toMutableList() + monitorsDiff.removeAll(deletableMonitors.map { it.id }) + + if (monitorsDiff.isNotEmpty()) { + actionListener.onFailure( + AlertingException( + "Not allowed to delete ${monitorsDiff.joinToString()} monitors", + RestStatus.FORBIDDEN, + IllegalStateException() + ) + ) + return + } + } + + val deleteResponse = deleteWorkflow(deleteRequest) + var deleteWorkflowResponse = DeleteWorkflowResponse(deleteResponse.id, deleteResponse.version) + + val workflowMetadataId = WorkflowMetadata.getId(workflow.id) + + val metadataIdsToDelete = mutableListOf(workflowMetadataId) + + if (deleteDelegateMonitors == true) { + val failedMonitorIds = tryDeletingMonitors(deletableMonitors, RefreshPolicy.IMMEDIATE) + // Update delete workflow response + deleteWorkflowResponse.nonDeletedMonitors = failedMonitorIds + // Delete monitors workflow metadata + // Monitor metadata will be in workflowId-monitorId-metadata format + metadataIdsToDelete.addAll(deletableMonitors.map { MonitorMetadata.getId(it, workflowMetadataId) }) + } + try { + // Delete the monitors workflow metadata + val deleteMonitorWorkflowMetadataResponse: BulkByScrollResponse = client.suspendUntil { + DeleteByQueryRequestBuilder(this, DeleteByQueryAction.INSTANCE) + .source(ScheduledJob.SCHEDULED_JOBS_INDEX) + .filter(QueryBuilders.idsQuery().addIds(*metadataIdsToDelete.toTypedArray())) + .execute(it) + } + } catch (t: Exception) { + log.error("Failed to delete delegate monitor metadata. But proceeding with workflow deletion $workflowId", t) + } + actionListener.onResponse(deleteWorkflowResponse) + } else { + actionListener.onFailure( + AlertingException( + "Not allowed to delete this workflow!", + RestStatus.FORBIDDEN, + IllegalStateException() + ) + ) + } + } catch (t: Exception) { + if (t is IndexNotFoundException) { + actionListener.onFailure( + OpenSearchStatusException( + "Workflow not found.", + RestStatus.NOT_FOUND + ) + ) + } else { + log.error("Failed to delete workflow $workflowId", t) + actionListener.onFailure(AlertingException.wrap(t)) + } + } + } + + /** + * Tries to delete the given list of the monitors. Return value contains all the monitorIds for which deletion failed + * @param monitorIds list of monitor ids to be deleted + * @param refreshPolicy + * @return list of the monitors that were not deleted + */ + private suspend fun tryDeletingMonitors(monitors: List, refreshPolicy: RefreshPolicy): List { + val nonDeletedMonitorIds = mutableListOf() + for (monitor in monitors) { + try { + DeleteMonitorService.deleteMonitor(monitor, refreshPolicy) + } catch (ex: Exception) { + log.error("failed to delete delegate monitor ${monitor.id} for $workflowId") + nonDeletedMonitorIds.add(monitor.id) + } + } + return nonDeletedMonitorIds + } + + /** + * Returns lit of monitor ids belonging only to a given workflow. + * if filterBy is enabled, it filters and returns only those monitors which user has permission to delete. + * @param workflowIdToBeDeleted Id of the workflow that should be deleted + * @param monitorIds List of delegate monitor ids (underlying monitor ids) + */ + private suspend fun getDeletableDelegates(workflowIdToBeDeleted: String, monitorIds: List, user: User?): List { + // Retrieve monitors belonging to another workflows + val queryBuilder = QueryBuilders.boolQuery().mustNot(QueryBuilders.termQuery("_id", workflowIdToBeDeleted)).filter( + QueryBuilders.nestedQuery( + WORKFLOW_DELEGATE_PATH, + QueryBuilders.boolQuery().must( + QueryBuilders.termsQuery( + WORKFLOW_MONITOR_PATH, + monitorIds + ) + ), + ScoreMode.None + ) + ) + + val searchRequest = SearchRequest() + .indices(ScheduledJob.SCHEDULED_JOBS_INDEX) + .source(SearchSourceBuilder().query(queryBuilder)) + + val searchResponse: SearchResponse = client.suspendUntil { search(searchRequest, it) } + + val workflows = searchResponse.hits.hits.map { hit -> + val xcp = XContentHelper.createParser( + xContentRegistry, LoggingDeprecationHandler.INSTANCE, + hit.sourceRef, XContentType.JSON + ).also { it.nextToken() } + lateinit var workflow: Workflow + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + xcp.nextToken() + when (xcp.currentName()) { + "workflow" -> workflow = Workflow.parse(xcp) + } + } + workflow.copy(id = hit.id, version = hit.version) + } + val workflowMonitors = workflows.flatMap { (it.inputs[0] as CompositeInput).getMonitorIds() }.distinct() + // Monitors that can be deleted -> all workflow delegates - monitors belonging to different workflows + val deletableMonitorIds = monitorIds.minus(workflowMonitors.toSet()) + + // filtering further to get the list of monitors that user has permission to delete if filterby is enabled and user is not null + val query = QueryBuilders.boolQuery().filter(QueryBuilders.termsQuery("_id", deletableMonitorIds)) + val searchSource = SearchSourceBuilder().query(query) + val monitorSearchRequest = SearchRequest(ScheduledJob.SCHEDULED_JOBS_INDEX).source(searchSource) + + if (user != null && filterByEnabled) { + addFilter(user, monitorSearchRequest.source(), "monitor.user.backend_roles.keyword") + } + + val searchMonitorResponse: SearchResponse = client.suspendUntil { search(monitorSearchRequest, it) } + if (searchMonitorResponse.isTimedOut) { + throw OpenSearchException("Cannot determine that the ${ScheduledJob.SCHEDULED_JOBS_INDEX} index is healthy") + } + val deletableMonitors = mutableListOf() + for (hit in searchMonitorResponse.hits) { + XContentType.JSON.xContent().createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, hit.sourceAsString + ).use { hitsParser -> + val monitor = ScheduledJob.parse(hitsParser, hit.id, hit.version) as Monitor + deletableMonitors.add(monitor) + } + } + + return deletableMonitors + } + + private suspend fun getWorkflow(): Workflow { + val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, workflowId) + + val getResponse: GetResponse = client.suspendUntil { get(getRequest, it) } + if (getResponse.isExists == false) { + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException("Workflow not found.", RestStatus.NOT_FOUND) + ) + ) + } + val xcp = XContentHelper.createParser( + xContentRegistry, LoggingDeprecationHandler.INSTANCE, + getResponse.sourceAsBytesRef, XContentType.JSON + ) + return ScheduledJob.parse(xcp, getResponse.id, getResponse.version) as Workflow + } + + private suspend fun deleteWorkflow(deleteRequest: DeleteRequest): DeleteResponse { + log.debug("Deleting the workflow with id ${deleteRequest.id()}") + return client.suspendUntil { delete(deleteRequest, it) } + } + + private suspend fun deleteWorkflowMetadata(workflow: Workflow) { + val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, WorkflowMetadata.getId(workflow.id)) + val deleteResponse: DeleteResponse = client.suspendUntil { delete(deleteRequest, it) } + } + } + + companion object { + const val WORKFLOW_DELEGATE_PATH = "workflow.inputs.composite_input.sequence.delegates" + const val WORKFLOW_MONITOR_PATH = "workflow.inputs.composite_input.sequence.delegates.monitor_id" + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteWorkflowAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteWorkflowAction.kt new file mode 100644 index 000000000..45c2a5cd1 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteWorkflowAction.kt @@ -0,0 +1,128 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.transport + +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.launch +import kotlinx.coroutines.withContext +import org.apache.logging.log4j.LogManager +import org.opensearch.OpenSearchStatusException +import org.opensearch.action.ActionListener +import org.opensearch.action.get.GetRequest +import org.opensearch.action.get.GetResponse +import org.opensearch.action.support.ActionFilters +import org.opensearch.action.support.HandledTransportAction +import org.opensearch.alerting.MonitorRunnerService +import org.opensearch.alerting.action.ExecuteWorkflowAction +import org.opensearch.alerting.action.ExecuteWorkflowRequest +import org.opensearch.alerting.action.ExecuteWorkflowResponse +import org.opensearch.alerting.util.AlertingException +import org.opensearch.alerting.workflow.WorkflowRunnerService +import org.opensearch.client.Client +import org.opensearch.common.inject.Inject +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.ConfigConstants +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.authuser.User +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.rest.RestStatus +import org.opensearch.tasks.Task +import org.opensearch.transport.TransportService +import java.time.Instant + +private val log = LogManager.getLogger(TransportExecuteWorkflowAction::class.java) + +class TransportExecuteWorkflowAction @Inject constructor( + transportService: TransportService, + private val client: Client, + private val runner: MonitorRunnerService, + actionFilters: ActionFilters, + val xContentRegistry: NamedXContentRegistry +) : HandledTransportAction( + ExecuteWorkflowAction.NAME, transportService, actionFilters, ::ExecuteWorkflowRequest +) { + override fun doExecute( + task: Task, + execWorkflowRequest: ExecuteWorkflowRequest, + actionListener: ActionListener, + ) { + val userStr = client.threadPool().threadContext.getTransient(ConfigConstants.OPENSEARCH_SECURITY_USER_INFO_THREAD_CONTEXT) + log.debug("User and roles string from thread context: $userStr") + val user: User? = User.parse(userStr) + + client.threadPool().threadContext.stashContext().use { + val executeWorkflow = fun(workflow: Workflow) { + runner.launch { + val (periodStart, periodEnd) = + workflow.schedule.getPeriodEndingAt(Instant.ofEpochMilli(execWorkflowRequest.requestEnd.millis)) + try { + val workflowRunResult = + WorkflowRunnerService.runJob(workflow, periodStart, periodEnd, execWorkflowRequest.dryrun) + withContext(Dispatchers.IO) { + actionListener.onResponse( + ExecuteWorkflowResponse( + workflowRunResult + ) + ) + } + } catch (e: Exception) { + log.error("Unexpected error running workflow", e) + withContext(Dispatchers.IO) { + actionListener.onFailure(AlertingException.wrap(e)) + } + } + } + } + + if (execWorkflowRequest.workflowId != null) { + val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX).id(execWorkflowRequest.workflowId) + client.get( + getRequest, + object : ActionListener { + override fun onResponse(response: GetResponse) { + if (!response.isExists) { + log.error("Can't find workflow with id: ${response.id}") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Can't find workflow with id: ${response.id}", + RestStatus.NOT_FOUND + ) + ) + ) + return + } + if (!response.isSourceEmpty) { + XContentHelper.createParser( + xContentRegistry, LoggingDeprecationHandler.INSTANCE, + response.sourceAsBytesRef, XContentType.JSON + ).use { xcp -> + val workflow = ScheduledJob.parse(xcp, response.id, response.version) as Workflow + executeWorkflow(workflow) + } + } + } + + override fun onFailure(t: Exception) { + log.error("Error getting workflow ${execWorkflowRequest.workflowId}", t) + actionListener.onFailure(AlertingException.wrap(t)) + } + } + ) + } else { + val workflow = when (user?.name.isNullOrEmpty()) { + true -> execWorkflowRequest.workflow as Workflow + false -> (execWorkflowRequest.workflow as Workflow).copy(user = user) + } + + executeWorkflow(workflow) + } + } + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportGetWorkflowAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportGetWorkflowAction.kt new file mode 100644 index 000000000..68df71e4a --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportGetWorkflowAction.kt @@ -0,0 +1,148 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.transport + +import org.apache.logging.log4j.LogManager +import org.opensearch.OpenSearchStatusException +import org.opensearch.action.ActionListener +import org.opensearch.action.get.GetRequest +import org.opensearch.action.get.GetResponse +import org.opensearch.action.support.ActionFilters +import org.opensearch.action.support.HandledTransportAction +import org.opensearch.alerting.settings.AlertingSettings +import org.opensearch.alerting.util.AlertingException +import org.opensearch.client.Client +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.inject.Inject +import org.opensearch.common.settings.Settings +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.action.AlertingActions +import org.opensearch.commons.alerting.action.GetWorkflowRequest +import org.opensearch.commons.alerting.action.GetWorkflowResponse +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.index.IndexNotFoundException +import org.opensearch.rest.RestStatus +import org.opensearch.tasks.Task +import org.opensearch.transport.TransportService + +class TransportGetWorkflowAction @Inject constructor( + transportService: TransportService, + val client: Client, + actionFilters: ActionFilters, + val xContentRegistry: NamedXContentRegistry, + val clusterService: ClusterService, + settings: Settings +) : HandledTransportAction( + AlertingActions.GET_WORKFLOW_ACTION_NAME, transportService, actionFilters, ::GetWorkflowRequest +), + SecureTransportAction { + + private val log = LogManager.getLogger(javaClass) + + @Volatile override var filterByEnabled = AlertingSettings.FILTER_BY_BACKEND_ROLES.get(settings) + + init { + listenFilterBySettingChange(clusterService) + } + + override fun doExecute(task: Task, getWorkflowRequest: GetWorkflowRequest, actionListener: ActionListener) { + val user = readUserFromThreadContext(client) + + val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, getWorkflowRequest.workflowId) + + if (!validateUserBackendRoles(user, actionListener)) { + return + } + + client.threadPool().threadContext.stashContext().use { + client.get( + getRequest, + object : ActionListener { + override fun onResponse(response: GetResponse) { + if (!response.isExists) { + log.error("Workflow with ${getWorkflowRequest.workflowId} not found") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Workflow not found.", + RestStatus.NOT_FOUND + ) + ) + ) + return + } + + var workflow: Workflow? = null + if (!response.isSourceEmpty) { + XContentHelper.createParser( + xContentRegistry, LoggingDeprecationHandler.INSTANCE, + response.sourceAsBytesRef, XContentType.JSON + ).use { xcp -> + val compositeMonitor = ScheduledJob.parse(xcp, response.id, response.version) + if (compositeMonitor is Workflow) { + workflow = compositeMonitor + } else { + log.error("Wrong monitor type returned") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Workflow not found.", + RestStatus.NOT_FOUND + ) + ) + ) + return + } + + // security is enabled and filterby is enabled + if (!checkUserPermissionsWithResource( + user, + workflow?.user, + actionListener, + "workflow", + getWorkflowRequest.workflowId + ) + ) { + return + } + } + } + + actionListener.onResponse( + GetWorkflowResponse( + response.id, + response.version, + response.seqNo, + response.primaryTerm, + RestStatus.OK, + workflow + ) + ) + } + + override fun onFailure(t: Exception) { + log.error("Getting the workflow failed", t) + + if (t is IndexNotFoundException) { + actionListener.onFailure( + OpenSearchStatusException( + "Workflow not found", + RestStatus.NOT_FOUND + ) + ) + } else { + actionListener.onFailure(AlertingException.wrap(t)) + } + } + } + ) + } + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt index dfebd5961..38d1aed6b 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt @@ -20,7 +20,6 @@ import org.opensearch.action.admin.cluster.health.ClusterHealthAction import org.opensearch.action.admin.cluster.health.ClusterHealthRequest import org.opensearch.action.admin.cluster.health.ClusterHealthResponse import org.opensearch.action.admin.indices.create.CreateIndexResponse -import org.opensearch.action.delete.DeleteRequest import org.opensearch.action.get.GetRequest import org.opensearch.action.get.GetResponse import org.opensearch.action.index.IndexRequest @@ -35,6 +34,7 @@ import org.opensearch.alerting.MonitorMetadataService import org.opensearch.alerting.core.ScheduledJobIndices import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.service.DeleteMonitorService import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.alerting.settings.AlertingSettings.Companion.ALERTING_MAX_MONITORS import org.opensearch.alerting.settings.AlertingSettings.Companion.INDEX_TIMEOUT @@ -546,11 +546,9 @@ class TransportIndexMonitorAction @Inject constructor( private suspend fun cleanupMonitorAfterPartialFailure(monitor: Monitor, indexMonitorResponse: IndexResponse) { // we simply log the success (debug log) or failure (error log) when we try clean up partially failed monitor creation request try { - TransportDeleteMonitorAction.deleteAllResourcesForMonitor( - client, + DeleteMonitorService.deleteMonitor( monitor = monitor, - DeleteRequest(SCHEDULED_JOBS_INDEX, indexMonitorResponse.id).setRefreshPolicy(RefreshPolicy.IMMEDIATE), - indexMonitorResponse.id + RefreshPolicy.IMMEDIATE ) log.debug( "Cleaned up monitor related resources after monitor creation request partial failure. " + diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexWorkflowAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexWorkflowAction.kt new file mode 100644 index 000000000..60f39a063 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexWorkflowAction.kt @@ -0,0 +1,722 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.transport + +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.launch +import org.apache.logging.log4j.LogManager +import org.opensearch.ExceptionsHelper +import org.opensearch.OpenSearchException +import org.opensearch.OpenSearchStatusException +import org.opensearch.ResourceAlreadyExistsException +import org.opensearch.action.ActionListener +import org.opensearch.action.ActionRequest +import org.opensearch.action.admin.cluster.health.ClusterHealthAction +import org.opensearch.action.admin.cluster.health.ClusterHealthRequest +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse +import org.opensearch.action.admin.indices.create.CreateIndexResponse +import org.opensearch.action.get.GetRequest +import org.opensearch.action.get.GetResponse +import org.opensearch.action.index.IndexRequest +import org.opensearch.action.index.IndexResponse +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +import org.opensearch.action.support.ActionFilters +import org.opensearch.action.support.HandledTransportAction +import org.opensearch.action.support.master.AcknowledgedResponse +import org.opensearch.alerting.core.ScheduledJobIndices +import org.opensearch.alerting.opensearchapi.InjectorContextElement +import org.opensearch.alerting.opensearchapi.addFilter +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.opensearchapi.withClosableContext +import org.opensearch.alerting.settings.AlertingSettings +import org.opensearch.alerting.settings.AlertingSettings.Companion.ALERTING_MAX_MONITORS +import org.opensearch.alerting.settings.AlertingSettings.Companion.INDEX_TIMEOUT +import org.opensearch.alerting.settings.AlertingSettings.Companion.MAX_ACTION_THROTTLE_VALUE +import org.opensearch.alerting.settings.AlertingSettings.Companion.REQUEST_TIMEOUT +import org.opensearch.alerting.settings.DestinationSettings.Companion.ALLOW_LIST +import org.opensearch.alerting.util.AlertingException +import org.opensearch.alerting.util.IndexUtils +import org.opensearch.alerting.util.isADMonitor +import org.opensearch.alerting.util.isQueryLevelMonitor +import org.opensearch.client.Client +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.inject.Inject +import org.opensearch.common.io.stream.NamedWriteableRegistry +import org.opensearch.common.settings.Settings +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentFactory.jsonBuilder +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.action.AlertingActions +import org.opensearch.commons.alerting.action.IndexWorkflowRequest +import org.opensearch.commons.alerting.action.IndexWorkflowResponse +import org.opensearch.commons.alerting.model.CompositeInput +import org.opensearch.commons.alerting.model.Delegate +import org.opensearch.commons.alerting.model.DocLevelMonitorInput +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.ScheduledJob.Companion.SCHEDULED_JOBS_INDEX +import org.opensearch.commons.alerting.model.SearchInput +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.authuser.User +import org.opensearch.commons.utils.recreateObject +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.index.IndexNotFoundException +import org.opensearch.index.query.QueryBuilders +import org.opensearch.rest.RestRequest +import org.opensearch.rest.RestStatus +import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.tasks.Task +import org.opensearch.transport.TransportService +import java.util.UUID +import java.util.stream.Collectors + +private val log = LogManager.getLogger(TransportIndexWorkflowAction::class.java) +private val scope: CoroutineScope = CoroutineScope(Dispatchers.IO) + +class TransportIndexWorkflowAction @Inject constructor( + transportService: TransportService, + val client: Client, + actionFilters: ActionFilters, + val scheduledJobIndices: ScheduledJobIndices, + val clusterService: ClusterService, + val settings: Settings, + val xContentRegistry: NamedXContentRegistry, + val namedWriteableRegistry: NamedWriteableRegistry, +) : HandledTransportAction( + AlertingActions.INDEX_WORKFLOW_ACTION_NAME, transportService, actionFilters, ::IndexWorkflowRequest +), + SecureTransportAction { + + @Volatile + private var maxMonitors = ALERTING_MAX_MONITORS.get(settings) + + @Volatile + private var requestTimeout = REQUEST_TIMEOUT.get(settings) + + @Volatile + private var indexTimeout = INDEX_TIMEOUT.get(settings) + + @Volatile + private var maxActionThrottle = MAX_ACTION_THROTTLE_VALUE.get(settings) + + @Volatile + private var allowList = ALLOW_LIST.get(settings) + + @Volatile + override var filterByEnabled = AlertingSettings.FILTER_BY_BACKEND_ROLES.get(settings) + + init { + clusterService.clusterSettings.addSettingsUpdateConsumer(ALERTING_MAX_MONITORS) { maxMonitors = it } + clusterService.clusterSettings.addSettingsUpdateConsumer(REQUEST_TIMEOUT) { requestTimeout = it } + clusterService.clusterSettings.addSettingsUpdateConsumer(INDEX_TIMEOUT) { indexTimeout = it } + clusterService.clusterSettings.addSettingsUpdateConsumer(MAX_ACTION_THROTTLE_VALUE) { maxActionThrottle = it } + clusterService.clusterSettings.addSettingsUpdateConsumer(ALLOW_LIST) { allowList = it } + listenFilterBySettingChange(clusterService) + } + + override fun doExecute(task: Task, request: ActionRequest, actionListener: ActionListener) { + val transformedRequest = request as? IndexWorkflowRequest + ?: recreateObject(request, namedWriteableRegistry) { + IndexWorkflowRequest(it) + } + + val user = readUserFromThreadContext(client) + + if (!validateUserBackendRoles(user, actionListener)) { + return + } + + if ( + user != null && + !isAdmin(user) && + transformedRequest.rbacRoles != null + ) { + if (transformedRequest.rbacRoles?.stream()?.anyMatch { !user.backendRoles.contains(it) } == true) { + log.error( + "User specified backend roles, ${transformedRequest.rbacRoles}, " + + "that they don' have access to. User backend roles: ${user.backendRoles}" + ) + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "User specified backend roles that they don't have access to. Contact administrator", + RestStatus.FORBIDDEN + ) + ) + ) + return + } else if (transformedRequest.rbacRoles?.isEmpty() == true) { + log.error( + "Non-admin user are not allowed to specify an empty set of backend roles. " + + "Please don't pass in the parameter or pass in at least one backend role." + ) + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Non-admin user are not allowed to specify an empty set of backend roles.", + RestStatus.FORBIDDEN + ) + ) + ) + return + } + } + + scope.launch { + try { + validateMonitorAccess( + transformedRequest, + user, + client, + object : ActionListener { + override fun onResponse(response: AcknowledgedResponse) { + // Stash the context and start the workflow creation + client.threadPool().threadContext.stashContext().use { + IndexWorkflowHandler(client, actionListener, transformedRequest, user).resolveUserAndStart() + } + } + + override fun onFailure(e: Exception) { + log.error("Error indexing workflow", e) + actionListener.onFailure(e) + } + } + ) + } catch (e: Exception) { + log.error("Failed to create workflow", e) + if (e is IndexNotFoundException) { + actionListener.onFailure( + OpenSearchStatusException( + "Monitors not found", + RestStatus.NOT_FOUND + ) + ) + } else { + actionListener.onFailure(e) + } + } + } + } + + inner class IndexWorkflowHandler( + private val client: Client, + private val actionListener: ActionListener, + private val request: IndexWorkflowRequest, + private val user: User?, + ) { + fun resolveUserAndStart() { + scope.launch { + if (user == null) { + // Security is disabled, add empty user to Workflow. user is null for older versions. + request.workflow = request.workflow + .copy(user = User("", listOf(), listOf(), listOf())) + start() + } else { + request.workflow = request.workflow + .copy(user = User(user.name, user.backendRoles, user.roles, user.customAttNames)) + start() + } + } + } + + fun start() { + if (!scheduledJobIndices.scheduledJobIndexExists()) { + scheduledJobIndices.initScheduledJobIndex(object : ActionListener { + override fun onResponse(response: CreateIndexResponse) { + onCreateMappingsResponse(response.isAcknowledged) + } + + override fun onFailure(t: Exception) { + // https://github.com/opensearch-project/alerting/issues/646 + if (ExceptionsHelper.unwrapCause(t) is ResourceAlreadyExistsException) { + scope.launch { + // Wait for the yellow status + val request = ClusterHealthRequest() + .indices(SCHEDULED_JOBS_INDEX) + .waitForYellowStatus() + val response: ClusterHealthResponse = client.suspendUntil { + execute(ClusterHealthAction.INSTANCE, request, it) + } + if (response.isTimedOut) { + log.error("Workflow creation timeout", t) + actionListener.onFailure( + OpenSearchException("Cannot determine that the $SCHEDULED_JOBS_INDEX index is healthy") + ) + } + // Retry mapping of workflow + onCreateMappingsResponse(true) + } + } else { + log.error("Failed to create workflow", t) + actionListener.onFailure(AlertingException.wrap(t)) + } + } + }) + } else if (!IndexUtils.scheduledJobIndexUpdated) { + IndexUtils.updateIndexMapping( + SCHEDULED_JOBS_INDEX, + ScheduledJobIndices.scheduledJobMappings(), clusterService.state(), client.admin().indices(), + object : ActionListener { + override fun onResponse(response: AcknowledgedResponse) { + onUpdateMappingsResponse(response) + } + + override fun onFailure(t: Exception) { + log.error("Failed to create workflow", t) + actionListener.onFailure(AlertingException.wrap(t)) + } + } + ) + } else { + prepareWorkflowIndexing() + } + } + + /** + * This function prepares for indexing a new workflow. + * If this is an update request we can simply update the workflow. Otherwise we first check to see how many monitors already exist, + * and compare this to the [maxMonitorCount]. Requests that breach this threshold will be rejected. + */ + private fun prepareWorkflowIndexing() { + if (request.method == RestRequest.Method.PUT) { + scope.launch { + updateWorkflow() + } + } else { + scope.launch { + indexWorkflow() + } + } + } + + private fun onCreateMappingsResponse(isAcknowledged: Boolean) { + if (isAcknowledged) { + log.info("Created $SCHEDULED_JOBS_INDEX with mappings.") + prepareWorkflowIndexing() + IndexUtils.scheduledJobIndexUpdated() + } else { + log.error("Create $SCHEDULED_JOBS_INDEX mappings call not acknowledged.") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Create $SCHEDULED_JOBS_INDEX mappings call not acknowledged", + RestStatus.INTERNAL_SERVER_ERROR + ) + ) + ) + } + } + + private fun onUpdateMappingsResponse(response: AcknowledgedResponse) { + if (response.isAcknowledged) { + log.info("Updated $SCHEDULED_JOBS_INDEX with mappings.") + IndexUtils.scheduledJobIndexUpdated() + prepareWorkflowIndexing() + } else { + log.error("Update $SCHEDULED_JOBS_INDEX mappings call not acknowledged.") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Updated $SCHEDULED_JOBS_INDEX mappings call not acknowledged.", + RestStatus.INTERNAL_SERVER_ERROR + ) + ) + ) + } + } + + private suspend fun indexWorkflow() { + if (user != null) { + val rbacRoles = if (request.rbacRoles == null) user.backendRoles.toSet() + else if (!isAdmin(user)) request.rbacRoles?.intersect(user.backendRoles)?.toSet() + else request.rbacRoles + + request.workflow = request.workflow.copy( + user = User(user.name, rbacRoles.orEmpty().toList(), user.roles, user.customAttNames) + ) + log.debug("Created workflow's backend roles: $rbacRoles") + } + + val indexRequest = IndexRequest(SCHEDULED_JOBS_INDEX) + .setRefreshPolicy(request.refreshPolicy) + .source( + request.workflow.toXContentWithUser( + jsonBuilder(), + ToXContent.MapParams(mapOf("with_type" to "true")) + ) + ) + .setIfSeqNo(request.seqNo) + .setIfPrimaryTerm(request.primaryTerm) + .timeout(indexTimeout) + + try { + val indexResponse: IndexResponse = client.suspendUntil { client.index(indexRequest, it) } + val failureReasons = checkShardsFailure(indexResponse) + if (failureReasons != null) { + log.error("Failed to create workflow: $failureReasons") + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + failureReasons.toString(), + indexResponse.status() + ) + ) + ) + return + } + actionListener.onResponse( + IndexWorkflowResponse( + indexResponse.id, indexResponse.version, indexResponse.seqNo, + indexResponse.primaryTerm, request.workflow.copy(id = indexResponse.id) + ) + ) + } catch (t: Exception) { + log.error("Failed to index workflow", t) + actionListener.onFailure(AlertingException.wrap(t)) + } + } + + private suspend fun updateWorkflow() { + val getRequest = GetRequest(SCHEDULED_JOBS_INDEX, request.workflowId) + try { + val getResponse: GetResponse = client.suspendUntil { client.get(getRequest, it) } + if (!getResponse.isExists) { + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "Workflow with ${request.workflowId} is not found", + RestStatus.NOT_FOUND + ) + ) + ) + return + } + val xcp = XContentHelper.createParser( + xContentRegistry, LoggingDeprecationHandler.INSTANCE, + getResponse.sourceAsBytesRef, XContentType.JSON + ) + val workflow = ScheduledJob.parse(xcp, getResponse.id, getResponse.version) as Workflow + onGetResponse(workflow) + } catch (t: Exception) { + actionListener.onFailure(AlertingException.wrap(t)) + } + } + + private suspend fun onGetResponse(currentWorkflow: Workflow) { + if (!checkUserPermissionsWithResource( + user, + currentWorkflow.user, + actionListener, + "workfklow", + request.workflowId + ) + ) { + return + } + + // If both are enabled, use the current existing monitor enabled time, otherwise the next execution will be + // incorrect. + if (request.workflow.enabled && currentWorkflow.enabled) + request.workflow = request.workflow.copy(enabledTime = currentWorkflow.enabledTime) + + /** + * On update workflow check which backend roles to associate to the workflow. + * Below are 2 examples of how the logic works + * + * Example 1, say we have a Workflow with backend roles [a, b, c, d] associated with it. + * If I'm User A (non-admin user) and I have backend roles [a, b, c] associated with me and I make a request to update + * the Workflow's backend roles to [a, b]. This would mean that the roles to remove are [c] and the roles to add are [a, b]. + * The Workflow's backend roles would then be [a, b, d]. + * + * Example 2, say we have a Workflow with backend roles [a, b, c, d] associated with it. + * If I'm User A (admin user) and I have backend roles [a, b, c] associated with me and I make a request to update + * the Workflow's backend roles to [a, b]. This would mean that the roles to remove are [c, d] and the roles to add are [a, b]. + * The Workflow's backend roles would then be [a, b]. + */ + if (user != null) { + if (request.rbacRoles != null) { + if (isAdmin(user)) { + request.workflow = request.workflow.copy( + user = User(user.name, request.rbacRoles, user.roles, user.customAttNames) + ) + } else { + // rolesToRemove: these are the backend roles to remove from the monitor + val rolesToRemove = user.backendRoles - request.rbacRoles.orEmpty() + // remove the monitor's roles with rolesToRemove and add any roles passed into the request.rbacRoles + val updatedRbac = + currentWorkflow.user?.backendRoles.orEmpty() - rolesToRemove + request.rbacRoles.orEmpty() + request.workflow = request.workflow.copy( + user = User(user.name, updatedRbac, user.roles, user.customAttNames) + ) + } + } else { + request.workflow = request.workflow + .copy( + user = User( + user.name, + currentWorkflow.user!!.backendRoles, + user.roles, + user.customAttNames + ) + ) + } + log.debug("Update workflow backend roles to: ${request.workflow.user?.backendRoles}") + } + + request.workflow = request.workflow.copy(schemaVersion = IndexUtils.scheduledJobIndexSchemaVersion) + val indexRequest = IndexRequest(SCHEDULED_JOBS_INDEX) + .setRefreshPolicy(request.refreshPolicy) + .source( + request.workflow.toXContentWithUser( + jsonBuilder(), + ToXContent.MapParams(mapOf("with_type" to "true")) + ) + ) + .id(request.workflowId) + .setIfSeqNo(request.seqNo) + .setIfPrimaryTerm(request.primaryTerm) + .timeout(indexTimeout) + + try { + val indexResponse: IndexResponse = client.suspendUntil { client.index(indexRequest, it) } + val failureReasons = checkShardsFailure(indexResponse) + if (failureReasons != null) { + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + failureReasons.toString(), + indexResponse.status() + ) + ) + ) + return + } + actionListener.onResponse( + IndexWorkflowResponse( + indexResponse.id, indexResponse.version, indexResponse.seqNo, + indexResponse.primaryTerm, request.workflow.copy(id = currentWorkflow.id) + ) + ) + } catch (t: Exception) { + actionListener.onFailure(AlertingException.wrap(t)) + } + } + + private fun checkShardsFailure(response: IndexResponse): String? { + val failureReasons = StringBuilder() + if (response.shardInfo.failed > 0) { + response.shardInfo.failures.forEach { entry -> + failureReasons.append(entry.reason()) + } + return failureReasons.toString() + } + return null + } + } + + private fun validateChainedMonitorFindingsMonitors(delegates: List, monitorDelegates: List) { + infix fun List.equalsIgnoreOrder(other: List) = + this.size == other.size && this.toSet() == other.toSet() + + val monitorsById = monitorDelegates.associateBy { it.id } + delegates.forEach { + val delegateMonitor = monitorsById[it.monitorId] ?: throw AlertingException.wrap( + IllegalArgumentException("Delegate monitor ${it.monitorId} doesn't exist") + ) + if (it.chainedMonitorFindings != null) { + val chainedFindingMonitor = + monitorsById[it.chainedMonitorFindings!!.monitorId] ?: throw AlertingException.wrap( + IllegalArgumentException("Chained finding monitor doesn't exist") + ) + + if (chainedFindingMonitor.isQueryLevelMonitor()) { + throw AlertingException.wrap(IllegalArgumentException("Query level monitor can't be part of chained findings")) + } + + val delegateMonitorIndices = getMonitorIndices(delegateMonitor) + + val chainedMonitorIndices = getMonitorIndices(chainedFindingMonitor) + + if (!delegateMonitorIndices.equalsIgnoreOrder(chainedMonitorIndices)) { + throw AlertingException.wrap( + IllegalArgumentException("Delegate monitor and it's chained finding monitor must query the same indices") + ) + } + } + } + } + + /** + * Returns list of indices for the given monitor depending on it's type + */ + private fun getMonitorIndices(monitor: Monitor): List { + return when (monitor.monitorType) { + Monitor.MonitorType.DOC_LEVEL_MONITOR -> (monitor.inputs[0] as DocLevelMonitorInput).indices + Monitor.MonitorType.BUCKET_LEVEL_MONITOR -> monitor.inputs.flatMap { s -> (s as SearchInput).indices } + Monitor.MonitorType.QUERY_LEVEL_MONITOR -> { + if (isADMonitor(monitor)) monitor.inputs.flatMap { s -> (s as SearchInput).indices } + else { + val indices = mutableListOf() + for (input in monitor.inputs) { + when (input) { + is SearchInput -> indices.addAll(input.indices) + else -> indices + } + } + indices + } + } + + else -> emptyList() + } + } + + private fun validateDelegateMonitorsExist( + monitorIds: List, + delegateMonitors: List + ) { + val reqMonitorIds: MutableList = monitorIds as MutableList + delegateMonitors.forEach { + reqMonitorIds.remove(it.id) + } + if (reqMonitorIds.isNotEmpty()) { + throw AlertingException.wrap(IllegalArgumentException(("${reqMonitorIds.joinToString()} are not valid monitor ids"))) + } + } + + /** + * Validates monitor and indices access + * 1. Validates the monitor access (if the filterByEnabled is set to true - adds backend role filter) as admin + * 2. Unstashes the context and checks if the user can access the monitor indices + */ + private suspend fun validateMonitorAccess( + request: IndexWorkflowRequest, + user: User?, + client: Client, + actionListener: ActionListener + ) { + val compositeInput = request.workflow.inputs[0] as CompositeInput + val monitorIds = compositeInput.sequence.delegates.stream().map { it.monitorId }.collect(Collectors.toList()) + val query = QueryBuilders.boolQuery().filter(QueryBuilders.termsQuery("_id", monitorIds)) + val searchSource = SearchSourceBuilder().query(query) + val searchRequest = SearchRequest(SCHEDULED_JOBS_INDEX).source(searchSource) + + if (user != null && !isAdmin(user) && filterByEnabled) { + addFilter(user, searchRequest.source(), "monitor.user.backend_roles.keyword") + } + + val searchMonitorResponse: SearchResponse = client.suspendUntil { client.search(searchRequest, it) } + + if (searchMonitorResponse.isTimedOut) { + throw OpenSearchException("Cannot determine that the $SCHEDULED_JOBS_INDEX index is healthy") + } + val monitors = mutableListOf() + for (hit in searchMonitorResponse.hits) { + XContentType.JSON.xContent().createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, hit.sourceAsString + ).use { hitsParser -> + val monitor = ScheduledJob.parse(hitsParser, hit.id, hit.version) as Monitor + monitors.add(monitor) + } + } + if (monitors.isEmpty()) { + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "User doesn't have read permissions for one or more configured monitors ${monitorIds.joinToString()}", + RestStatus.FORBIDDEN + ) + ) + ) + return + } + // Validate delegates and it's chained findings + try { + validateDelegateMonitorsExist(monitorIds, monitors) + validateChainedMonitorFindingsMonitors(compositeInput.sequence.delegates, monitors) + } catch (e: Exception) { + actionListener.onFailure(e) + return + } + val indices = getMonitorIndices(monitors) + + val indicesSearchRequest = SearchRequest().indices(*indices.toTypedArray()) + .source(SearchSourceBuilder.searchSource().size(1).query(QueryBuilders.matchAllQuery())) + + if (user != null && filterByEnabled) { + // Unstash the context and check if user with specified roles has indices access + withClosableContext( + InjectorContextElement( + user.name.plus(UUID.randomUUID().toString()), + settings, + client.threadPool().threadContext, + user.roles, + user + ) + ) { + checkIndicesAccess(client, indicesSearchRequest, indices, actionListener) + } + } else { + checkIndicesAccess(client, indicesSearchRequest, indices, actionListener) + } + } + + /** + * Checks if the client can access the given indices + */ + private fun checkIndicesAccess( + client: Client, + indicesSearchRequest: SearchRequest?, + indices: MutableList, + actionListener: ActionListener, + ) { + client.search( + indicesSearchRequest, + object : ActionListener { + override fun onResponse(response: SearchResponse?) { + actionListener.onResponse(AcknowledgedResponse(true)) + } + + override fun onFailure(e: Exception) { + log.error("Error accessing the monitor indices", e) + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException( + "User doesn't have read permissions for one or more configured index ${indices.joinToString()}", + RestStatus.FORBIDDEN + ) + ) + ) + } + } + ) + } + + /** + * Extract indices from monitors + */ + private fun getMonitorIndices(monitors: List): MutableList { + val indices = mutableListOf() + + val searchInputs = + monitors.flatMap { monitor -> + monitor.inputs.filter { + it.name() == SearchInput.SEARCH_FIELD || it.name() == DocLevelMonitorInput.DOC_LEVEL_INPUT_FIELD + } + } + searchInputs.forEach { + val inputIndices = if (it.name() == SearchInput.SEARCH_FIELD) (it as SearchInput).indices + else (it as DocLevelMonitorInput).indices + indices.addAll(inputIndices) + } + return indices + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt b/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt index 1b8a153b3..5255de8c9 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt @@ -45,6 +45,8 @@ fun Destination.isTestAction(): Boolean = this.type == DestinationType.TEST_ACTI fun Monitor.isDocLevelMonitor(): Boolean = this.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR +fun Monitor.isQueryLevelMonitor(): Boolean = this.monitorType == Monitor.MonitorType.QUERY_LEVEL_MONITOR + /** * Since buckets can have multi-value keys, this converts the bucket key values to a string that can be used * as the key for a HashMap to easily retrieve [AggregationResultBucket] based on the bucket key values. diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt new file mode 100644 index 000000000..013ee3056 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt @@ -0,0 +1,180 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.workflow + +import org.apache.logging.log4j.LogManager +import org.opensearch.ExceptionsHelper +import org.opensearch.alerting.BucketLevelMonitorRunner +import org.opensearch.alerting.DocumentLevelMonitorRunner +import org.opensearch.alerting.MonitorRunnerExecutionContext +import org.opensearch.alerting.QueryLevelMonitorRunner +import org.opensearch.alerting.WorkflowMetadataService +import org.opensearch.alerting.model.MonitorRunResult +import org.opensearch.alerting.model.WorkflowRunResult +import org.opensearch.alerting.util.AlertingException +import org.opensearch.alerting.util.isDocLevelMonitor +import org.opensearch.alerting.util.isQueryLevelMonitor +import org.opensearch.commons.alerting.model.CompositeInput +import org.opensearch.commons.alerting.model.Delegate +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.alerting.util.isBucketLevelMonitor +import java.time.Instant +import java.time.LocalDateTime +import java.time.ZoneOffset +import java.util.UUID + +object CompositeWorkflowRunner : WorkflowRunner() { + private val logger = LogManager.getLogger(javaClass) + + override suspend fun runWorkflow( + workflow: Workflow, + monitorCtx: MonitorRunnerExecutionContext, + periodStart: Instant, + periodEnd: Instant, + dryRun: Boolean + ): WorkflowRunResult { + val workflowExecutionStartTime = Instant.now() + + val isTempWorkflow = dryRun || workflow.id == Workflow.NO_ID + + val executionId = generateExecutionId(isTempWorkflow, workflow) + + val (workflowMetadata, _) = WorkflowMetadataService.getOrCreateWorkflowMetadata( + workflow = workflow, + skipIndex = isTempWorkflow, + executionId = executionId + ) + + var workflowResult = WorkflowRunResult(mutableListOf(), workflowExecutionStartTime, null, executionId) + + logger.debug("Workflow ${workflow.id} in $executionId execution is running") + val delegates = (workflow.inputs[0] as CompositeInput).sequence.delegates.sortedBy { it.order } + var monitors: List + + try { + monitors = monitorCtx.workflowService!!.getMonitorsById(delegates.map { it.monitorId }, delegates.size) + } catch (e: Exception) { + logger.error("Failed getting workflow delegates. Error: ${e.message}", e) + return workflowResult.copy(error = AlertingException.wrap(e)) + } + // Validate the monitors size + validateMonitorSize(delegates, monitors, workflow) + + val monitorsById = monitors.associateBy { it.id } + val resultList = mutableListOf>() + var lastErrorDelegateRun: Exception? = null + + for (delegate in delegates) { + var indexToDocIds = mapOf>() + var delegateMonitor: Monitor + delegateMonitor = monitorsById[delegate.monitorId] + ?: throw AlertingException.wrap( + IllegalStateException("Delegate monitor not found ${delegate.monitorId} for the workflow $workflow.id") + ) + if (delegate.chainedMonitorFindings != null) { + val chainedMonitor = monitorsById[delegate.chainedMonitorFindings!!.monitorId] + ?: throw AlertingException.wrap( + IllegalStateException("Chained finding monitor not found ${delegate.monitorId} for the workflow $workflow.id") + ) + + try { + indexToDocIds = monitorCtx.workflowService!!.getFindingDocIdsByExecutionId(chainedMonitor, executionId) + } catch (e: Exception) { + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + // If it is not IndexNotFound exception return the result + if (unwrappedException.message?.contains("Configured indices are not found") == false) { + logger.error("Failed to execute workflow. Error: ${e.message}", e) + return workflowResult.copy(error = AlertingException.wrap(e)) + } + // Log that finding index is not found and proceed with the execution + logger.error("Finding index ${chainedMonitor.dataSources.findingsIndex} doesn't exist") + } + } + + val workflowRunContext = WorkflowRunContext( + workflowId = workflowMetadata.workflowId, + workflowMetadataId = workflowMetadata.id, + chainedMonitorId = delegate.chainedMonitorFindings?.monitorId, + executionId = executionId, + matchingDocIdsPerIndex = indexToDocIds + ) + + var delegateRunResult: MonitorRunResult<*>? + try { + delegateRunResult = if (delegateMonitor.isBucketLevelMonitor()) { + BucketLevelMonitorRunner.runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext + ) + } else if (delegateMonitor.isDocLevelMonitor()) { + DocumentLevelMonitorRunner.runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext + ) + } else if (delegateMonitor.isQueryLevelMonitor()) { + QueryLevelMonitorRunner.runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext + ) + } else { + throw AlertingException.wrap( + IllegalStateException("Unsupported monitor type") + ) + } + } catch (ex: Exception) { + logger.error("Error executing workflow delegate. Error: ${ex.message}", ex) + lastErrorDelegateRun = AlertingException.wrap(ex) + continue + } + if (delegateRunResult != null) resultList.add(delegateRunResult) + } + logger.debug("Workflow ${workflow.id} in $executionId finished") + // Update metadata only if the workflow is not temp + if (!isTempWorkflow) { + WorkflowMetadataService.upsertWorkflowMetadata( + workflowMetadata.copy(latestRunTime = workflowExecutionStartTime, latestExecutionId = executionId), + true + ) + } + + return workflowResult.copy(workflowRunResult = resultList, executionEndTime = Instant.now(), error = lastErrorDelegateRun) + } + + private fun generateExecutionId( + isTempWorkflow: Boolean, + workflow: Workflow, + ): String { + val randomPart = "${LocalDateTime.now(ZoneOffset.UTC)}${UUID.randomUUID()}" + return if (isTempWorkflow) randomPart else workflow.id.plus(randomPart) + } + + private fun validateMonitorSize( + delegates: List, + monitors: List, + workflow: Workflow, + ) { + if (delegates.size != monitors.size) { + val diffMonitorIds = delegates.map { it.monitorId }.minus(monitors.map { it.id }.toSet()).joinToString() + logger.error("Delegate monitors don't exist $diffMonitorIds for the workflow $workflow.id") + throw AlertingException.wrap( + IllegalStateException("Delegate monitors don't exist $diffMonitorIds for the workflow $workflow.id") + ) + } + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunContext.kt b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunContext.kt new file mode 100644 index 000000000..60285e70b --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunContext.kt @@ -0,0 +1,15 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.workflow + +data class WorkflowRunContext( + // In case of dry run it's random generated id, while in other cases it's workflowId + val workflowId: String, + val workflowMetadataId: String, + val chainedMonitorId: String?, + val executionId: String, + val matchingDocIdsPerIndex: Map> +) diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunner.kt new file mode 100644 index 000000000..a7272a3dc --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunner.kt @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.workflow + +import org.opensearch.alerting.MonitorRunnerExecutionContext +import org.opensearch.alerting.model.WorkflowRunResult +import org.opensearch.commons.alerting.model.Workflow +import java.time.Instant + +abstract class WorkflowRunner { + abstract suspend fun runWorkflow( + workflow: Workflow, + monitorCtx: MonitorRunnerExecutionContext, + periodStart: Instant, + periodEnd: Instant, + dryRun: Boolean + ): WorkflowRunResult +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunnerService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunnerService.kt new file mode 100644 index 000000000..5ea979119 --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/WorkflowRunnerService.kt @@ -0,0 +1,252 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.workflow + +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.Job +import kotlinx.coroutines.SupervisorJob +import kotlinx.coroutines.launch +import org.apache.logging.log4j.LogManager +import org.opensearch.action.bulk.BackoffPolicy +import org.opensearch.alerting.AlertService +import org.opensearch.alerting.InputService +import org.opensearch.alerting.MonitorRunnerExecutionContext +import org.opensearch.alerting.TriggerService +import org.opensearch.alerting.WorkflowService +import org.opensearch.alerting.alerts.AlertIndices +import org.opensearch.alerting.core.JobRunner +import org.opensearch.alerting.model.WorkflowRunResult +import org.opensearch.alerting.model.destination.DestinationContextFactory +import org.opensearch.alerting.script.TriggerExecutionContext +import org.opensearch.alerting.settings.AlertingSettings.Companion.ALERT_BACKOFF_COUNT +import org.opensearch.alerting.settings.AlertingSettings.Companion.ALERT_BACKOFF_MILLIS +import org.opensearch.alerting.settings.AlertingSettings.Companion.INDEX_TIMEOUT +import org.opensearch.alerting.settings.AlertingSettings.Companion.MAX_ACTIONABLE_ALERT_COUNT +import org.opensearch.alerting.settings.AlertingSettings.Companion.MOVE_ALERTS_BACKOFF_COUNT +import org.opensearch.alerting.settings.AlertingSettings.Companion.MOVE_ALERTS_BACKOFF_MILLIS +import org.opensearch.alerting.settings.DestinationSettings.Companion.ALLOW_LIST +import org.opensearch.alerting.settings.DestinationSettings.Companion.HOST_DENY_LIST +import org.opensearch.alerting.settings.DestinationSettings.Companion.loadDestinationSettings +import org.opensearch.alerting.util.DocLevelMonitorQueries +import org.opensearch.client.Client +import org.opensearch.cluster.metadata.IndexNameExpressionResolver +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.settings.Settings +import org.opensearch.commons.alerting.model.Alert +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.alerting.model.action.Action +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.script.Script +import org.opensearch.script.ScriptService +import org.opensearch.script.TemplateScript +import org.opensearch.threadpool.ThreadPool +import java.time.Instant +import kotlin.coroutines.CoroutineContext + +object WorkflowRunnerService : JobRunner, CoroutineScope, AbstractLifecycleComponent() { + + private val logger = LogManager.getLogger(javaClass) + + var monitorCtx: MonitorRunnerExecutionContext = MonitorRunnerExecutionContext() + private lateinit var runnerSupervisor: Job + override val coroutineContext: CoroutineContext + get() = Dispatchers.Default + runnerSupervisor + + fun registerClusterService(clusterService: ClusterService): WorkflowRunnerService { + monitorCtx.clusterService = clusterService + return this + } + + fun registerClient(client: Client): WorkflowRunnerService { + monitorCtx.client = client + return this + } + + fun registerNamedXContentRegistry(xContentRegistry: NamedXContentRegistry): WorkflowRunnerService { + monitorCtx.xContentRegistry = xContentRegistry + return this + } + + fun registerScriptService(scriptService: ScriptService): WorkflowRunnerService { + monitorCtx.scriptService = scriptService + return this + } + + fun registerIndexNameExpressionResolver(indexNameExpressionResolver: IndexNameExpressionResolver): WorkflowRunnerService { + monitorCtx.indexNameExpressionResolver = indexNameExpressionResolver + return this + } + + fun registerSettings(settings: Settings): WorkflowRunnerService { + monitorCtx.settings = settings + return this + } + + fun registerThreadPool(threadPool: ThreadPool): WorkflowRunnerService { + monitorCtx.threadPool = threadPool + return this + } + + fun registerAlertIndices(alertIndices: AlertIndices): WorkflowRunnerService { + monitorCtx.alertIndices = alertIndices + return this + } + + fun registerInputService(inputService: InputService): WorkflowRunnerService { + monitorCtx.inputService = inputService + return this + } + + fun registerWorkflowService(workflowService: WorkflowService): WorkflowRunnerService { + monitorCtx.workflowService = workflowService + return this + } + + fun registerTriggerService(triggerService: TriggerService): WorkflowRunnerService { + monitorCtx.triggerService = triggerService + return this + } + + fun registerAlertService(alertService: AlertService): WorkflowRunnerService { + monitorCtx.alertService = alertService + return this + } + + fun registerDocLevelMonitorQueries(docLevelMonitorQueries: DocLevelMonitorQueries): WorkflowRunnerService { + monitorCtx.docLevelMonitorQueries = docLevelMonitorQueries + return this + } + + // Must be called after registerClusterService and registerSettings in AlertingPlugin + fun registerConsumers(): WorkflowRunnerService { + monitorCtx.retryPolicy = BackoffPolicy.constantBackoff( + ALERT_BACKOFF_MILLIS.get(monitorCtx.settings), + ALERT_BACKOFF_COUNT.get(monitorCtx.settings) + ) + monitorCtx.clusterService!!.clusterSettings.addSettingsUpdateConsumer(ALERT_BACKOFF_MILLIS, ALERT_BACKOFF_COUNT) { millis, count -> + monitorCtx.retryPolicy = BackoffPolicy.constantBackoff(millis, count) + } + + monitorCtx.moveAlertsRetryPolicy = + BackoffPolicy.exponentialBackoff( + MOVE_ALERTS_BACKOFF_MILLIS.get(monitorCtx.settings), + MOVE_ALERTS_BACKOFF_COUNT.get(monitorCtx.settings) + ) + monitorCtx.clusterService!!.clusterSettings.addSettingsUpdateConsumer(MOVE_ALERTS_BACKOFF_MILLIS, MOVE_ALERTS_BACKOFF_COUNT) { + millis, count -> + monitorCtx.moveAlertsRetryPolicy = BackoffPolicy.exponentialBackoff(millis, count) + } + + monitorCtx.allowList = ALLOW_LIST.get(monitorCtx.settings) + monitorCtx.clusterService!!.clusterSettings.addSettingsUpdateConsumer(ALLOW_LIST) { + monitorCtx.allowList = it + } + + // Host deny list is not a dynamic setting so no consumer is registered but the variable is set here + monitorCtx.hostDenyList = HOST_DENY_LIST.get(monitorCtx.settings) + + monitorCtx.maxActionableAlertCount = MAX_ACTIONABLE_ALERT_COUNT.get(monitorCtx.settings) + monitorCtx.clusterService!!.clusterSettings.addSettingsUpdateConsumer(MAX_ACTIONABLE_ALERT_COUNT) { + monitorCtx.maxActionableAlertCount = it + } + + monitorCtx.indexTimeout = INDEX_TIMEOUT.get(monitorCtx.settings) + + return this + } + + // To be safe, call this last as it depends on a number of other components being registered beforehand (client, settings, etc.) + fun registerDestinationSettings(): WorkflowRunnerService { + monitorCtx.destinationSettings = loadDestinationSettings(monitorCtx.settings!!) + monitorCtx.destinationContextFactory = + DestinationContextFactory(monitorCtx.client!!, monitorCtx.xContentRegistry!!, monitorCtx.destinationSettings!!) + return this + } + + // Updates destination settings when the reload API is called so that new keystore values are visible + fun reloadDestinationSettings(settings: Settings) { + monitorCtx.destinationSettings = loadDestinationSettings(settings) + + // Update destinationContextFactory as well since destinationSettings has been updated + monitorCtx.destinationContextFactory!!.updateDestinationSettings(monitorCtx.destinationSettings!!) + } + + override fun doStart() { + runnerSupervisor = SupervisorJob() + } + + override fun doStop() { + runnerSupervisor.cancel() + } + + override fun doClose() { } + + override fun postIndex(job: ScheduledJob) { + } + + override fun postDelete(jobId: String) { + } + + override fun runJob(job: ScheduledJob, periodStart: Instant, periodEnd: Instant) { + if (job !is Workflow) { + throw IllegalArgumentException("Invalid job type") + } + launch { + runJob(job, periodStart, periodEnd, false) + } + } + + suspend fun runJob(job: ScheduledJob, periodStart: Instant, periodEnd: Instant, dryrun: Boolean): WorkflowRunResult { + val workflow = job as Workflow + return CompositeWorkflowRunner.runWorkflow(workflow, monitorCtx, periodStart, periodEnd, dryrun) + } + + // TODO: See if we can move below methods (or few of these) to a common utils + internal fun getRolesForMonitor(monitor: Monitor): List { + /* + * We need to handle 3 cases: + * 1. Monitors created by older versions and never updated. These monitors wont have User details in the + * monitor object. `monitor.user` will be null. Insert `all_access, AmazonES_all_access` role. + * 2. Monitors are created when security plugin is disabled, these will have empty User object. + * (`monitor.user.name`, `monitor.user.roles` are empty ) + * 3. Monitors are created when security plugin is enabled, these will have an User object. + */ + return if (monitor.user == null) { + // fixme: discuss and remove hardcoded to settings? + // TODO: Remove "AmazonES_all_access" role? + monitorCtx.settings!!.getAsList("", listOf("all_access", "AmazonES_all_access")) + } else { + monitor.user!!.roles + } + } + + // TODO: Can this be updated to just use 'Instant.now()'? + // 'threadPool.absoluteTimeInMillis()' is referring to a cached value of System.currentTimeMillis() that by default updates every 200ms + internal fun currentTime() = Instant.ofEpochMilli(monitorCtx.threadPool!!.absoluteTimeInMillis()) + + internal fun isActionActionable(action: Action, alert: Alert?): Boolean { + if (alert == null || action.throttle == null) { + return true + } + if (action.throttleEnabled) { + val result = alert.actionExecutionResults.firstOrNull { r -> r.actionId == action.id } + val lastExecutionTime: Instant? = result?.lastExecutionTime + val throttledTimeBound = currentTime().minus(action.throttle!!.value.toLong(), action.throttle!!.unit) + return (lastExecutionTime == null || lastExecutionTime.isBefore(throttledTimeBound)) + } + return true + } + + internal fun compileTemplate(template: Script, ctx: TriggerExecutionContext): String { + return monitorCtx.scriptService!!.compile(template, TemplateScript.CONTEXT) + .newInstance(template.params + mapOf("ctx" to ctx.asTemplateArg())) + .execute() + } +} diff --git a/alerting/src/main/resources/org/opensearch/alerting/alerts/finding_mapping.json b/alerting/src/main/resources/org/opensearch/alerting/alerts/finding_mapping.json index 421dc202c..3ffc39478 100644 --- a/alerting/src/main/resources/org/opensearch/alerting/alerts/finding_mapping.json +++ b/alerting/src/main/resources/org/opensearch/alerting/alerts/finding_mapping.json @@ -1,7 +1,7 @@ { "dynamic": "strict", "_meta" : { - "schema_version": 2 + "schema_version": 3 }, "properties": { "schema_version": { @@ -60,6 +60,9 @@ "type" : "keyword" } } + }, + "execution_id": { + "type": "keyword" } } } \ No newline at end of file diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/TestHelpers.kt b/alerting/src/test/kotlin/org/opensearch/alerting/TestHelpers.kt index 2dfd6f593..0eed1a1d6 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/TestHelpers.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/TestHelpers.kt @@ -35,8 +35,11 @@ import org.opensearch.commons.alerting.model.ActionExecutionResult import org.opensearch.commons.alerting.model.AggregationResultBucket import org.opensearch.commons.alerting.model.Alert import org.opensearch.commons.alerting.model.BucketLevelTrigger +import org.opensearch.commons.alerting.model.ChainedMonitorFindings import org.opensearch.commons.alerting.model.ClusterMetricsInput +import org.opensearch.commons.alerting.model.CompositeInput import org.opensearch.commons.alerting.model.DataSources +import org.opensearch.commons.alerting.model.Delegate import org.opensearch.commons.alerting.model.DocLevelMonitorInput import org.opensearch.commons.alerting.model.DocLevelQuery import org.opensearch.commons.alerting.model.DocumentLevelTrigger @@ -47,7 +50,10 @@ import org.opensearch.commons.alerting.model.Monitor import org.opensearch.commons.alerting.model.QueryLevelTrigger import org.opensearch.commons.alerting.model.Schedule import org.opensearch.commons.alerting.model.SearchInput +import org.opensearch.commons.alerting.model.Sequence import org.opensearch.commons.alerting.model.Trigger +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.alerting.model.Workflow.WorkflowType import org.opensearch.commons.alerting.model.action.Action import org.opensearch.commons.alerting.model.action.ActionExecutionPolicy import org.opensearch.commons.alerting.model.action.ActionExecutionScope @@ -218,6 +224,65 @@ fun randomDocumentLevelMonitor( ) } +fun randomWorkflow( + id: String = Workflow.NO_ID, + monitorIds: List, + name: String = OpenSearchRestTestCase.randomAlphaOfLength(10), + user: User? = randomUser(), + schedule: Schedule = IntervalSchedule(interval = 5, unit = ChronoUnit.MINUTES), + enabled: Boolean = randomBoolean(), + enabledTime: Instant? = if (enabled) Instant.now().truncatedTo(ChronoUnit.MILLIS) else null, + lastUpdateTime: Instant = Instant.now().truncatedTo(ChronoUnit.MILLIS) +): Workflow { + val delegates = mutableListOf() + if (!monitorIds.isNullOrEmpty()) { + delegates.add(Delegate(1, monitorIds[0])) + for (i in 1 until monitorIds.size) { + // Order of monitors in workflow will be the same like forwarded meaning that the first monitorId will be used as second monitor chained finding + delegates.add(Delegate(i + 1, monitorIds [i], ChainedMonitorFindings(monitorIds[i - 1]))) + } + } + + return Workflow( + id = id, + name = name, + enabled = enabled, + schedule = schedule, + lastUpdateTime = lastUpdateTime, + enabledTime = enabledTime, + workflowType = WorkflowType.COMPOSITE, + user = user, + inputs = listOf(CompositeInput(Sequence(delegates))), + version = -1L, + schemaVersion = 0 + ) +} + +fun randomWorkflowWithDelegates( + id: String = Workflow.NO_ID, + delegates: List, + name: String = OpenSearchRestTestCase.randomAlphaOfLength(10), + user: User? = randomUser(), + schedule: Schedule = IntervalSchedule(interval = 5, unit = ChronoUnit.MINUTES), + enabled: Boolean = randomBoolean(), + enabledTime: Instant? = if (enabled) Instant.now().truncatedTo(ChronoUnit.MILLIS) else null, + lastUpdateTime: Instant = Instant.now().truncatedTo(ChronoUnit.MILLIS), +): Workflow { + return Workflow( + id = id, + name = name, + enabled = enabled, + schedule = schedule, + lastUpdateTime = lastUpdateTime, + enabledTime = enabledTime, + workflowType = WorkflowType.COMPOSITE, + user = user, + inputs = listOf(CompositeInput(Sequence(delegates))), + version = -1L, + schemaVersion = 0 + ) +} + fun randomQueryLevelTrigger( id: String = UUIDs.base64UUID(), name: String = OpenSearchRestTestCase.randomAlphaOfLength(10), @@ -699,3 +764,7 @@ fun assertUserNull(map: Map) { fun assertUserNull(monitor: Monitor) { assertNull("User is not null", monitor.user) } + +fun assertUserNull(workflow: Workflow) { + assertNull("User is not null", workflow.user) +} diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowMonitorIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowMonitorIT.kt new file mode 100644 index 000000000..6e98ea102 --- /dev/null +++ b/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowMonitorIT.kt @@ -0,0 +1,1269 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting + +import org.opensearch.alerting.model.WorkflowMetadata +import org.opensearch.alerting.transport.WorkflowSingleNodeTestCase +import org.opensearch.commons.alerting.action.IndexMonitorResponse +import org.opensearch.commons.alerting.model.ChainedMonitorFindings +import org.opensearch.commons.alerting.model.CompositeInput +import org.opensearch.commons.alerting.model.DataSources +import org.opensearch.commons.alerting.model.Delegate +import org.opensearch.commons.alerting.model.DocLevelMonitorInput +import org.opensearch.commons.alerting.model.DocLevelQuery +import org.opensearch.commons.alerting.model.Monitor +import org.opensearch.rest.RestRequest +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter +import java.time.temporal.ChronoUnit +import java.util.Collections +import java.util.UUID + +class WorkflowMonitorIT : WorkflowSingleNodeTestCase() { + + fun `test create workflow success`() { + val docQuery1 = DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3") + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val customQueryIndex = "custom_alerts_index" + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitorResponse1 = createMonitor(monitor1)!! + val monitorResponse2 = createMonitor(monitor2)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + assertNotNull(workflowResponse.workflow) + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowResponse.id) + assertTrue("incorrect version", workflowResponse.version > 0) + + val workflowById = searchWorkflow(workflowResponse.id)!! + assertNotNull(workflowById) + + // Verify workflow + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowById.id) + assertTrue("incorrect version", workflowById.version > 0) + assertEquals("Workflow name not correct", workflow.name, workflowById.name) + assertEquals("Workflow owner not correct", workflow.owner, workflowById.owner) + assertEquals("Workflow input not correct", workflow.inputs, workflowById.inputs) + + // Delegate verification + @Suppress("UNCHECKED_CAST") + val delegates = (workflowById.inputs as List)[0].sequence.delegates.sortedBy { it.order } + assertEquals("Delegates size not correct", 2, delegates.size) + + val delegate1 = delegates[0] + assertNotNull(delegate1) + assertEquals("Delegate1 order not correct", 1, delegate1.order) + assertEquals("Delegate1 id not correct", monitorResponse1.id, delegate1.monitorId) + + val delegate2 = delegates[1] + assertNotNull(delegate2) + assertEquals("Delegate2 order not correct", 2, delegate2.order) + assertEquals("Delegate2 id not correct", monitorResponse2.id, delegate2.monitorId) + assertEquals( + "Delegate2 Chained finding not correct", monitorResponse1.id, delegate2.chainedMonitorFindings!!.monitorId + ) + } + + fun `test update workflow add monitor success`() { + val docQuery1 = DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3") + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val customQueryIndex = "custom_alerts_index" + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitorResponse1 = createMonitor(monitor1)!! + val monitorResponse2 = createMonitor(monitor2)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + assertNotNull(workflowResponse.workflow) + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowResponse.id) + assertTrue("incorrect version", workflowResponse.version > 0) + + var workflowById = searchWorkflow(workflowResponse.id)!! + assertNotNull(workflowById) + + val monitor3 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + val monitorResponse3 = createMonitor(monitor3)!! + + val updatedWorkflowResponse = upsertWorkflow( + randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id, monitorResponse3.id) + ), + workflowResponse.id, + RestRequest.Method.PUT + )!! + + assertNotNull("Workflow creation failed", updatedWorkflowResponse) + assertNotNull(updatedWorkflowResponse.workflow) + assertEquals("Workflow id changed", workflowResponse.id, updatedWorkflowResponse.id) + assertTrue("incorrect version", updatedWorkflowResponse.version > 0) + + workflowById = searchWorkflow(updatedWorkflowResponse.id)!! + + // Verify workflow + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowById.id) + assertTrue("incorrect version", workflowById.version > 0) + assertEquals("Workflow name not correct", updatedWorkflowResponse.workflow.name, workflowById.name) + assertEquals("Workflow owner not correct", updatedWorkflowResponse.workflow.owner, workflowById.owner) + assertEquals("Workflow input not correct", updatedWorkflowResponse.workflow.inputs, workflowById.inputs) + + // Delegate verification + @Suppress("UNCHECKED_CAST") + val delegates = (workflowById.inputs as List)[0].sequence.delegates.sortedBy { it.order } + assertEquals("Delegates size not correct", 3, delegates.size) + + val delegate1 = delegates[0] + assertNotNull(delegate1) + assertEquals("Delegate1 order not correct", 1, delegate1.order) + assertEquals("Delegate1 id not correct", monitorResponse1.id, delegate1.monitorId) + + val delegate2 = delegates[1] + assertNotNull(delegate2) + assertEquals("Delegate2 order not correct", 2, delegate2.order) + assertEquals("Delegate2 id not correct", monitorResponse2.id, delegate2.monitorId) + assertEquals( + "Delegate2 Chained finding not correct", monitorResponse1.id, delegate2.chainedMonitorFindings!!.monitorId + ) + + val delegate3 = delegates[2] + assertNotNull(delegate3) + assertEquals("Delegate3 order not correct", 3, delegate3.order) + assertEquals("Delegate3 id not correct", monitorResponse3.id, delegate3.monitorId) + assertEquals( + "Delegate3 Chained finding not correct", monitorResponse2.id, delegate3.chainedMonitorFindings!!.monitorId + ) + } + + fun `test update workflow change order of delegate monitors`() { + val docQuery1 = DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3") + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val customQueryIndex = "custom_alerts_index" + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitorResponse1 = createMonitor(monitor1)!! + val monitorResponse2 = createMonitor(monitor2)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + assertNotNull(workflowResponse.workflow) + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowResponse.id) + assertTrue("incorrect version", workflowResponse.version > 0) + + var workflowById = searchWorkflow(workflowResponse.id)!! + assertNotNull(workflowById) + + val updatedWorkflowResponse = upsertWorkflow( + randomWorkflow( + monitorIds = listOf(monitorResponse2.id, monitorResponse1.id) + ), + workflowResponse.id, + RestRequest.Method.PUT + )!! + + assertNotNull("Workflow creation failed", updatedWorkflowResponse) + assertNotNull(updatedWorkflowResponse.workflow) + assertEquals("Workflow id changed", workflowResponse.id, updatedWorkflowResponse.id) + assertTrue("incorrect version", updatedWorkflowResponse.version > 0) + + workflowById = searchWorkflow(updatedWorkflowResponse.id)!! + + // Verify workflow + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowById.id) + assertTrue("incorrect version", workflowById.version > 0) + assertEquals("Workflow name not correct", updatedWorkflowResponse.workflow.name, workflowById.name) + assertEquals("Workflow owner not correct", updatedWorkflowResponse.workflow.owner, workflowById.owner) + assertEquals("Workflow input not correct", updatedWorkflowResponse.workflow.inputs, workflowById.inputs) + + // Delegate verification + @Suppress("UNCHECKED_CAST") + val delegates = (workflowById.inputs as List)[0].sequence.delegates.sortedBy { it.order } + assertEquals("Delegates size not correct", 2, delegates.size) + + val delegate1 = delegates[0] + assertNotNull(delegate1) + assertEquals("Delegate1 order not correct", 1, delegate1.order) + assertEquals("Delegate1 id not correct", monitorResponse2.id, delegate1.monitorId) + + val delegate2 = delegates[1] + assertNotNull(delegate2) + assertEquals("Delegate2 order not correct", 2, delegate2.order) + assertEquals("Delegate2 id not correct", monitorResponse1.id, delegate2.monitorId) + assertEquals( + "Delegate2 Chained finding not correct", monitorResponse2.id, delegate2.chainedMonitorFindings!!.monitorId + ) + } + + fun `test update workflow remove monitor success`() { + val docQuery1 = DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3") + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val customQueryIndex = "custom_alerts_index" + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitorResponse1 = createMonitor(monitor1)!! + val monitorResponse2 = createMonitor(monitor2)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + assertNotNull(workflowResponse.workflow) + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowResponse.id) + assertTrue("incorrect version", workflowResponse.version > 0) + + var workflowById = searchWorkflow(workflowResponse.id)!! + assertNotNull(workflowById) + + val updatedWorkflowResponse = upsertWorkflow( + randomWorkflow( + monitorIds = listOf(monitorResponse1.id) + ), + workflowResponse.id, + RestRequest.Method.PUT + )!! + + assertNotNull("Workflow creation failed", updatedWorkflowResponse) + assertNotNull(updatedWorkflowResponse.workflow) + assertEquals("Workflow id changed", workflowResponse.id, updatedWorkflowResponse.id) + assertTrue("incorrect version", updatedWorkflowResponse.version > 0) + + workflowById = searchWorkflow(updatedWorkflowResponse.id)!! + + // Verify workflow + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowById.id) + assertTrue("incorrect version", workflowById.version > 0) + assertEquals("Workflow name not correct", updatedWorkflowResponse.workflow.name, workflowById.name) + assertEquals("Workflow owner not correct", updatedWorkflowResponse.workflow.owner, workflowById.owner) + assertEquals("Workflow input not correct", updatedWorkflowResponse.workflow.inputs, workflowById.inputs) + + // Delegate verification + @Suppress("UNCHECKED_CAST") + val delegates = (workflowById.inputs as List)[0].sequence.delegates.sortedBy { it.order } + assertEquals("Delegates size not correct", 1, delegates.size) + + val delegate1 = delegates[0] + assertNotNull(delegate1) + assertEquals("Delegate1 order not correct", 1, delegate1.order) + assertEquals("Delegate1 id not correct", monitorResponse1.id, delegate1.monitorId) + } + + fun `test update workflow doesn't exist failure`() { + val docQuery1 = DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3") + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val customQueryIndex = "custom_alerts_index" + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + queryIndex = customQueryIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + + val monitorResponse1 = createMonitor(monitor1)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + try { + upsertWorkflow(workflow, "testId", RestRequest.Method.PUT) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow with testId is not found") + ) + } + } + } + + fun `test get workflow`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + + val workflowResponse = upsertWorkflow(workflowRequest)!! + assertNotNull("Workflow creation failed", workflowResponse) + assertNotNull(workflowResponse.workflow) + assertNotEquals("response is missing Id", Monitor.NO_ID, workflowResponse.id) + assertTrue("incorrect version", workflowResponse.version > 0) + + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + assertNotNull(getWorkflowResponse) + + val workflowById = getWorkflowResponse.workflow!! + // Verify workflow + assertNotEquals("response is missing Id", Monitor.NO_ID, getWorkflowResponse.id) + assertTrue("incorrect version", getWorkflowResponse.version > 0) + assertEquals("Workflow name not correct", workflowRequest.name, workflowById.name) + assertEquals("Workflow owner not correct", workflowRequest.owner, workflowById.owner) + assertEquals("Workflow input not correct", workflowRequest.inputs, workflowById.inputs) + + // Delegate verification + @Suppress("UNCHECKED_CAST") + val delegates = (workflowById.inputs as List)[0].sequence.delegates.sortedBy { it.order } + assertEquals("Delegates size not correct", 1, delegates.size) + + val delegate = delegates[0] + assertNotNull(delegate) + assertEquals("Delegate order not correct", 1, delegate.order) + assertEquals("Delegate id not correct", monitorResponse.id, delegate.monitorId) + } + + fun `test get workflow for invalid id monitor index doesn't exist`() { + // Get workflow for non existing workflow id + try { + getWorkflowById(id = "-1") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow not found") + ) + } + } + } + + fun `test get workflow for invalid id monitor index exists`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + ) + createMonitor(monitor) + // Get workflow for non existing workflow id + try { + getWorkflowById(id = "-1") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow not found") + ) + } + } + } + + fun `test delete workflow keeping delegate monitor`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflowRequest)!! + val workflowId = workflowResponse.id + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + + assertNotNull(getWorkflowResponse) + assertEquals(workflowId, getWorkflowResponse.id) + + deleteWorkflow(workflowId, false) + // Verify that the workflow is deleted + try { + getWorkflowById(workflowId) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow not found.") + ) + } + } + // Verify that the monitor is not deleted + val existingDelegate = getMonitorResponse(monitorResponse.id) + assertNotNull(existingDelegate) + } + + fun `test delete workflow delegate monitor deleted`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflowRequest)!! + val workflowId = workflowResponse.id + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + + assertNotNull(getWorkflowResponse) + assertEquals(workflowId, getWorkflowResponse.id) + + deleteWorkflow(workflowId, true) + // Verify that the workflow is deleted + try { + getWorkflowById(workflowId) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow not found.") + ) + } + } + // Verify that the monitor is deleted + try { + getMonitorResponse(monitorResponse.id) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetMonitor Action error ", + it.contains("Monitor not found") + ) + } + } + } + + fun `test delete executed workflow with metadata deleted`() { + val docQuery1 = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput1 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery1)) + val trigger1 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput1), + triggers = listOf(trigger1) + ) + val monitorResponse = createMonitor(monitor1)!! + + val docQuery2 = DocLevelQuery(query = "source.ip.v6.v2:16645", name = "4") + val docLevelInput2 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery2)) + val trigger2 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput2), + triggers = listOf(trigger2), + ) + + val monitorResponse2 = createMonitor(monitor2)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id, monitorResponse2.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults = executeWorkflowResponse.workflowRunResult.workflowRunResult + assertEquals(2, monitorsRunResults.size) + + val workflowMetadata = searchWorkflowMetadata(workflowId) + assertNotNull(workflowMetadata) + + val monitorMetadataId1 = getDelegateMonitorMetadataId(workflowMetadata, monitorResponse) + val monitorMetadata1 = searchMonitorMetadata(monitorMetadataId1) + assertNotNull(monitorMetadata1) + + val monitorMetadataId2 = getDelegateMonitorMetadataId(workflowMetadata, monitorResponse2) + val monitorMetadata2 = searchMonitorMetadata(monitorMetadataId2) + assertNotNull(monitorMetadata2) + + assertFalse(monitorMetadata1!!.id == monitorMetadata2!!.id) + + deleteWorkflow(workflowId, true) + // Verify that the workflow is deleted + try { + getWorkflowById(workflowId) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("Workflow not found.") + ) + } + } + // Verify that the workflow metadata is deleted + try { + searchWorkflowMetadata(workflowId) + fail("expected searchWorkflowMetadata method to throw exception") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetMonitor Action error ", + it.contains("List is empty") + ) + } + } + // Verify that the monitors metadata are deleted + try { + searchMonitorMetadata(monitorMetadataId1) + fail("expected searchMonitorMetadata method to throw exception") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetMonitor Action error ", + it.contains("List is empty") + ) + } + } + + try { + searchMonitorMetadata(monitorMetadataId2) + fail("expected searchMonitorMetadata method to throw exception") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetMonitor Action error ", + it.contains("List is empty") + ) + } + } + } + + private fun getDelegateMonitorMetadataId( + workflowMetadata: WorkflowMetadata?, + monitorResponse: IndexMonitorResponse, + ) = "${workflowMetadata!!.id}-${monitorResponse.id}-metadata" + + fun `test delete workflow delegate monitor part of another workflow not deleted`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflowRequest)!! + val workflowId = workflowResponse.id + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + + assertNotNull(getWorkflowResponse) + assertEquals(workflowId, getWorkflowResponse.id) + + val workflowRequest2 = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse2 = upsertWorkflow(workflowRequest2)!! + val workflowId2 = workflowResponse2.id + val getWorkflowResponse2 = getWorkflowById(id = workflowResponse2.id) + + assertNotNull(getWorkflowResponse2) + assertEquals(workflowId2, getWorkflowResponse2.id) + + try { + deleteWorkflow(workflowId, true) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetWorkflow Action error ", + it.contains("[Not allowed to delete ${monitorResponse.id} monitors") + ) + } + } + val existingMonitor = getMonitorResponse(monitorResponse.id) + assertNotNull(existingMonitor) + } + + fun `test trying to delete monitor that is part of workflow sequence`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + + val workflowResponse = upsertWorkflow(workflowRequest)!! + val workflowId = workflowResponse.id + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + + assertNotNull(getWorkflowResponse) + assertEquals(workflowId, getWorkflowResponse.id) + + // Verify that the monitor can't be deleted because it's included in the workflow + try { + deleteMonitor(monitorResponse.id) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning DeleteMonitor Action error ", + it.contains("Monitor can't be deleted because it is a part of workflow(s)") + ) + } + } + } + + fun `test delete workflow for invalid id monitor index doesn't exists`() { + // Try deleting non-existing workflow + try { + deleteWorkflow("-1") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning DeleteWorkflow Action error ", + it.contains("Workflow not found.") + ) + } + } + } + + fun `test delete workflow for invalid id monitor index exists`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + ) + createMonitor(monitor) + // Try deleting non-existing workflow + try { + deleteWorkflow("-1") + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning DeleteWorkflow Action error ", + it.contains("Workflow not found.") + ) + } + } + } + + fun `test create workflow without delegate failure`() { + val workflow = randomWorkflow( + monitorIds = Collections.emptyList() + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Delegates list can not be empty.") + ) + } + } + } + + fun `test create workflow with 26 delegates failure`() { + val monitorsIds = mutableListOf() + for (i in 0..25) { + monitorsIds.add(UUID.randomUUID().toString()) + } + val workflow = randomWorkflow( + monitorIds = monitorsIds + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Delegates list can not be larger then 25.") + ) + } + } + } + + fun `test update workflow without delegate failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + ) + + val monitorResponse1 = createMonitor(monitor1)!! + val monitorResponse2 = createMonitor(monitor2)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse1.id, monitorResponse2.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + workflow = randomWorkflow( + id = workflowResponse.id, + monitorIds = Collections.emptyList() + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Delegates list can not be empty.") + ) + } + } + } + + fun `test create workflow duplicate delegate failure`() { + val workflow = randomWorkflow( + monitorIds = listOf("1", "1", "2") + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Duplicate delegates not allowed") + ) + } + } + } + + fun `test update workflow duplicate delegate failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + workflow = randomWorkflow( + id = workflowResponse.id, + monitorIds = listOf("1", "1", "2") + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Duplicate delegates not allowed") + ) + } + } + } + + fun `test create workflow delegate monitor doesn't exist failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + val workflow = randomWorkflow( + monitorIds = listOf("-1", monitorResponse.id) + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("are not valid monitor ids") + ) + } + } + } + + fun `test update workflow delegate monitor doesn't exist failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + workflow = randomWorkflow( + id = workflowResponse.id, + monitorIds = listOf("-1", monitorResponse.id) + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("are not valid monitor ids") + ) + } + } + } + + fun `test create workflow sequence order not correct failure`() { + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(1, "monitor-2"), + Delegate(2, "monitor-3") + ) + val workflow = randomWorkflowWithDelegates( + delegates = delegates + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Sequence ordering of delegate monitor shouldn't contain duplicate order values") + ) + } + } + } + + fun `test update workflow sequence order not correct failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(1, "monitor-2"), + Delegate(2, "monitor-3") + ) + workflow = randomWorkflowWithDelegates( + id = workflowResponse.id, + delegates = delegates + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Sequence ordering of delegate monitor shouldn't contain duplicate order values") + ) + } + } + } + + fun `test create workflow chained findings monitor not in sequence failure`() { + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(2, "monitor-2", ChainedMonitorFindings("monitor-1")), + Delegate(3, "monitor-3", ChainedMonitorFindings("monitor-x")) + ) + val workflow = randomWorkflowWithDelegates( + delegates = delegates + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Chained Findings Monitor monitor-x doesn't exist in sequence") + ) + } + } + } + + fun `test create workflow query monitor chained findings monitor failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val docMonitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val docMonitorResponse = createMonitor(docMonitor)!! + + val queryMonitor = randomQueryLevelMonitor() + val queryMonitorResponse = createMonitor(queryMonitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(queryMonitorResponse.id, docMonitorResponse.id) + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Query level monitor can't be part of chained findings") + ) + } + } + } + + fun `test create workflow delegate and chained finding monitor different indices failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val docMonitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val docMonitorResponse = createMonitor(docMonitor)!! + + val index1 = "$index-1" + createTestIndex(index1) + + val docLevelInput1 = DocLevelMonitorInput( + "description", listOf(index1), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + + val docMonitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput1), + triggers = listOf(trigger) + ) + val docMonitorResponse1 = createMonitor(docMonitor1)!! + + val workflow = randomWorkflow( + monitorIds = listOf(docMonitorResponse1.id, docMonitorResponse.id) + ) + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Delegate monitor and it's chained finding monitor must query the same indices") + ) + } + } + } + + fun `test create workflow when monitor index not initialized failure`() { + val delegates = listOf( + Delegate(1, "monitor-1") + ) + val workflow = randomWorkflowWithDelegates( + delegates = delegates + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Monitors not found") + ) + } + } + } + + fun `test update workflow chained findings monitor not in sequence failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(2, "monitor-2", ChainedMonitorFindings("monitor-1")), + Delegate(3, "monitor-3", ChainedMonitorFindings("monitor-x")) + ) + workflow = randomWorkflowWithDelegates( + id = workflowResponse.id, + delegates = delegates + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Chained Findings Monitor monitor-x doesn't exist in sequence") + ) + } + } + } + + fun `test create workflow chained findings order not correct failure`() { + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(3, "monitor-2", ChainedMonitorFindings("monitor-1")), + Delegate(2, "monitor-3", ChainedMonitorFindings("monitor-2")) + ) + val workflow = randomWorkflowWithDelegates( + delegates = delegates + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Chained Findings Monitor monitor-2 should be executed before monitor monitor-3") + ) + } + } + } + + fun `test update workflow chained findings order not correct failure`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + assertNotNull("Workflow creation failed", workflowResponse) + + val delegates = listOf( + Delegate(1, "monitor-1"), + Delegate(3, "monitor-2", ChainedMonitorFindings("monitor-1")), + Delegate(2, "monitor-3", ChainedMonitorFindings("monitor-2")) + ) + workflow = randomWorkflowWithDelegates( + delegates = delegates + ) + + try { + upsertWorkflow(workflow) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning IndexWorkflow Action error ", + it.contains("Chained Findings Monitor monitor-2 should be executed before monitor monitor-3") + ) + } + } + } +} diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowRunnerIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowRunnerIT.kt new file mode 100644 index 000000000..f2daf618b --- /dev/null +++ b/alerting/src/test/kotlin/org/opensearch/alerting/WorkflowRunnerIT.kt @@ -0,0 +1,999 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting + +import org.junit.Assert +import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.alerts.AlertIndices +import org.opensearch.alerting.model.DocumentLevelTriggerRunResult +import org.opensearch.alerting.transport.WorkflowSingleNodeTestCase +import org.opensearch.alerting.util.AlertingException +import org.opensearch.commons.alerting.action.AcknowledgeAlertRequest +import org.opensearch.commons.alerting.action.AlertingActions +import org.opensearch.commons.alerting.action.GetAlertsRequest +import org.opensearch.commons.alerting.action.GetAlertsResponse +import org.opensearch.commons.alerting.aggregation.bucketselectorext.BucketSelectorExtAggregationBuilder +import org.opensearch.commons.alerting.model.Alert +import org.opensearch.commons.alerting.model.DataSources +import org.opensearch.commons.alerting.model.DocLevelMonitorInput +import org.opensearch.commons.alerting.model.DocLevelQuery +import org.opensearch.commons.alerting.model.SearchInput +import org.opensearch.commons.alerting.model.Table +import org.opensearch.index.query.QueryBuilders +import org.opensearch.rest.RestStatus +import org.opensearch.script.Script +import org.opensearch.search.aggregations.bucket.composite.CompositeAggregationBuilder +import org.opensearch.search.aggregations.bucket.composite.TermsValuesSourceBuilder +import org.opensearch.search.builder.SearchSourceBuilder +import java.lang.Exception +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter +import java.time.temporal.ChronoUnit +import java.util.NoSuchElementException +import java.util.concurrent.ExecutionException + +class WorkflowRunnerIT : WorkflowSingleNodeTestCase() { + + fun `test execute workflow with custom alerts and finding index with doc level delegates`() { + val docQuery1 = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput1 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery1)) + val trigger1 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customAlertsIndex1 = "custom_alerts_index" + val customFindingsIndex1 = "custom_findings_index" + val customFindingsIndexPattern1 = "custom_findings_index-1" + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput1), + triggers = listOf(trigger1), + dataSources = DataSources( + alertsIndex = customAlertsIndex1, + findingsIndex = customFindingsIndex1, + findingsIndexPattern = customFindingsIndexPattern1 + ) + ) + val monitorResponse = createMonitor(monitor1)!! + + val docQuery2 = DocLevelQuery(query = "source.ip.v6.v2:16645", name = "4") + val docLevelInput2 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery2)) + val trigger2 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customAlertsIndex2 = "custom_alerts_index_2" + val customFindingsIndex2 = "custom_findings_index_2" + val customFindingsIndexPattern2 = "custom_findings_index-2" + var monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput2), + triggers = listOf(trigger2), + dataSources = DataSources( + alertsIndex = customAlertsIndex2, + findingsIndex = customFindingsIndex2, + findingsIndexPattern = customFindingsIndexPattern2 + ) + ) + + val monitorResponse2 = createMonitor(monitor2)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id, monitorResponse2.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + + testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 and monitor2 + val testDoc2 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16645, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "2", testDoc2) + + testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Doesn't match + val testDoc3 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16645, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-east-1" + }""" + indexDoc(index, "3", testDoc3) + + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults = executeWorkflowResponse.workflowRunResult.workflowRunResult + assertEquals(2, monitorsRunResults.size) + + assertEquals(monitor1.name, monitorsRunResults[0].monitorName) + assertEquals(1, monitorsRunResults[0].triggerResults.size) + + Assert.assertEquals(monitor2.name, monitorsRunResults[1].monitorName) + Assert.assertEquals(1, monitorsRunResults[1].triggerResults.size) + + val getAlertsResponse = assertAlerts(monitorResponse.id, customAlertsIndex1, 2) + assertAcknowledges(getAlertsResponse.alerts, monitorResponse.id, 2) + assertFindings(monitorResponse.id, customFindingsIndex1, 2, 2, listOf("1", "2")) + + val getAlertsResponse2 = assertAlerts(monitorResponse2.id, customAlertsIndex2, 1) + assertAcknowledges(getAlertsResponse2.alerts, monitorResponse2.id, 1) + assertFindings(monitorResponse2.id, customFindingsIndex2, 1, 1, listOf("2")) + } + + fun `test execute workflows with shared doc level monitor delegate`() { + val docQuery = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(index), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val customAlertsIndex = "custom_alerts_index" + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + dataSources = DataSources( + alertsIndex = customAlertsIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ) + val monitorResponse = createMonitor(monitor)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + var workflow1 = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse1 = upsertWorkflow(workflow1)!! + val workflowById1 = searchWorkflow(workflowResponse1.id) + assertNotNull(workflowById1) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + + testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + val testDoc2 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16645, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "2", testDoc2) + + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults = executeWorkflowResponse.workflowRunResult.workflowRunResult + assertEquals(1, monitorsRunResults.size) + + assertEquals(monitor.name, monitorsRunResults[0].monitorName) + assertEquals(1, monitorsRunResults[0].triggerResults.size) + + // Assert and not ack the alerts (in order to verify later on that all the alerts are generated) + assertAlerts(monitorResponse.id, customAlertsIndex, 2) + assertFindings(monitorResponse.id, customFindingsIndex, 2, 2, listOf("1", "2")) + // Verify workflow and monitor delegate metadata + val workflowMetadata = searchWorkflowMetadata(id = workflowId) + assertNotNull("Workflow metadata not initialized", workflowMetadata) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse.workflowRunResult.executionId, + workflowMetadata!!.latestExecutionId + ) + val monitorMetadataId = "${monitorResponse.id}-${workflowMetadata.id}" + val monitorMetadata = searchMonitorMetadata(monitorMetadataId) + assertNotNull(monitorMetadata) + + // Execute second workflow + val workflowId1 = workflowResponse1.id + val executeWorkflowResponse1 = executeWorkflow(workflowById1, workflowId1, false)!! + val monitorsRunResults1 = executeWorkflowResponse1.workflowRunResult.workflowRunResult + assertEquals(1, monitorsRunResults1.size) + + assertEquals(monitor.name, monitorsRunResults1[0].monitorName) + assertEquals(1, monitorsRunResults1[0].triggerResults.size) + + val getAlertsResponse = assertAlerts(monitorResponse.id, customAlertsIndex, 4) + assertAcknowledges(getAlertsResponse.alerts, monitorResponse.id, 4) + assertFindings(monitorResponse.id, customFindingsIndex, 4, 4, listOf("1", "2", "1", "2")) + // Verify workflow and monitor delegate metadata + val workflowMetadata1 = searchWorkflowMetadata(id = workflowId1) + assertNotNull("Workflow metadata not initialized", workflowMetadata1) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse1.workflowRunResult.executionId, + workflowMetadata1!!.latestExecutionId + ) + val monitorMetadataId1 = "${monitorResponse.id}-${workflowMetadata1.id}" + val monitorMetadata1 = searchMonitorMetadata(monitorMetadataId1) + assertNotNull(monitorMetadata1) + // Verify that for two workflows two different doc level monitor metadata has been created + assertTrue("Different monitor is used in workflows", monitorMetadata!!.monitorId == monitorMetadata1!!.monitorId) + assertTrue(monitorMetadata.id != monitorMetadata1.id) + } + + fun `test execute workflows with shared doc level monitor delegate updating delegate datasource`() { + val docQuery = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(index), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor)!! + + val workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + val workflow1 = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse1 = upsertWorkflow(workflow1)!! + val workflowById1 = searchWorkflow(workflowResponse1.id) + assertNotNull(workflowById1) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + + testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + val testDoc2 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16645, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "2", testDoc2) + + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults = executeWorkflowResponse.workflowRunResult.workflowRunResult + assertEquals(1, monitorsRunResults.size) + + assertEquals(monitor.name, monitorsRunResults[0].monitorName) + assertEquals(1, monitorsRunResults[0].triggerResults.size) + + assertAlerts(monitorResponse.id, AlertIndices.ALERT_INDEX, 2) + assertFindings(monitorResponse.id, AlertIndices.FINDING_HISTORY_WRITE_INDEX, 2, 2, listOf("1", "2")) + // Verify workflow and monitor delegate metadata + val workflowMetadata = searchWorkflowMetadata(id = workflowId) + assertNotNull("Workflow metadata not initialized", workflowMetadata) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse.workflowRunResult.executionId, + workflowMetadata!!.latestExecutionId + ) + val monitorMetadataId = "${monitorResponse.id}-${workflowMetadata.id}" + val monitorMetadata = searchMonitorMetadata(monitorMetadataId) + assertNotNull(monitorMetadata) + + val customAlertsIndex = "custom_alerts_index" + val customFindingsIndex = "custom_findings_index" + val customFindingsIndexPattern = "custom_findings_index-1" + val monitorId = monitorResponse.id + updateMonitor( + monitor = monitor.copy( + dataSources = DataSources( + alertsIndex = customAlertsIndex, + findingsIndex = customFindingsIndex, + findingsIndexPattern = customFindingsIndexPattern + ) + ), + monitorId + ) + + // Execute second workflow + val workflowId1 = workflowResponse1.id + val executeWorkflowResponse1 = executeWorkflow(workflowById1, workflowId1, false)!! + val monitorsRunResults1 = executeWorkflowResponse1.workflowRunResult.workflowRunResult + assertEquals(1, monitorsRunResults1.size) + + assertEquals(monitor.name, monitorsRunResults1[0].monitorName) + assertEquals(1, monitorsRunResults1[0].triggerResults.size) + + // Verify alerts for the custom index + val getAlertsResponse = assertAlerts(monitorResponse.id, customAlertsIndex, 2) + assertAcknowledges(getAlertsResponse.alerts, monitorResponse.id, 2) + assertFindings(monitorResponse.id, customFindingsIndex, 2, 2, listOf("1", "2")) + + // Verify workflow and monitor delegate metadata + val workflowMetadata1 = searchWorkflowMetadata(id = workflowId1) + assertNotNull("Workflow metadata not initialized", workflowMetadata1) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse1.workflowRunResult.executionId, + workflowMetadata1!!.latestExecutionId + ) + val monitorMetadataId1 = "${monitorResponse.id}-${workflowMetadata1.id}" + val monitorMetadata1 = searchMonitorMetadata(monitorMetadataId1) + assertNotNull(monitorMetadata1) + // Verify that for two workflows two different doc level monitor metadata has been created + assertTrue("Different monitor is used in workflows", monitorMetadata!!.monitorId == monitorMetadata1!!.monitorId) + assertTrue(monitorMetadata.id != monitorMetadata1.id) + } + + fun `test execute workflow verify workflow metadata`() { + val docQuery1 = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput1 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery1)) + val trigger1 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput1), + triggers = listOf(trigger1) + ) + val monitorResponse = createMonitor(monitor1)!! + + val docQuery2 = DocLevelQuery(query = "source.ip.v6.v2:16645", name = "4") + val docLevelInput2 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery2)) + val trigger2 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput2), + triggers = listOf(trigger2), + ) + + val monitorResponse2 = createMonitor(monitor2)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id, monitorResponse2.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + // First execution + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults = executeWorkflowResponse.workflowRunResult.workflowRunResult + assertEquals(2, monitorsRunResults.size) + + val workflowMetadata = searchWorkflowMetadata(id = workflowId) + assertNotNull("Workflow metadata not initialized", workflowMetadata) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse.workflowRunResult.executionId, + workflowMetadata!!.latestExecutionId + ) + val monitorMetadataId = "${monitorResponse.id}-${workflowMetadata.id}" + val monitorMetadata = searchMonitorMetadata(monitorMetadataId) + assertNotNull(monitorMetadata) + + // Second execution + val executeWorkflowResponse1 = executeWorkflow(workflowById, workflowId, false)!! + val monitorsRunResults1 = executeWorkflowResponse1.workflowRunResult.workflowRunResult + assertEquals(2, monitorsRunResults1.size) + + val workflowMetadata1 = searchWorkflowMetadata(id = workflowId) + assertNotNull("Workflow metadata not initialized", workflowMetadata) + assertEquals( + "Workflow metadata execution id not correct", + executeWorkflowResponse1.workflowRunResult.executionId, + workflowMetadata1!!.latestExecutionId + ) + val monitorMetadataId1 = "${monitorResponse.id}-${workflowMetadata1.id}" + assertTrue(monitorMetadataId == monitorMetadataId1) + val monitorMetadata1 = searchMonitorMetadata(monitorMetadataId1) + assertNotNull(monitorMetadata1) + } + + fun `test execute workflow dryrun verify workflow metadata not created`() { + val docQuery1 = DocLevelQuery(query = "test_field_1:\"us-west-2\"", name = "3") + val docLevelInput1 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery1)) + val trigger1 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput1), + triggers = listOf(trigger1) + ) + val monitorResponse = createMonitor(monitor1)!! + + val docQuery2 = DocLevelQuery(query = "source.ip.v6.v2:16645", name = "4") + val docLevelInput2 = DocLevelMonitorInput("description", listOf(index), listOf(docQuery2)) + val trigger2 = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor2 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput2), + triggers = listOf(trigger2), + ) + + val monitorResponse2 = createMonitor(monitor2)!! + + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id, monitorResponse2.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + var testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(ChronoUnit.MILLIS)) + // Matches monitor1 + val testDoc1 = """{ + "message" : "This is an error from IAD region", + "source.ip.v6.v2" : 16644, + "test_strict_date_time" : "$testTime", + "test_field_1" : "us-west-2" + }""" + indexDoc(index, "1", testDoc1) + // First execution + val workflowId = workflowResponse.id + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, true) + + assertNotNull("Workflow run result is null", executeWorkflowResponse) + val monitorsRunResults = executeWorkflowResponse!!.workflowRunResult.workflowRunResult + assertEquals(2, monitorsRunResults.size) + + var exception: Exception? = null + try { + searchWorkflowMetadata(id = workflowId) + } catch (ex: Exception) { + exception = ex + } + assertTrue(exception is NoSuchElementException) + } + + fun `test execute workflow with custom alerts and finding index when bucket monitor is used in chained finding of doc monitor`() { + val query = QueryBuilders.rangeQuery("test_strict_date_time") + .gt("{{period_end}}||-10d") + .lte("{{period_end}}") + .format("epoch_millis") + val compositeSources = listOf( + TermsValuesSourceBuilder("test_field_1").field("test_field_1") + ) + val compositeAgg = CompositeAggregationBuilder("composite_agg", compositeSources) + val input = SearchInput(indices = listOf(index), query = SearchSourceBuilder().size(0).query(query).aggregation(compositeAgg)) + // Bucket level monitor will reduce the size of matched doc ids on those that belong + // to a bucket that contains more than 1 document after term grouping + val triggerScript = """ + params.docCount > 1 + """.trimIndent() + + var trigger = randomBucketLevelTrigger() + trigger = trigger.copy( + bucketSelector = BucketSelectorExtAggregationBuilder( + name = trigger.id, + bucketsPathsMap = mapOf("docCount" to "_count"), + script = Script(triggerScript), + parentBucketPath = "composite_agg", + filter = null, + ) + ) + val bucketCustomAlertsIndex = "custom_alerts_index" + val bucketCustomFindingsIndex = "custom_findings_index" + val bucketCustomFindingsIndexPattern = "custom_findings_index-1" + + val bucketLevelMonitorResponse = createMonitor( + randomBucketLevelMonitor( + inputs = listOf(input), + enabled = false, + triggers = listOf(trigger), + dataSources = DataSources( + findingsEnabled = true, + alertsIndex = bucketCustomAlertsIndex, + findingsIndex = bucketCustomFindingsIndex, + findingsIndexPattern = bucketCustomFindingsIndexPattern + ) + ) + )!! + + val docQuery1 = DocLevelQuery(query = "test_field_1:\"test_value_2\"", name = "1") + val docQuery2 = DocLevelQuery(query = "test_field_1:\"test_value_1\"", name = "2") + val docQuery3 = DocLevelQuery(query = "test_field_1:\"test_value_3\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(index), listOf(docQuery1, docQuery2, docQuery3)) + val docTrigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val docCustomAlertsIndex = "custom_alerts_index" + val docCustomFindingsIndex = "custom_findings_index" + val docCustomFindingsIndexPattern = "custom_findings_index-1" + var docLevelMonitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(docTrigger), + dataSources = DataSources( + alertsIndex = docCustomAlertsIndex, + findingsIndex = docCustomFindingsIndex, + findingsIndexPattern = docCustomFindingsIndexPattern + ) + ) + + val docLevelMonitorResponse = createMonitor(docLevelMonitor)!! + // 1. bucketMonitor (chainedFinding = null) 2. docMonitor (chainedFinding = bucketMonitor) + var workflow = randomWorkflow( + monitorIds = listOf(bucketLevelMonitorResponse.id, docLevelMonitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + // Creates 5 documents + insertSampleTimeSerializedData( + index, + listOf( + "test_value_1", + "test_value_1", // adding duplicate to verify aggregation + "test_value_2", + "test_value_2", + "test_value_3" + ) + ) + + val workflowId = workflowResponse.id + // 1. bucket level monitor should reduce the doc findings to 4 (1, 2, 3, 4) + // 2. Doc level monitor will match those 4 documents although it contains rules for matching all 5 documents (docQuery3 matches the fifth) + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + assertNotNull(executeWorkflowResponse) + + for (monitorRunResults in executeWorkflowResponse.workflowRunResult.workflowRunResult) { + if (bucketLevelMonitorResponse.monitor.name == monitorRunResults.monitorName) { + val searchResult = monitorRunResults.inputResults.results.first() + + @Suppress("UNCHECKED_CAST") + val buckets = searchResult.stringMap("aggregations")?.stringMap("composite_agg")?.get("buckets") as List> + assertEquals("Incorrect search result", 3, buckets.size) + + val getAlertsResponse = assertAlerts(bucketLevelMonitorResponse.id, bucketCustomAlertsIndex, 2) + assertAcknowledges(getAlertsResponse.alerts, bucketLevelMonitorResponse.id, 2) + assertFindings(bucketLevelMonitorResponse.id, bucketCustomFindingsIndex, 1, 4, listOf("1", "2", "3", "4")) + } else { + assertEquals(1, monitorRunResults.inputResults.results.size) + val values = monitorRunResults.triggerResults.values + assertEquals(1, values.size) + @Suppress("UNCHECKED_CAST") + val docLevelTrigger = values.iterator().next() as DocumentLevelTriggerRunResult + val triggeredDocIds = docLevelTrigger.triggeredDocs.map { it.split("|")[0] } + val expectedTriggeredDocIds = listOf("1", "2", "3", "4") + assertEquals(expectedTriggeredDocIds, triggeredDocIds.sorted()) + + val getAlertsResponse = assertAlerts(docLevelMonitorResponse.id, docCustomAlertsIndex, 4) + assertAcknowledges(getAlertsResponse.alerts, docLevelMonitorResponse.id, 4) + assertFindings(docLevelMonitorResponse.id, docCustomFindingsIndex, 4, 4, listOf("1", "2", "3", "4")) + } + } + } + + fun `test execute workflow with custom alerts and finding index when doc level delegate is used in chained finding`() { + val docQuery1 = DocLevelQuery(query = "test_field_1:\"test_value_2\"", name = "1") + val docQuery2 = DocLevelQuery(query = "test_field_1:\"test_value_3\"", name = "2") + + var docLevelMonitor = randomDocumentLevelMonitor( + inputs = listOf(DocLevelMonitorInput("description", listOf(index), listOf(docQuery1, docQuery2))), + triggers = listOf(randomDocumentLevelTrigger(condition = ALWAYS_RUN)), + dataSources = DataSources( + alertsIndex = "custom_alerts_index", + findingsIndex = "custom_findings_index", + findingsIndexPattern = "custom_findings_index-1" + ) + ) + + val docLevelMonitorResponse = createMonitor(docLevelMonitor)!! + + val query = QueryBuilders.rangeQuery("test_strict_date_time") + .gt("{{period_end}}||-10d") + .lte("{{period_end}}") + .format("epoch_millis") + val compositeSources = listOf( + TermsValuesSourceBuilder("test_field_1").field("test_field_1") + ) + val compositeAgg = CompositeAggregationBuilder("composite_agg", compositeSources) + val input = SearchInput(indices = listOf(index), query = SearchSourceBuilder().size(0).query(query).aggregation(compositeAgg)) + // Bucket level monitor will reduce the size of matched doc ids on those that belong to a bucket that contains more than 1 document after term grouping + val triggerScript = """ + params.docCount > 1 + """.trimIndent() + + var trigger = randomBucketLevelTrigger() + trigger = trigger.copy( + bucketSelector = BucketSelectorExtAggregationBuilder( + name = trigger.id, + bucketsPathsMap = mapOf("docCount" to "_count"), + script = Script(triggerScript), + parentBucketPath = "composite_agg", + filter = null, + ) + ) + + val bucketLevelMonitorResponse = createMonitor( + randomBucketLevelMonitor( + inputs = listOf(input), + enabled = false, + triggers = listOf(trigger), + dataSources = DataSources( + findingsEnabled = true, + alertsIndex = "custom_alerts_index", + findingsIndex = "custom_findings_index", + findingsIndexPattern = "custom_findings_index-1" + ) + ) + )!! + + var docLevelMonitor1 = randomDocumentLevelMonitor( + // Match the documents with test_field_1: test_value_3 + inputs = listOf(DocLevelMonitorInput("description", listOf(index), listOf(docQuery2))), + triggers = listOf(randomDocumentLevelTrigger(condition = ALWAYS_RUN)), + dataSources = DataSources( + findingsEnabled = true, + alertsIndex = "custom_alerts_index_1", + findingsIndex = "custom_findings_index_1", + findingsIndexPattern = "custom_findings_index_1-1" + ) + ) + + val docLevelMonitorResponse1 = createMonitor(docLevelMonitor1)!! + + val queryMonitorInput = SearchInput( + indices = listOf(index), + query = SearchSourceBuilder().query( + QueryBuilders + .rangeQuery("test_strict_date_time") + .gt("{{period_end}}||-10d") + .lte("{{period_end}}") + .format("epoch_millis") + ) + ) + val queryTriggerScript = """ + return ctx.results[0].hits.hits.size() > 0 + """.trimIndent() + + val queryLevelTrigger = randomQueryLevelTrigger(condition = Script(queryTriggerScript)) + val queryMonitorResponse = + createMonitor(randomQueryLevelMonitor(inputs = listOf(queryMonitorInput), triggers = listOf(queryLevelTrigger)))!! + + // 1. docMonitor (chainedFinding = null) 2. bucketMonitor (chainedFinding = docMonitor) 3. docMonitor (chainedFinding = bucketMonitor) 4. queryMonitor (chainedFinding = docMonitor 3) + var workflow = randomWorkflow( + monitorIds = listOf( + docLevelMonitorResponse.id, + bucketLevelMonitorResponse.id, + docLevelMonitorResponse1.id, + queryMonitorResponse.id + ) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + // Creates 5 documents + insertSampleTimeSerializedData( + index, + listOf( + "test_value_1", + "test_value_1", // adding duplicate to verify aggregation + "test_value_2", + "test_value_2", + "test_value_3", + "test_value_3" + ) + ) + + val workflowId = workflowResponse.id + // 1. Doc level monitor should reduce the doc findings to 4 (3 - test_value_2, 4 - test_value_2, 5 - test_value_3, 6 - test_value_3) + // 2. Bucket level monitor will match the fetch the docs from current findings execution, although it contains rules for matching documents which has test_value_2 and test value_3 + val executeWorkflowResponse = executeWorkflow(workflowById, workflowId, false)!! + assertNotNull(executeWorkflowResponse) + + for (monitorRunResults in executeWorkflowResponse.workflowRunResult.workflowRunResult) { + when (monitorRunResults.monitorName) { + // Verify first doc level monitor execution, alerts and findings + docLevelMonitorResponse.monitor.name -> { + assertEquals(1, monitorRunResults.inputResults.results.size) + val values = monitorRunResults.triggerResults.values + assertEquals(1, values.size) + @Suppress("UNCHECKED_CAST") + val docLevelTrigger = values.iterator().next() as DocumentLevelTriggerRunResult + val triggeredDocIds = docLevelTrigger.triggeredDocs.map { it.split("|")[0] } + val expectedTriggeredDocIds = listOf("3", "4", "5", "6") + assertEquals(expectedTriggeredDocIds, triggeredDocIds.sorted()) + + val getAlertsResponse = + assertAlerts(docLevelMonitorResponse.id, docLevelMonitorResponse.monitor.dataSources.alertsIndex, 4) + assertAcknowledges(getAlertsResponse.alerts, docLevelMonitorResponse.id, 4) + assertFindings( + docLevelMonitorResponse.id, + docLevelMonitorResponse.monitor.dataSources.findingsIndex, + 4, + 4, + listOf("3", "4", "5", "6") + ) + } + // Verify second bucket level monitor execution, alerts and findings + bucketLevelMonitorResponse.monitor.name -> { + val searchResult = monitorRunResults.inputResults.results.first() + + @Suppress("UNCHECKED_CAST") + val buckets = + searchResult + .stringMap("aggregations")?.stringMap("composite_agg")?.get("buckets") as List> + assertEquals("Incorrect search result", 2, buckets.size) + + val getAlertsResponse = + assertAlerts(bucketLevelMonitorResponse.id, bucketLevelMonitorResponse.monitor.dataSources.alertsIndex, 2) + assertAcknowledges(getAlertsResponse.alerts, bucketLevelMonitorResponse.id, 2) + assertFindings( + bucketLevelMonitorResponse.id, + bucketLevelMonitorResponse.monitor.dataSources.findingsIndex, + 1, + 4, + listOf("3", "4", "5", "6") + ) + } + // Verify third doc level monitor execution, alerts and findings + docLevelMonitorResponse1.monitor.name -> { + assertEquals(1, monitorRunResults.inputResults.results.size) + val values = monitorRunResults.triggerResults.values + assertEquals(1, values.size) + @Suppress("UNCHECKED_CAST") + val docLevelTrigger = values.iterator().next() as DocumentLevelTriggerRunResult + val triggeredDocIds = docLevelTrigger.triggeredDocs.map { it.split("|")[0] } + val expectedTriggeredDocIds = listOf("5", "6") + assertEquals(expectedTriggeredDocIds, triggeredDocIds.sorted()) + + val getAlertsResponse = + assertAlerts(docLevelMonitorResponse1.id, docLevelMonitorResponse1.monitor.dataSources.alertsIndex, 2) + assertAcknowledges(getAlertsResponse.alerts, docLevelMonitorResponse1.id, 2) + assertFindings( + docLevelMonitorResponse1.id, + docLevelMonitorResponse1.monitor.dataSources.findingsIndex, + 2, + 2, + listOf("5", "6") + ) + } + // Verify fourth query level monitor execution + queryMonitorResponse.monitor.name -> { + assertEquals(1, monitorRunResults.inputResults.results.size) + val values = monitorRunResults.triggerResults.values + assertEquals(1, values.size) + @Suppress("UNCHECKED_CAST") + val totalHits = + ((monitorRunResults.inputResults.results[0]["hits"] as Map)["total"] as Map)["value"] + assertEquals(2, totalHits) + @Suppress("UNCHECKED_CAST") + val docIds = + ( + (monitorRunResults.inputResults.results[0]["hits"] as Map)["hits"] as List> + ).map { it["_id"]!! } + assertEquals(listOf("5", "6"), docIds.sorted()) + } + } + } + } + + fun `test execute workflow input error`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + var workflow = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflow)!! + val workflowById = searchWorkflow(workflowResponse.id) + assertNotNull(workflowById) + + deleteIndex(index) + + val response = executeWorkflow(workflowById, workflowById!!.id, false)!! + val error = response.workflowRunResult.workflowRunResult[0].error + assertNotNull(error) + assertTrue(error is AlertingException) + assertEquals(RestStatus.NOT_FOUND, (error as AlertingException).status) + assertEquals("Configured indices are not found: [$index]", error.message) + } + + fun `test execute workflow wrong workflow id`() { + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(DocLevelQuery(query = "source.ip.v6.v1:12345", name = "3")) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + + val monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + + val monitorResponse = createMonitor(monitor)!! + + val workflowRequest = randomWorkflow( + monitorIds = listOf(monitorResponse.id) + ) + val workflowResponse = upsertWorkflow(workflowRequest)!! + val workflowId = workflowResponse.id + val getWorkflowResponse = getWorkflowById(id = workflowResponse.id) + + assertNotNull(getWorkflowResponse) + assertEquals(workflowId, getWorkflowResponse.id) + + var exception: Exception? = null + val badWorkflowId = getWorkflowResponse.id + "bad" + try { + executeWorkflow(id = badWorkflowId) + } catch (ex: Exception) { + exception = ex + } + assertTrue(exception is ExecutionException) + assertTrue(exception!!.cause is AlertingException) + assertEquals(RestStatus.NOT_FOUND, (exception.cause as AlertingException).status) + assertEquals("Can't find workflow with id: $badWorkflowId", exception.cause!!.message) + } + + private fun assertFindings( + monitorId: String, + customFindingsIndex: String, + findingSize: Int, + matchedQueryNumber: Int, + relatedDocIds: List, + ) { + val findings = searchFindings(monitorId, customFindingsIndex) + assertEquals("Findings saved for test monitor", findingSize, findings.size) + + val findingDocIds = findings.flatMap { it.relatedDocIds } + + assertEquals("Didn't match $matchedQueryNumber query", matchedQueryNumber, findingDocIds.size) + assertTrue("Findings saved for test monitor", relatedDocIds.containsAll(findingDocIds)) + } + + private fun assertAlerts( + monitorId: String, + customAlertsIndex: String, + alertSize: Int, + ): GetAlertsResponse { + val alerts = searchAlerts(monitorId, customAlertsIndex) + assertEquals("Alert saved for test monitor", alertSize, alerts.size) + val table = Table("asc", "id", null, alertSize, 0, "") + var getAlertsResponse = client() + .execute( + AlertingActions.GET_ALERTS_ACTION_TYPE, + GetAlertsRequest(table, "ALL", "ALL", null, customAlertsIndex) + ) + .get() + assertTrue(getAlertsResponse != null) + assertTrue(getAlertsResponse.alerts.size == alertSize) + getAlertsResponse = client() + .execute(AlertingActions.GET_ALERTS_ACTION_TYPE, GetAlertsRequest(table, "ALL", "ALL", monitorId, null)) + .get() + assertTrue(getAlertsResponse != null) + assertTrue(getAlertsResponse.alerts.size == alertSize) + + return getAlertsResponse + } + + private fun assertAcknowledges( + alerts: List, + monitorId: String, + alertSize: Int, + ) { + val alertIds = alerts.map { it.id } + val acknowledgeAlertResponse = client().execute( + AlertingActions.ACKNOWLEDGE_ALERTS_ACTION_TYPE, + AcknowledgeAlertRequest(monitorId, alertIds, WriteRequest.RefreshPolicy.IMMEDIATE) + ).get() + + assertEquals(alertSize, acknowledgeAlertResponse.acknowledged.size) + } + + fun `test execute workflow with bucket-level and doc-level chained monitors`() { + createTestIndex(TEST_HR_INDEX) + + val compositeSources = listOf( + TermsValuesSourceBuilder("test_field").field("test_field") + ) + val compositeAgg = CompositeAggregationBuilder("composite_agg", compositeSources) + val input = SearchInput( + indices = listOf(TEST_HR_INDEX), + query = SearchSourceBuilder().size(0).query(QueryBuilders.matchAllQuery()).aggregation(compositeAgg) + ) + val triggerScript = """ + params.docCount > 0 + """.trimIndent() + + var trigger = randomBucketLevelTrigger() + trigger = trigger.copy( + bucketSelector = BucketSelectorExtAggregationBuilder( + name = trigger.id, + bucketsPathsMap = mapOf("docCount" to "_count"), + script = Script(triggerScript), + parentBucketPath = "composite_agg", + filter = null + ), + actions = listOf() + ) + val bucketMonitor = createMonitor( + randomBucketLevelMonitor( + inputs = listOf(input), + enabled = false, + triggers = listOf(trigger) + ) + ) + assertNotNull("The bucket monitor was not created", bucketMonitor) + + val docQuery1 = DocLevelQuery(query = "test_field:\"a\"", name = "3") + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(DocLevelMonitorInput("description", listOf(TEST_HR_INDEX), listOf(docQuery1))), + triggers = listOf(randomDocumentLevelTrigger(condition = ALWAYS_RUN)) + ) + val docMonitor = createMonitor(monitor1)!! + assertNotNull("The doc level monitor was not created", docMonitor) + + val workflow = randomWorkflow(monitorIds = listOf(bucketMonitor!!.id, docMonitor.id)) + val workflowResponse = upsertWorkflow(workflow) + assertNotNull("The workflow was not created", workflowResponse) + + // Add a doc that is accessible to the user + indexDoc( + TEST_HR_INDEX, + "1", + """ + { + "test_field": "a", + "accessible": true + } + """.trimIndent() + ) + + // Add a second doc that is not accessible to the user + indexDoc( + TEST_HR_INDEX, + "2", + """ + { + "test_field": "b", + "accessible": false + } + """.trimIndent() + ) + + indexDoc( + TEST_HR_INDEX, + "3", + """ + { + "test_field": "c", + "accessible": true + } + """.trimIndent() + ) + + val executeResult = executeWorkflow(id = workflowResponse!!.id) + assertNotNull(executeResult) + assertEquals(2, executeResult!!.workflowRunResult.workflowRunResult.size) + } +} diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt index 6416b8c69..0359f8e03 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt @@ -841,7 +841,9 @@ class MonitorRestApiIT : AlertingRestTestCase() { refreshIndex("*") val updatedMonitor = monitor.copy(triggers = emptyList()) val updateResponse = client().makeRequest( - "PUT", "$ALERTING_BASE_URI/${monitor.id}", emptyMap(), + "PUT", + "$ALERTING_BASE_URI/${monitor.id}", + emptyMap(), updatedMonitor.toHttpEntity() ) assertEquals("Update request not successful", RestStatus.OK, updateResponse.restStatus()) diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt b/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt index 7bfbcf4ac..ac2739b66 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt @@ -7,6 +7,7 @@ package org.opensearch.alerting.transport import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest import org.opensearch.action.admin.indices.get.GetIndexRequest import org.opensearch.action.admin.indices.get.GetIndexRequestBuilder import org.opensearch.action.admin.indices.get.GetIndexResponse @@ -23,6 +24,7 @@ import org.opensearch.alerting.action.GetMonitorRequest import org.opensearch.alerting.alerts.AlertIndices import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue +import org.opensearch.common.xcontent.XContentFactory import org.opensearch.common.xcontent.XContentType import org.opensearch.common.xcontent.json.JsonXContent import org.opensearch.commons.alerting.action.AlertingActions @@ -35,16 +37,23 @@ import org.opensearch.commons.alerting.model.Alert import org.opensearch.commons.alerting.model.Finding import org.opensearch.commons.alerting.model.Monitor import org.opensearch.commons.alerting.model.Table +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.index.IndexService import org.opensearch.index.query.TermQueryBuilder import org.opensearch.index.reindex.ReindexPlugin import org.opensearch.index.seqno.SequenceNumbers import org.opensearch.join.ParentJoinPlugin +import org.opensearch.painless.PainlessPlugin import org.opensearch.plugins.Plugin import org.opensearch.rest.RestRequest +import org.opensearch.script.mustache.MustachePlugin import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.fetch.subphase.FetchSourceContext import org.opensearch.test.OpenSearchSingleNodeTestCase import java.time.Instant +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter +import java.time.temporal.ChronoUnit import java.util.Locale /** @@ -75,19 +84,79 @@ abstract class AlertingSingleNodeTestCase : OpenSearchSingleNodeTestCase() { return client().execute(ExecuteMonitorAction.INSTANCE, request).get() } + protected fun insertSampleTimeSerializedData(index: String, data: List) { + data.forEachIndexed { i, value -> + val twoMinsAgo = ZonedDateTime.now().minus(2, ChronoUnit.MINUTES).truncatedTo(ChronoUnit.MILLIS) + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(twoMinsAgo) + val testDoc = """ + { + "test_strict_date_time": "$testTime", + "test_field_1": "$value", + "number": "$i" + } + """.trimIndent() + // Indexing documents with deterministic doc id to allow for easy selected deletion during testing + indexDoc(index, (i + 1).toString(), testDoc) + } + } + + @Suppress("UNCHECKED_CAST") + fun Map.stringMap(key: String): Map? { + val map = this as Map> + return map[key] + } + /** A test index that can be used across tests. Feel free to add new fields but don't remove any. */ protected fun createTestIndex() { + val mapping = XContentFactory.jsonBuilder() + mapping.startObject() + .startObject("properties") + .startObject("test_strict_date_time") + .field("type", "date") + .field("format", "strict_date_time") + .endObject() + .startObject("test_field_1") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + createIndex( - index, Settings.EMPTY, - """ - "properties" : { - "test_strict_date_time" : { "type" : "date", "format" : "strict_date_time" }, - "test_field" : { "type" : "keyword" } - } - """.trimIndent() + index, Settings.EMPTY, mapping ) } + protected fun createTestIndex(index: String) { + val mapping = XContentFactory.jsonBuilder() + mapping.startObject() + .startObject("properties") + .startObject("test_strict_date_time") + .field("type", "date") + .field("format", "strict_date_time") + .endObject() + .startObject("test_field_1") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + + createIndex( + index, Settings.EMPTY, mapping + ) + } + + private fun createIndex( + index: String?, + settings: Settings?, + mappings: XContentBuilder?, + ): IndexService? { + val createIndexRequestBuilder = client().admin().indices().prepareCreate(index).setSettings(settings) + if (mappings != null) { + createIndexRequestBuilder.setMapping(mappings) + } + return this.createIndex(index, createIndexRequestBuilder) + } + protected fun indexDoc(index: String, id: String, doc: String) { client().prepareIndex(index).setId(id) .setSource(doc, XContentType.JSON).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get() @@ -230,7 +299,18 @@ abstract class AlertingSingleNodeTestCase : OpenSearchSingleNodeTestCase() { ).get() override fun getPlugins(): List> { - return listOf(AlertingPlugin::class.java, ReindexPlugin::class.java, ParentJoinPlugin::class.java) + return listOf( + AlertingPlugin::class.java, + ReindexPlugin::class.java, + MustachePlugin::class.java, + PainlessPlugin::class.java, + ParentJoinPlugin::class.java + ) + } + + protected fun deleteIndex(index: String) { + val response = client().admin().indices().delete(DeleteIndexRequest(index)).get() + assertTrue("Unable to delete index", response.isAcknowledged()) } override fun resetNodeAfterTest(): Boolean { diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/transport/WorkflowSingleNodeTestCase.kt b/alerting/src/test/kotlin/org/opensearch/alerting/transport/WorkflowSingleNodeTestCase.kt new file mode 100644 index 000000000..699a3d565 --- /dev/null +++ b/alerting/src/test/kotlin/org/opensearch/alerting/transport/WorkflowSingleNodeTestCase.kt @@ -0,0 +1,161 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.transport + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope +import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.action.ExecuteWorkflowAction +import org.opensearch.alerting.action.ExecuteWorkflowRequest +import org.opensearch.alerting.action.ExecuteWorkflowResponse +import org.opensearch.alerting.model.MonitorMetadata +import org.opensearch.alerting.model.WorkflowMetadata +import org.opensearch.common.unit.TimeValue +import org.opensearch.common.xcontent.json.JsonXContent +import org.opensearch.commons.alerting.action.AlertingActions +import org.opensearch.commons.alerting.action.DeleteWorkflowRequest +import org.opensearch.commons.alerting.action.GetWorkflowRequest +import org.opensearch.commons.alerting.action.GetWorkflowResponse +import org.opensearch.commons.alerting.action.IndexWorkflowRequest +import org.opensearch.commons.alerting.action.IndexWorkflowResponse +import org.opensearch.commons.alerting.model.ScheduledJob +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.core.xcontent.XContentParser +import org.opensearch.index.query.TermQueryBuilder +import org.opensearch.index.seqno.SequenceNumbers +import org.opensearch.rest.RestRequest +import org.opensearch.search.builder.SearchSourceBuilder +import java.time.Instant + +/** + * A test that keep a singleton node started for all tests that can be used to get + * references to Guice injectors in unit tests. + */ + +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) +abstract class WorkflowSingleNodeTestCase : AlertingSingleNodeTestCase() { + + protected fun searchWorkflow( + id: String, + indices: String = ScheduledJob.SCHEDULED_JOBS_INDEX, + refresh: Boolean = true, + ): Workflow? { + try { + if (refresh) refreshIndex(indices) + } catch (e: Exception) { + logger.warn("Could not refresh index $indices because: ${e.message}") + return null + } + val ssb = SearchSourceBuilder() + ssb.version(true) + ssb.query(TermQueryBuilder("_id", id)) + val searchResponse = client().prepareSearch(indices).setRouting(id).setSource(ssb).get() + + return searchResponse.hits.hits.map { it -> + val xcp = createParser(JsonXContent.jsonXContent, it.sourceRef).also { it.nextToken() } + lateinit var workflow: Workflow + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + xcp.nextToken() + when (xcp.currentName()) { + "workflow" -> workflow = Workflow.parse(xcp) + } + } + workflow.copy(id = it.id, version = it.version) + }.first() + } + + protected fun searchWorkflowMetadata( + id: String, + indices: String = ScheduledJob.SCHEDULED_JOBS_INDEX, + refresh: Boolean = true, + ): WorkflowMetadata? { + try { + if (refresh) refreshIndex(indices) + } catch (e: Exception) { + logger.warn("Could not refresh index $indices because: ${e.message}") + return null + } + val ssb = SearchSourceBuilder() + ssb.version(true) + ssb.query(TermQueryBuilder("workflow_metadata.workflow_id", id)) + val searchResponse = client().prepareSearch(indices).setRouting(id).setSource(ssb).get() + + return searchResponse.hits.hits.map { it -> + val xcp = createParser(JsonXContent.jsonXContent, it.sourceRef).also { it.nextToken() } + lateinit var workflowMetadata: WorkflowMetadata + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + xcp.nextToken() + when (xcp.currentName()) { + "workflow_metadata" -> workflowMetadata = WorkflowMetadata.parse(xcp) + } + } + workflowMetadata.copy(id = it.id) + }.first() + } + + protected fun searchMonitorMetadata( + id: String, + indices: String = ScheduledJob.SCHEDULED_JOBS_INDEX, + refresh: Boolean = true, + ): MonitorMetadata? { + try { + if (refresh) refreshIndex(indices) + } catch (e: Exception) { + logger.warn("Could not refresh index $indices because: ${e.message}") + return null + } + val ssb = SearchSourceBuilder() + ssb.version(true) + ssb.query(TermQueryBuilder("_id", id)) + val searchResponse = client().prepareSearch(indices).setRouting(id).setSource(ssb).get() + + return searchResponse.hits.hits.map { it -> + val xcp = createParser(JsonXContent.jsonXContent, it.sourceRef).also { it.nextToken() } + lateinit var monitorMetadata: MonitorMetadata + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + xcp.nextToken() + when (xcp.currentName()) { + "metadata" -> monitorMetadata = MonitorMetadata.parse(xcp) + } + } + monitorMetadata.copy(id = it.id) + }.first() + } + + protected fun upsertWorkflow( + workflow: Workflow, + id: String = Workflow.NO_ID, + method: RestRequest.Method = RestRequest.Method.POST, + ): IndexWorkflowResponse? { + val request = IndexWorkflowRequest( + workflowId = id, + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, + refreshPolicy = WriteRequest.RefreshPolicy.parse("true"), + method = method, + workflow = workflow + ) + + return client().execute(AlertingActions.INDEX_WORKFLOW_ACTION_TYPE, request).actionGet() + } + protected fun getWorkflowById(id: String): GetWorkflowResponse { + return client().execute( + AlertingActions.GET_WORKFLOW_ACTION_TYPE, + GetWorkflowRequest(id, RestRequest.Method.GET) + ).get() + } + + protected fun deleteWorkflow(workflowId: String, deleteDelegateMonitors: Boolean? = null) { + client().execute( + AlertingActions.DELETE_WORKFLOW_ACTION_TYPE, + DeleteWorkflowRequest(workflowId, deleteDelegateMonitors) + ).get() + } + + protected fun executeWorkflow(workflow: Workflow? = null, id: String? = null, dryRun: Boolean = true): ExecuteWorkflowResponse? { + val request = ExecuteWorkflowRequest(dryRun, TimeValue(Instant.now().toEpochMilli()), id, workflow) + return client().execute(ExecuteWorkflowAction.INSTANCE, request).get() + } +} diff --git a/core/src/main/resources/mappings/scheduled-jobs.json b/core/src/main/resources/mappings/scheduled-jobs.json index 4f8d71d82..2e844d300 100644 --- a/core/src/main/resources/mappings/scheduled-jobs.json +++ b/core/src/main/resources/mappings/scheduled-jobs.json @@ -1,6 +1,6 @@ { "_meta" : { - "schema_version": 6 + "schema_version": 7 }, "properties": { "monitor": { @@ -299,6 +299,151 @@ } } }, + "workflow": { + "dynamic": "false", + "properties": { + "schema_version": { + "type": "integer" + }, + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "owner": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "workflow_type": { + "type": "keyword" + }, + "user": { + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "backend_roles": { + "type" : "text", + "fields" : { + "keyword" : { + "type" : "keyword" + } + } + }, + "roles": { + "type" : "text", + "fields" : { + "keyword" : { + "type" : "keyword" + } + } + }, + "custom_attribute_names": { + "type" : "text", + "fields" : { + "keyword" : { + "type" : "keyword" + } + } + } + } + }, + "type": { + "type": "keyword" + }, + "enabled": { + "type": "boolean" + }, + "enabled_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "last_update_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "schedule": { + "properties": { + "period": { + "properties": { + "interval": { + "type": "integer" + }, + "unit": { + "type": "keyword" + } + } + }, + "cron": { + "properties": { + "expression": { + "type": "text" + }, + "timezone": { + "type": "keyword" + } + } + } + } + }, + "inputs": { + "type": "nested", + "properties": { + "composite_input": { + "type": "nested", + "properties": { + "sequence": { + "properties": { + "delegates": { + "type": "nested", + "properties": { + "order": { + "type": "integer" + }, + "monitor_id": { + "type": "keyword" + }, + "chained_monitor_findings": { + "properties": { + "monitor_id": { + "type": "keyword" + } + } + } + } + } + } + } + } + } + } + }, + "group_by_fields": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + } + } + }, "destination": { "dynamic": "false", "properties": { @@ -518,6 +663,29 @@ "enabled": false } } + }, + "workflow_metadata" : { + "properties": { + "workflow_id": { + "type": "keyword" + }, + "monitor_ids": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 1000 + } + } + }, + "latest_run_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "latest_execution_id": { + "type": "keyword" + } + } } } }