From d84bf82091704daee015f4dc2fb57418bad488f7 Mon Sep 17 00:00:00 2001 From: Surya Sashank Nistala Date: Wed, 21 Feb 2024 10:10:18 -0800 Subject: [PATCH] Add jvm aware setting and max num docs settings for batching docs for percolate queries (#1435) * add jvm aware and max docs settings for batching docs for percolate queries Signed-off-by: Surya Sashank Nistala * fix stats logging Signed-off-by: Surya Sashank Nistala * add queryfieldnames field in findings mapping Signed-off-by: Surya Sashank Nistala --------- Signed-off-by: Surya Sashank Nistala --- .../org/opensearch/alerting/AlertingPlugin.kt | 26 +- .../alerting/DocumentLevelMonitorRunner.kt | 331 +++++++++++---- .../alerting/MonitorRunnerExecutionContext.kt | 3 + .../alerting/MonitorRunnerService.kt | 35 +- .../alerting/settings/AlertingSettings.kt | 23 + .../workflow/CompositeWorkflowRunner.kt | 392 ++++++++++++++++++ .../alerting/alerts/finding_mapping.json | 11 +- 7 files changed, 727 insertions(+), 94 deletions(-) create mode 100644 alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt index 1c92069da..555afe570 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt @@ -6,15 +6,14 @@ 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 -import org.opensearch.alerting.action.GetMonitorAction +import org.opensearch.alerting.action.GetRemoteIndexesAction import org.opensearch.alerting.action.SearchEmailAccountAction import org.opensearch.alerting.action.SearchEmailGroupAction -import org.opensearch.alerting.action.SearchMonitorAction import org.opensearch.alerting.alerts.AlertIndices import org.opensearch.alerting.core.JobSweeper import org.opensearch.alerting.core.ScheduledJobIndices @@ -33,25 +32,37 @@ import org.opensearch.alerting.resthandler.RestGetEmailAccountAction import org.opensearch.alerting.resthandler.RestGetEmailGroupAction import org.opensearch.alerting.resthandler.RestGetFindingsAction import org.opensearch.alerting.resthandler.RestGetMonitorAction +import org.opensearch.alerting.resthandler.RestGetRemoteIndexesAction +import org.opensearch.alerting.resthandler.RestGetWorkflowAction +import org.opensearch.alerting.resthandler.RestGetWorkflowAlertsAction import org.opensearch.alerting.resthandler.RestIndexMonitorAction +import org.opensearch.alerting.resthandler.RestIndexWorkflowAction 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.TransportAcknowledgeChainedAlertAction 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.TransportGetRemoteIndexesAction +import org.opensearch.alerting.transport.TransportGetWorkflowAction +import org.opensearch.alerting.transport.TransportGetWorkflowAlertsAction 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 @@ -61,8 +72,6 @@ import org.opensearch.client.Client import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.node.DiscoveryNodes import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.io.stream.NamedWriteableRegistry -import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.ClusterSettings import org.opensearch.common.settings.IndexScopedSettings import org.opensearch.common.settings.Setting @@ -83,6 +92,9 @@ import org.opensearch.core.xcontent.XContentParser import org.opensearch.env.Environment import org.opensearch.env.NodeEnvironment import org.opensearch.index.IndexModule +import org.opensearch.monitor.jvm.JvmStats +import org.opensearch.painless.spi.Allowlist +import org.opensearch.painless.spi.AllowlistLoader import org.opensearch.painless.spi.PainlessExtension import org.opensearch.painless.spi.Whitelist import org.opensearch.painless.spi.WhitelistLoader @@ -237,6 +249,8 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R .registerTriggerService(TriggerService(scriptService)) .registerAlertService(AlertService(client, xContentRegistry, alertIndices)) .registerDocLevelMonitorQueries(DocLevelMonitorQueries(client, clusterService)) + .registerJvmStats(JvmStats.jvmStats()) + .registerWorkflowService(WorkflowService(client, xContentRegistry)) .registerConsumers() .registerDestinationSettings() scheduledJobIndices = ScheduledJobIndices(client.admin(), clusterService) @@ -284,6 +298,8 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R AlertingSettings.ALERT_HISTORY_MAX_DOCS, AlertingSettings.ALERT_HISTORY_RETENTION_PERIOD, AlertingSettings.ALERTING_MAX_MONITORS, + AlertingSettings.PERCOLATE_QUERY_DOCS_SIZE_MEMORY_PERCENTAGE_LIMIT, + AlertingSettings.PERCOLATE_QUERY_MAX_NUM_DOCS_IN_MEMORY, AlertingSettings.REQUEST_TIMEOUT, AlertingSettings.MAX_ACTION_THROTTLE_VALUE, AlertingSettings.FILTER_BY_BACKEND_ROLES, diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt index 43e6eb566..2f0ea35ec 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt @@ -25,6 +25,8 @@ import org.opensearch.alerting.model.MonitorRunResult import org.opensearch.alerting.model.userErrorMessage import org.opensearch.alerting.opensearchapi.suspendUntil import org.opensearch.alerting.script.DocumentLevelTriggerExecutionContext +import org.opensearch.alerting.settings.AlertingSettings.Companion.PERCOLATE_QUERY_DOCS_SIZE_MEMORY_PERCENTAGE_LIMIT +import org.opensearch.alerting.settings.AlertingSettings.Companion.PERCOLATE_QUERY_MAX_NUM_DOCS_IN_MEMORY import org.opensearch.alerting.util.AlertingException import org.opensearch.alerting.util.IndexUtils import org.opensearch.alerting.util.defaultToPerExecutionAction @@ -60,16 +62,27 @@ import org.opensearch.index.query.BoolQueryBuilder import org.opensearch.index.query.Operator import org.opensearch.index.query.QueryBuilders import org.opensearch.percolator.PercolateQueryBuilderExt +import org.opensearch.search.SearchHit import org.opensearch.search.SearchHits import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.sort.SortOrder import java.io.IOException import java.time.Instant import java.util.UUID +import java.util.stream.Collectors import kotlin.math.max -object DocumentLevelMonitorRunner : MonitorRunner() { +class DocumentLevelMonitorRunner : MonitorRunner() { private val logger = LogManager.getLogger(javaClass) + var nonPercolateSearchesTimeTakenStat = 0L + var percolateQueriesTimeTakenStat = 0L + var totalDocsQueriedStat = 0L + var docTransformTimeTakenStat = 0L + var totalDocsSizeInBytesStat = 0L + var docsSizeOfBatchInBytes = 0L + /* Contains list of docs source that are held in memory to submit to percolate query against query index. + * Docs are fetched from the source index per shard and transformed.*/ + val transformedDocs = mutableListOf>() override suspend fun runMonitor( monitor: Monitor, @@ -152,6 +165,8 @@ object DocumentLevelMonitorRunner : MonitorRunner() { // Map of document ids per index when monitor is workflow delegate and has chained findings val matchingDocIdsPerIndex = workflowRunContext?.matchingDocIdsPerIndex + val concreteIndicesSeenSoFar = mutableListOf() + val updatedIndexNames = mutableListOf() docLevelMonitorInput.indices.forEach { indexName -> var concreteIndices = IndexUtils.resolveAllIndices( listOf(indexName), @@ -173,7 +188,9 @@ object DocumentLevelMonitorRunner : MonitorRunner() { ) } } + concreteIndicesSeenSoFar.addAll(concreteIndices) val updatedIndexName = indexName.replace("*", "_") + updatedIndexNames.add(updatedIndexName) val conflictingFields = monitorCtx.docLevelMonitorQueries!!.getAllConflictingFields( monitorCtx.clusterService!!.state(), concreteIndices @@ -195,7 +212,7 @@ object DocumentLevelMonitorRunner : MonitorRunner() { val indexUpdatedRunContext = updateLastRunContext( indexLastRunContext.toMutableMap(), monitorCtx, - concreteIndexName + concreteIndexName, ) as MutableMap if (IndexUtils.isAlias(indexName, monitorCtx.clusterService!!.state()) || IndexUtils.isDataStream(indexName, monitorCtx.clusterService!!.state()) @@ -222,41 +239,35 @@ object DocumentLevelMonitorRunner : MonitorRunner() { // Prepare DocumentExecutionContext for each index val docExecutionContext = DocumentExecutionContext(queries, indexLastRunContext, indexUpdatedRunContext) - val matchingDocs = getMatchingDocs( + fetchShardDataAndMaybeExecutePercolateQueries( monitor, monitorCtx, docExecutionContext, updatedIndexName, concreteIndexName, conflictingFields.toList(), - matchingDocIdsPerIndex?.get(concreteIndexName) + matchingDocIdsPerIndex?.get(concreteIndexName), + monitorMetadata, + inputRunResults, + docsToQueries, + updatedIndexNames, + concreteIndicesSeenSoFar, ) - - if (matchingDocs.isNotEmpty()) { - val matchedQueriesForDocs = getMatchedQueries( - monitorCtx, - matchingDocs.map { it.second }, - monitor, - monitorMetadata, - updatedIndexName, - concreteIndexName - ) - - matchedQueriesForDocs.forEach { hit -> - val id = hit.id - .replace("_${updatedIndexName}_${monitor.id}", "") - .replace("_${concreteIndexName}_${monitor.id}", "") - - val docIndices = hit.field("_percolator_document_slot").values.map { it.toString().toInt() } - docIndices.forEach { idx -> - val docIndex = "${matchingDocs[idx].first}|$concreteIndexName" - inputRunResults.getOrPut(id) { mutableSetOf() }.add(docIndex) - docsToQueries.getOrPut(docIndex) { mutableListOf() }.add(id) - } - } - } } } + /* if all indices are covered still in-memory docs size limit is not breached we would need to submit + the percolate query at the end */ + if (transformedDocs.isNotEmpty()) { + performPercolateQueryAndResetCounters( + monitorCtx, + monitor, + monitorMetadata, + updatedIndexNames, + concreteIndicesSeenSoFar, + inputRunResults, + docsToQueries, + ) + } monitorResult = monitorResult.copy(inputResults = InputRunResults(listOf(inputRunResults))) /* @@ -327,6 +338,22 @@ object DocumentLevelMonitorRunner : MonitorRunner() { e ) return monitorResult.copy(error = alertingException, inputResults = InputRunResults(emptyList(), alertingException)) + } finally { + logger.debug( + "PERF_DEBUG_STATS: Monitor ${monitor.id} " + + "Time spent on fetching data from shards in millis: $nonPercolateSearchesTimeTakenStat" + ) + logger.debug( + "PERF_DEBUG_STATS: Monitor {} Time spent on percolate queries in millis: {}", + monitor.id, + percolateQueriesTimeTakenStat + ) + logger.debug( + "PERF_DEBUG_STATS: Monitor {} Time spent on transforming doc fields in millis: {}", + monitor.id, + docTransformTimeTakenStat + ) + logger.debug("PERF_DEBUG_STATS: Monitor {} Num docs queried: {}", monitor.id, totalDocsQueriedStat) } } @@ -567,7 +594,7 @@ object DocumentLevelMonitorRunner : MonitorRunner() { private suspend fun updateLastRunContext( lastRunContext: Map, monitorCtx: MonitorRunnerExecutionContext, - index: String + index: String, ): Map { val count: Int = getShardsCount(monitorCtx.clusterService!!, index) val updatedLastRunContext = lastRunContext.toMutableMap() @@ -626,6 +653,7 @@ object DocumentLevelMonitorRunner : MonitorRunner() { if (response.status() !== RestStatus.OK) { throw IOException("Failed to get max seq no for shard: $shard") } + nonPercolateSearchesTimeTakenStat += response.took.millis if (response.hits.hits.isEmpty()) { return -1L } @@ -638,17 +666,26 @@ object DocumentLevelMonitorRunner : MonitorRunner() { return allShards.filter { it.primary() }.size } - private suspend fun getMatchingDocs( + /** 1. Fetch data per shard for given index. (only 10000 docs are fetched. + * needs to be converted to scroll if not performant enough) + * 2. Transform documents to conform to format required for percolate query + * 3a. Check if docs in memory are crossing threshold defined by setting. + * 3b. If yes, perform percolate query and update docToQueries Map with all hits from percolate queries */ + private suspend fun fetchShardDataAndMaybeExecutePercolateQueries( monitor: Monitor, monitorCtx: MonitorRunnerExecutionContext, docExecutionCtx: DocumentExecutionContext, - index: String, - concreteIndex: String, + indexName: String, + concreteIndexName: String, conflictingFields: List, - docIds: List? = null - ): List> { + docIds: List? = null, + monitorMetadata: MonitorMetadata, + inputRunResults: MutableMap>, + docsToQueries: MutableMap>, + monitorInputIndices: List, + concreteIndices: List, + ) { val count: Int = docExecutionCtx.updatedLastRunContext["shards_count"] as Int - val matchingDocs = mutableListOf>() for (i: Int in 0 until count) { val shard = i.toString() try { @@ -657,24 +694,96 @@ object DocumentLevelMonitorRunner : MonitorRunner() { val hits: SearchHits = searchShard( monitorCtx, - concreteIndex, + concreteIndexName, shard, prevSeqNo, maxSeqNo, null, docIds ) + val startTime = System.currentTimeMillis() + transformedDocs.addAll( + transformSearchHitsAndReconstructDocs( + hits, + indexName, + concreteIndexName, + monitor.id, + conflictingFields, + ) + ) + docTransformTimeTakenStat += System.currentTimeMillis() - startTime + } catch (e: Exception) { + logger.error( + "Monitor ${monitor.id} :" + + " Failed to run fetch data from shard [$shard] of index [$concreteIndexName]. Error: ${e.message}", + e + ) + } + if ( + transformedDocs.isNotEmpty() && + shouldPerformPercolateQueryAndFlushInMemoryDocs(transformedDocs.size, monitorCtx) + ) { + performPercolateQueryAndResetCounters( + monitorCtx, + monitor, + monitorMetadata, + monitorInputIndices, + concreteIndices, + inputRunResults, + docsToQueries, + ) + } + } + } + + private fun shouldPerformPercolateQueryAndFlushInMemoryDocs( + numDocs: Int, + monitorCtx: MonitorRunnerExecutionContext, + ): Boolean { + return isInMemoryDocsSizeExceedingMemoryLimit(docsSizeOfBatchInBytes, monitorCtx) || + isInMemoryNumDocsExceedingMaxDocsPerPercolateQueryLimit(numDocs, monitorCtx) + } - if (hits.hits.isNotEmpty()) { - matchingDocs.addAll(getAllDocs(hits, index, concreteIndex, monitor.id, conflictingFields)) + private suspend fun performPercolateQueryAndResetCounters( + monitorCtx: MonitorRunnerExecutionContext, + monitor: Monitor, + monitorMetadata: MonitorMetadata, + monitorInputIndices: List, + concreteIndices: List, + inputRunResults: MutableMap>, + docsToQueries: MutableMap>, + ) { + try { + val percolateQueryResponseHits = runPercolateQueryOnTransformedDocs( + monitorCtx, + transformedDocs, + monitor, + monitorMetadata, + concreteIndices, + monitorInputIndices, + ) + + percolateQueryResponseHits.forEach { hit -> + var id = hit.id + concreteIndices.forEach { id = id.replace("_${it}_${monitor.id}", "") } + monitorInputIndices.forEach { id = id.replace("_${it}_${monitor.id}", "") } + val docIndices = hit.field("_percolator_document_slot").values.map { it.toString().toInt() } + docIndices.forEach { idx -> + val docIndex = "${transformedDocs[idx].first}|${transformedDocs[idx].second.concreteIndexName}" + inputRunResults.getOrPut(id) { mutableSetOf() }.add(docIndex) + docsToQueries.getOrPut(docIndex) { mutableListOf() }.add(id) } - } catch (e: Exception) { - logger.error("Failed to run for shard $shard. Error: ${e.message}") } + totalDocsQueriedStat += transformedDocs.size.toLong() + } finally { + transformedDocs.clear() + docsSizeOfBatchInBytes = 0 } - return matchingDocs } + /** Executes search query on given shard of given index to fetch docs with sequene number greater than prevSeqNo. + * This method hence fetches only docs from shard which haven't been queried before + */ private suspend fun searchShard( monitorCtx: MonitorRunnerExecutionContext, index: String, @@ -682,7 +791,7 @@ object DocumentLevelMonitorRunner : MonitorRunner() { prevSeqNo: Long?, maxSeqNo: Long, query: String?, - docIds: List? = null + docIds: List? = null, ): SearchHits { if (prevSeqNo?.equals(maxSeqNo) == true && maxSeqNo != 0L) { return SearchHits.empty() @@ -705,46 +814,57 @@ object DocumentLevelMonitorRunner : MonitorRunner() { SearchSourceBuilder() .version(true) .query(boolQueryBuilder) - .size(10000) // fixme: make this configurable. + .size(10000) ) .preference(Preference.PRIMARY_FIRST.type()) val response: SearchResponse = monitorCtx.client!!.suspendUntil { monitorCtx.client!!.search(request, it) } if (response.status() !== RestStatus.OK) { - throw IOException("Failed to search shard: $shard") + logger.error("Failed search shard. Response: $response") + throw IOException("Failed to search shard: [$shard] in index [$index]. Response status is ${response.status()}") } + nonPercolateSearchesTimeTakenStat += response.took.millis return response.hits } - private suspend fun getMatchedQueries( + /** Executes percolate query on the docs against the monitor's query index and return the hits from the search response*/ + private suspend fun runPercolateQueryOnTransformedDocs( monitorCtx: MonitorRunnerExecutionContext, - docs: List, + docs: MutableList>, monitor: Monitor, monitorMetadata: MonitorMetadata, - index: String, - concreteIndex: String + concreteIndices: List, + monitorInputIndices: List, ): SearchHits { - val boolQueryBuilder = BoolQueryBuilder().must(QueryBuilders.matchQuery("index", index).operator(Operator.AND)) - - val percolateQueryBuilder = PercolateQueryBuilderExt("query", docs, XContentType.JSON) + val indices = docs.stream().map { it.second.indexName }.distinct().collect(Collectors.toList()) + val boolQueryBuilder = BoolQueryBuilder().must(buildShouldClausesOverPerIndexMatchQueries(indices)) + val percolateQueryBuilder = + PercolateQueryBuilderExt("query", docs.map { it.second.docSource }, XContentType.JSON) if (monitor.id.isNotEmpty()) { boolQueryBuilder.must(QueryBuilders.matchQuery("monitor_id", monitor.id).operator(Operator.AND)) } boolQueryBuilder.filter(percolateQueryBuilder) - - val queryIndex = monitorMetadata.sourceToQueryIndexMapping[index + monitor.id] - if (queryIndex == null) { - val message = "Failed to resolve concrete queryIndex from sourceIndex during monitor execution!" + - " sourceIndex:$concreteIndex queryIndex:${monitor.dataSources.queryIndex}" + val queryIndices = + docs.map { monitorMetadata.sourceToQueryIndexMapping[it.second.indexName + monitor.id] }.distinct() + if (queryIndices.isEmpty()) { + val message = + "Monitor ${monitor.id}: Failed to resolve query Indices from source indices during monitor execution!" + + " sourceIndices: $monitorInputIndices" logger.error(message) throw AlertingException.wrap( OpenSearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR) ) } - val searchRequest = SearchRequest(queryIndex).preference(Preference.PRIMARY_FIRST.type()) + + val searchRequest = + SearchRequest().indices(*queryIndices.toTypedArray()).preference(Preference.PRIMARY_FIRST.type()) val searchSourceBuilder = SearchSourceBuilder() searchSourceBuilder.query(boolQueryBuilder) searchRequest.source(searchSourceBuilder) - + logger.debug( + "Monitor ${monitor.id}: " + + "Executing percolate query for docs from source indices " + + "$monitorInputIndices against query index $queryIndices" + ) var response: SearchResponse try { response = monitorCtx.client!!.suspendUntil { @@ -752,42 +872,60 @@ object DocumentLevelMonitorRunner : MonitorRunner() { } } catch (e: Exception) { throw IllegalStateException( - "Failed to run percolate search for sourceIndex [$index] and queryIndex [$queryIndex] for ${docs.size} document(s)", e + "Monitor ${monitor.id}:" + + " Failed to run percolate search for sourceIndex [${concreteIndices.joinToString()}] " + + "and queryIndex [${queryIndices.joinToString()}] for ${docs.size} document(s)", + e ) } if (response.status() !== RestStatus.OK) { - throw IOException("Failed to search percolate index: $queryIndex") + throw IOException( + "Monitor ${monitor.id}: Failed to search percolate index: ${queryIndices.joinToString()}. " + + "Response status is ${response.status()}" + ) } + logger.debug("Monitor ${monitor.id} PERF_DEBUG: Percolate query time taken millis = ${response.took}") + percolateQueriesTimeTakenStat += response.took.millis return response.hits } + /** we cannot use terms query because `index` field's mapping is of type TEXT and not keyword. Refer doc-level-queries.json*/ + private fun buildShouldClausesOverPerIndexMatchQueries(indices: List): BoolQueryBuilder { + val boolQueryBuilder = QueryBuilders.boolQuery() + indices.forEach { boolQueryBuilder.should(QueryBuilders.matchQuery("index", it)) } + return boolQueryBuilder + } - private fun getAllDocs( + /** Transform field names and index names in all the search hits to format required to run percolate search against them. + * Hits are transformed using method transformDocumentFieldNames() */ + private fun transformSearchHitsAndReconstructDocs( hits: SearchHits, index: String, concreteIndex: String, monitorId: String, - conflictingFields: List - ): List> { - return hits.map { hit -> - val sourceMap = hit.sourceAsMap - - transformDocumentFieldNames( - sourceMap, - conflictingFields, - "_${index}_$monitorId", - "_${concreteIndex}_$monitorId", - "" - ) - - var xContentBuilder = XContentFactory.jsonBuilder().map(sourceMap) - - val sourceRef = BytesReference.bytes(xContentBuilder) - - logger.debug("Document [${hit.id}] payload after transform: ", sourceRef.utf8ToString()) - - Pair(hit.id, sourceRef) - } + conflictingFields: List, + ): List> { + return hits.mapNotNull(fun(hit: SearchHit): Pair? { + try { + val sourceMap = hit.sourceAsMap + transformDocumentFieldNames( + sourceMap, + conflictingFields, + "_${index}_$monitorId", + "_${concreteIndex}_$monitorId", + "" + ) + var xContentBuilder = XContentFactory.jsonBuilder().map(sourceMap) + val sourceRef = BytesReference.bytes(xContentBuilder) + docsSizeOfBatchInBytes += sourceRef.ramBytesUsed() + totalDocsSizeInBytesStat += sourceRef.ramBytesUsed() + return Pair(hit.id, TransformedDocDto(index, concreteIndex, hit.id, sourceRef)) + } catch (e: Exception) { + logger.error("Monitor $monitorId: Failed to transform payload $hit for percolate query", e) + // skip any document which we fail to transform because we anyway won't be able to run percolate queries on them. + return null + } + }) } /** @@ -840,4 +978,33 @@ object DocumentLevelMonitorRunner : MonitorRunner() { } jsonAsMap.putAll(tempMap) } + + /** + * Returns true, if the docs fetched from shards thus far amount to less than threshold + * amount of percentage (default:10. setting is dynamic and configurable) of the total heap size or not. + * + */ + private fun isInMemoryDocsSizeExceedingMemoryLimit(docsBytesSize: Long, monitorCtx: MonitorRunnerExecutionContext): Boolean { + var thresholdPercentage = PERCOLATE_QUERY_DOCS_SIZE_MEMORY_PERCENTAGE_LIMIT.get(monitorCtx.settings) + val heapMaxBytes = monitorCtx.jvmStats!!.mem.heapMax.bytes + val thresholdBytes = (thresholdPercentage.toDouble() / 100.0) * heapMaxBytes + + return docsBytesSize > thresholdBytes + } + + private fun isInMemoryNumDocsExceedingMaxDocsPerPercolateQueryLimit(numDocs: Int, monitorCtx: MonitorRunnerExecutionContext): Boolean { + var maxNumDocsThreshold = PERCOLATE_QUERY_MAX_NUM_DOCS_IN_MEMORY.get(monitorCtx.settings) + return numDocs >= maxNumDocsThreshold + } + + /** + * POJO holding information about each doc's concrete index, id, input index pattern/alias/datastream name + * and doc source. A list of these POJOs would be passed to percolate query execution logic. + */ + data class TransformedDocDto( + var indexName: String, + var concreteIndexName: String, + var docId: String, + var docSource: BytesReference + ) } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt index 10ad88ac2..2e72af40b 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt @@ -18,6 +18,7 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.monitor.jvm.JvmStats import org.opensearch.script.ScriptService import org.opensearch.threadpool.ThreadPool @@ -35,6 +36,8 @@ data class MonitorRunnerExecutionContext( var triggerService: TriggerService? = null, var alertService: AlertService? = null, var docLevelMonitorQueries: DocLevelMonitorQueries? = null, + var workflowService: WorkflowService? = null, + var jvmStats: JvmStats? = null, @Volatile var retryPolicy: BackoffPolicy? = null, @Volatile var moveAlertsRetryPolicy: BackoffPolicy? = null, diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt index a2c4dc9ee..bf29b0407 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt @@ -47,6 +47,7 @@ import org.opensearch.commons.alerting.model.ScheduledJob import org.opensearch.commons.alerting.model.action.Action import org.opensearch.commons.alerting.util.isBucketLevelMonitor import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.monitor.jvm.JvmStats import org.opensearch.script.Script import org.opensearch.script.ScriptService import org.opensearch.script.TemplateScript @@ -123,6 +124,11 @@ object MonitorRunnerService : JobRunner, CoroutineScope, AbstractLifecycleCompon return this } + fun registerJvmStats(jvmStats: JvmStats): MonitorRunnerService { + this.monitorCtx.jvmStats = jvmStats + return this + } + // Must be called after registerClusterService and registerSettings in AlertingPlugin fun registerConsumers(): MonitorRunnerService { monitorCtx.retryPolicy = BackoffPolicy.constantBackoff( @@ -225,11 +231,28 @@ object MonitorRunnerService : JobRunner, CoroutineScope, AbstractLifecycleCompon } override fun runJob(job: ScheduledJob, periodStart: Instant, periodEnd: Instant) { - if (job !is Monitor) { - throw IllegalArgumentException("Invalid job type") - } - launch { - runJob(job, periodStart, periodEnd, false) + when (job) { + is Workflow -> { + launch { + logger.debug( + "PERF_DEBUG: executing workflow ${job.id} on node " + + monitorCtx.clusterService!!.state().nodes().localNode.id + ) + runJob(job, periodStart, periodEnd, false) + } + } + is Monitor -> { + launch { + logger.debug( + "PERF_DEBUG: executing ${job.monitorType} ${job.id} on node " + + monitorCtx.clusterService!!.state().nodes().localNode.id + ) + runJob(job, periodStart, periodEnd, false) + } + } + else -> { + throw IllegalArgumentException("Invalid job type") + } } } @@ -255,7 +278,7 @@ object MonitorRunnerService : JobRunner, CoroutineScope, AbstractLifecycleCompon val runResult = if (monitor.isBucketLevelMonitor()) { BucketLevelMonitorRunner.runMonitor(monitor, monitorCtx, periodStart, periodEnd, dryrun) } else if (monitor.isDocLevelMonitor()) { - DocumentLevelMonitorRunner.runMonitor(monitor, monitorCtx, periodStart, periodEnd, dryrun) + DocumentLevelMonitorRunner().runMonitor(monitor, monitorCtx, periodStart, periodEnd, dryrun, executionId = executionId) } else { QueryLevelMonitorRunner.runMonitor(monitor, monitorCtx, periodStart, periodEnd, dryrun) } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/settings/AlertingSettings.kt b/alerting/src/main/kotlin/org/opensearch/alerting/settings/AlertingSettings.kt index 8d00d4569..d106db95d 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/settings/AlertingSettings.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/settings/AlertingSettings.kt @@ -26,6 +26,29 @@ class AlertingSettings { Setting.Property.Dynamic ) + /** Defines the threshold percentage of heap size in bytes till which we accumulate docs in memory before we query against percolate query + * index in document level monitor execution. + */ + val PERCOLATE_QUERY_DOCS_SIZE_MEMORY_PERCENTAGE_LIMIT = Setting.intSetting( + "plugins.alerting.monitor.percolate_query_docs_size_memory_percentage_limit", + 10, + 0, + 100, + Setting.Property.NodeScope, Setting.Property.Dynamic + ) + + /** Defines the threshold of the maximum number of docs accumulated in memory to query against percolate query index in document + * level monitor execution. The docs are being collected from searching on shards of indices mentioned in the + * monitor input indices field. When the number of in-memory docs reaches or exceeds threshold we immediately perform percolate + * query with the current set of docs and clear the cache and repeat the process till we have queried all indices in current + * execution + */ + val PERCOLATE_QUERY_MAX_NUM_DOCS_IN_MEMORY = Setting.intSetting( + "plugins.alerting.monitor.percolate_query_max_num_docs_in_memory", + 300000, 1000, + Setting.Property.NodeScope, Setting.Property.Dynamic + ) + val INPUT_TIMEOUT = Setting.positiveTimeSetting( "plugins.alerting.input_timeout", LegacyOpenDistroAlertingSettings.INPUT_TIMEOUT, 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..a22b09bdc --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/workflow/CompositeWorkflowRunner.kt @@ -0,0 +1,392 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting.workflow + +import org.apache.logging.log4j.LogManager +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +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.ChainedAlertTriggerRunResult +import org.opensearch.alerting.model.MonitorRunResult +import org.opensearch.alerting.model.WorkflowRunResult +import org.opensearch.alerting.opensearchapi.suspendUntil +import org.opensearch.alerting.script.ChainedAlertTriggerExecutionContext +import org.opensearch.alerting.util.AlertingException +import org.opensearch.alerting.util.isDocLevelMonitor +import org.opensearch.alerting.util.isQueryLevelMonitor +import org.opensearch.cluster.routing.Preference +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.commons.alerting.model.Alert +import org.opensearch.commons.alerting.model.ChainedAlertTrigger +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.Monitor +import org.opensearch.commons.alerting.model.Workflow +import org.opensearch.commons.alerting.util.isBucketLevelMonitor +import org.opensearch.core.xcontent.XContentParser +import org.opensearch.core.xcontent.XContentParserUtils +import org.opensearch.index.query.QueryBuilders +import org.opensearch.index.query.QueryBuilders.boolQuery +import org.opensearch.index.query.QueryBuilders.existsQuery +import org.opensearch.index.query.QueryBuilders.termsQuery +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 dataSources: DataSources? = null + 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 WorkflowRunResult( + workflow.id, + workflow.name, + emptyList(), + workflowExecutionStartTime, + Instant.now(), + executionId, + 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 chainedMonitorIds: MutableList = mutableListOf() + if (delegate.chainedMonitorFindings!!.monitorId.isNullOrBlank()) { + chainedMonitorIds.addAll(delegate.chainedMonitorFindings!!.monitorIds) + } else { + chainedMonitorIds.add(delegate.chainedMonitorFindings!!.monitorId!!) + } + val chainedMonitors = mutableListOf() + chainedMonitorIds.forEach { + val chainedMonitor = monitorsById[it] + ?: throw AlertingException.wrap( + IllegalStateException("Chained finding monitor not found ${delegate.monitorId} for the workflow $workflow.id") + ) + chainedMonitors.add(chainedMonitor) + } + + try { + indexToDocIds = monitorCtx.workflowService!!.getFindingDocIdsByExecutionId(chainedMonitors, executionId) + } catch (e: Exception) { + logger.error("Failed to execute workflow due to failure in chained findings. Error: ${e.message}", e) + return WorkflowRunResult( + workflow.id, workflow.name, emptyList(), workflowExecutionStartTime, Instant.now(), executionId, + AlertingException.wrap(e) + ) + } + } + val workflowRunContext = WorkflowRunContext( + workflowId = workflowMetadata.workflowId, + workflowMetadataId = workflowMetadata.id, + chainedMonitorId = delegate.chainedMonitorFindings?.monitorId, + matchingDocIdsPerIndex = indexToDocIds, + auditDelegateMonitorAlerts = if (workflow.auditDelegateMonitorAlerts == null) true + else workflow.auditDelegateMonitorAlerts!! + ) + try { + dataSources = delegateMonitor.dataSources + val delegateRunResult = + runDelegateMonitor(delegateMonitor, monitorCtx, periodStart, periodEnd, dryRun, workflowRunContext, executionId) + resultList.add(delegateRunResult!!) + } catch (ex: Exception) { + logger.error("Error executing workflow delegate monitor ${delegate.monitorId}", ex) + lastErrorDelegateRun = AlertingException.wrap(ex) + break + } + } + logger.debug("Workflow ${workflow.id} delegate monitors in execution $executionId completed") + // Update metadata only if the workflow is not temp + if (!isTempWorkflow) { + WorkflowMetadataService.upsertWorkflowMetadata( + workflowMetadata.copy(latestRunTime = workflowExecutionStartTime, latestExecutionId = executionId), + true + ) + } + val triggerResults = mutableMapOf() + val workflowRunResult = WorkflowRunResult( + workflowId = workflow.id, + workflowName = workflow.name, + monitorRunResults = resultList, + executionStartTime = workflowExecutionStartTime, + executionEndTime = null, + executionId = executionId, + error = lastErrorDelegateRun, + triggerResults = triggerResults + ) + val currentAlerts = try { + monitorCtx.alertIndices!!.createOrUpdateAlertIndex(dataSources!!) + monitorCtx.alertIndices!!.createOrUpdateInitialAlertHistoryIndex(dataSources) + monitorCtx.alertService!!.loadCurrentAlertsForWorkflow(workflow, dataSources) + } catch (e: Exception) { + logger.error("Failed to fetch current alerts for workflow", e) + // We can't save ERROR alerts to the index here as we don't know if there are existing ACTIVE alerts + val id = if (workflow.id.trim().isEmpty()) "_na_" else workflow.id + logger.error("Error loading alerts for workflow: $id", e) + return workflowRunResult.copy(error = e) + } + try { + monitorCtx.alertIndices!!.createOrUpdateAlertIndex(dataSources) + val updatedAlerts = mutableListOf() + val monitorIdToAlertIdsMap = fetchAlertsGeneratedInCurrentExecution(dataSources, executionId, monitorCtx, workflow) + for (trigger in workflow.triggers) { + val currentAlert = currentAlerts[trigger] + val caTrigger = trigger as ChainedAlertTrigger + val triggerCtx = ChainedAlertTriggerExecutionContext( + workflow = workflow, + workflowRunResult = workflowRunResult, + periodStart = workflowRunResult.executionStartTime, + periodEnd = workflowRunResult.executionEndTime, + trigger = caTrigger, + alertGeneratingMonitors = monitorIdToAlertIdsMap.keys, + monitorIdToAlertIdsMap = monitorIdToAlertIdsMap, + alert = currentAlert + ) + runChainedAlertTrigger( + monitorCtx, + workflow, + trigger, + executionId, + triggerCtx, + dryRun, + triggerResults, + updatedAlerts + ) + } + if (!dryRun && workflow.id != Workflow.NO_ID && updatedAlerts.isNotEmpty()) { + monitorCtx.retryPolicy?.let { + monitorCtx.alertService!!.saveAlerts( + dataSources, + updatedAlerts, + it, + routingId = workflow.id + ) + } + } + } catch (e: Exception) { + // We can't save ERROR alerts to the index here as we don't know if there are existing ACTIVE alerts + val id = if (workflow.id.trim().isEmpty()) "_na_" else workflow.id + logger.error("Error loading current chained alerts for workflow: $id", e) + return WorkflowRunResult( + workflowId = workflow.id, + workflowName = workflow.name, + monitorRunResults = emptyList(), + executionStartTime = workflowExecutionStartTime, + executionEndTime = Instant.now(), + executionId = executionId, + error = AlertingException.wrap(e), + triggerResults = emptyMap() + ) + } + workflowRunResult.executionEndTime = Instant.now() + return workflowRunResult + } + + private suspend fun runDelegateMonitor( + delegateMonitor: Monitor, + monitorCtx: MonitorRunnerExecutionContext, + periodStart: Instant, + periodEnd: Instant, + dryRun: Boolean, + workflowRunContext: WorkflowRunContext, + executionId: String, + ): MonitorRunResult<*>? { + + if (delegateMonitor.isBucketLevelMonitor()) { + return BucketLevelMonitorRunner.runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext, + executionId + ) + } else if (delegateMonitor.isDocLevelMonitor()) { + return DocumentLevelMonitorRunner().runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext, + executionId + ) + } else if (delegateMonitor.isQueryLevelMonitor()) { + return QueryLevelMonitorRunner.runMonitor( + delegateMonitor, + monitorCtx, + periodStart, + periodEnd, + dryRun, + workflowRunContext, + executionId + ) + } else { + throw AlertingException.wrap( + IllegalStateException("Unsupported monitor type ${delegateMonitor.monitorType}") + ) + } + } + + 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") + ) + } + } + + private suspend fun runChainedAlertTrigger( + monitorCtx: MonitorRunnerExecutionContext, + workflow: Workflow, + trigger: ChainedAlertTrigger, + executionId: String, + triggerCtx: ChainedAlertTriggerExecutionContext, + dryRun: Boolean, + triggerResults: MutableMap, + updatedAlerts: MutableList, + ) { + val triggerRunResult = monitorCtx.triggerService!!.runChainedAlertTrigger( + workflow, trigger, triggerCtx.alertGeneratingMonitors, triggerCtx.monitorIdToAlertIdsMap + ) + triggerResults[trigger.id] = triggerRunResult + if (monitorCtx.triggerService!!.isChainedAlertTriggerActionable(triggerCtx, triggerRunResult)) { + val actionCtx = triggerCtx + for (action in trigger.actions) { + triggerRunResult.actionResults[action.id] = this.runAction(action, actionCtx, monitorCtx, workflow, dryRun) + } + } + val alert = monitorCtx.alertService!!.composeChainedAlert( + triggerCtx, executionId, workflow, triggerRunResult.associatedAlertIds.toList(), triggerRunResult + ) + if (alert != null) { + updatedAlerts.add(alert) + } + } + + private suspend fun fetchAlertsGeneratedInCurrentExecution( + dataSources: DataSources, + executionId: String, + monitorCtx: MonitorRunnerExecutionContext, + workflow: Workflow, + ): MutableMap> { + try { + val searchRequest = + SearchRequest(getDelegateMonitorAlertIndex(dataSources, workflow, monitorCtx.alertIndices!!.isAlertHistoryEnabled())) + searchRequest.preference(Preference.PRIMARY_FIRST.type()) + val queryBuilder = boolQuery() + queryBuilder.must(QueryBuilders.termQuery("execution_id", executionId)) + queryBuilder.must(QueryBuilders.termQuery("state", getDelegateMonitorAlertState(workflow).name)) + val noErrorQuery = boolQuery() + .should(boolQuery().mustNot(existsQuery(Alert.ERROR_MESSAGE_FIELD))) + .should(termsQuery(Alert.ERROR_MESSAGE_FIELD, "")) + queryBuilder.must(noErrorQuery) + searchRequest.source().query(queryBuilder).size(9999) + val searchResponse: SearchResponse = monitorCtx.client!!.suspendUntil { monitorCtx.client!!.search(searchRequest, it) } + val alerts = searchResponse.hits.map { hit -> + val xcp = XContentHelper.createParser( + monitorCtx.xContentRegistry, LoggingDeprecationHandler.INSTANCE, + hit.sourceRef, XContentType.JSON + ) + XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.nextToken(), xcp) + val alert = Alert.parse(xcp, hit.id, hit.version) + alert + } + val map = mutableMapOf>() + for (alert in alerts) { + if (map.containsKey(alert.monitorId)) { + map[alert.monitorId]!!.add(alert.id) + } else { + map[alert.monitorId] = mutableSetOf(alert.id) + } + } + return map + } catch (e: Exception) { + logger.error("failed to get alerts generated by delegate monitors in current execution $executionId", e) + return mutableMapOf() + } + } + + fun getDelegateMonitorAlertIndex( + dataSources: DataSources, + workflow: Workflow, + isAlertHistoryEnabled: Boolean, + ): String { + return if (workflow.triggers.isNotEmpty()) { + if (isAlertHistoryEnabled) { + dataSources.alertsHistoryIndex!! + } else dataSources.alertsIndex + } else dataSources.alertsIndex + } + + fun getDelegateMonitorAlertState( + workflow: Workflow, + ): Alert.State { + return if (workflow.triggers.isNotEmpty()) { + Alert.State.AUDIT + } else Alert.State.ACTIVE + } +} 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..1bfea4ebc 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": 4 }, "properties": { "schema_version": { @@ -46,6 +46,12 @@ "type" : "keyword" } } + }, + "fields": { + "type": "text" + }, + "query_field_names": { + "type": "keyword" } } }, @@ -60,6 +66,9 @@ "type" : "keyword" } } + }, + "execution_id": { + "type": "keyword" } } } \ No newline at end of file