diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt index 89ed19c23..942422f9c 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/AlertingPlugin.kt @@ -221,6 +221,7 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R .registerClusterService(clusterService) .registerClient(client) .registerNamedXContentRegistry(xContentRegistry) + .registerindexNameExpressionResolver(indexNameExpressionResolver) .registerScriptService(scriptService) .registerSettings(settings) .registerThreadPool(threadPool) @@ -238,6 +239,14 @@ internal class AlertingPlugin : PainlessExtension, ActionPlugin, ScriptPlugin, R destinationMigrationCoordinator = DestinationMigrationCoordinator(client, clusterService, threadPool, scheduledJobIndices) this.threadPool = threadPool this.clusterService = clusterService + + MonitorMetadataService.initialize( + client, + clusterService, + xContentRegistry, + settings + ) + return listOf(sweeper, scheduler, runner, scheduledJobIndices, docLevelMonitorQueries, destinationMigrationCoordinator) } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt index ee31fe1a3..416789470 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/DocumentLevelMonitorRunner.kt @@ -7,8 +7,7 @@ package org.opensearch.alerting import org.apache.logging.log4j.LogManager import org.opensearch.ExceptionsHelper -import org.opensearch.action.admin.indices.get.GetIndexRequest -import org.opensearch.action.admin.indices.get.GetIndexResponse +import org.opensearch.OpenSearchStatusException import org.opensearch.action.index.IndexRequest import org.opensearch.action.index.IndexResponse import org.opensearch.action.search.SearchAction @@ -21,23 +20,24 @@ import org.opensearch.alerting.core.model.DocLevelQuery import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.model.ActionExecutionResult import org.opensearch.alerting.model.Alert -import org.opensearch.alerting.model.AlertingConfigAccessor.Companion.getMonitorMetadata import org.opensearch.alerting.model.DocumentExecutionContext import org.opensearch.alerting.model.DocumentLevelTrigger import org.opensearch.alerting.model.DocumentLevelTriggerRunResult import org.opensearch.alerting.model.Finding import org.opensearch.alerting.model.InputRunResults import org.opensearch.alerting.model.Monitor +import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.model.MonitorRunResult import org.opensearch.alerting.model.action.PerAlertActionScope import org.opensearch.alerting.opensearchapi.string import org.opensearch.alerting.opensearchapi.suspendUntil import org.opensearch.alerting.script.DocumentLevelTriggerExecutionContext 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.util.updateMonitorMetadata import org.opensearch.client.Client +import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.routing.ShardRouting import org.opensearch.cluster.service.ClusterService import org.opensearch.common.bytes.BytesReference @@ -46,6 +46,7 @@ import org.opensearch.common.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.common.xcontent.XContentType 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.rest.RestStatus @@ -54,8 +55,7 @@ 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 kotlin.collections.HashMap +import java.util.* import kotlin.math.max object DocumentLevelMonitorRunner : MonitorRunner() { @@ -69,6 +69,7 @@ object DocumentLevelMonitorRunner : MonitorRunner() { dryrun: Boolean ): MonitorRunResult { logger.debug("Document-level-monitor is running ...") + val isTempMonitor = dryrun || monitor.id == Monitor.NO_ID var monitorResult = MonitorRunResult(monitor.name, periodStart, periodEnd) try { @@ -78,33 +79,26 @@ object DocumentLevelMonitorRunner : MonitorRunner() { } catch (e: Exception) { val id = if (monitor.id.trim().isEmpty()) "_na_" else monitor.id logger.error("Error setting up alerts and findings indices for monitor: $id", e) - return monitorResult.copy(error = AlertingException.wrap(e)) + monitorResult = monitorResult.copy(error = AlertingException.wrap(e)) } try { validate(monitor) } catch (e: Exception) { logger.error("Failed to start Document-level-monitor. Error: ${e.message}") - return monitorResult.copy(error = AlertingException.wrap(e)) + monitorResult = monitorResult.copy(error = AlertingException.wrap(e)) } - monitorCtx.docLevelMonitorQueries!!.initDocLevelQueryIndex() - monitorCtx.docLevelMonitorQueries!!.indexDocLevelQueries( + var (monitorMetadata, _) = MonitorMetadataService.getOrCreateMetadata( monitor = monitor, - monitorId = monitor.id, - indexTimeout = monitorCtx.indexTimeout!! + createWithRunContext = false, + skipIndex = isTempMonitor ) val docLevelMonitorInput = monitor.inputs[0] as DocLevelMonitorInput - val index = docLevelMonitorInput.indices[0] - val queries: List = docLevelMonitorInput.queries - var monitorMetadata = getMonitorMetadata(monitorCtx.client!!, monitorCtx.xContentRegistry!!, "${monitor.id}-metadata") - if (monitorMetadata == null) { - monitorMetadata = createMonitorMetadata(monitor.id) - } + val queries: List = docLevelMonitorInput.queries - val isTempMonitor = dryrun || monitor.id == Monitor.NO_ID val lastRunContext = if (monitorMetadata.lastRunContext.isNullOrEmpty()) mutableMapOf() else monitorMetadata.lastRunContext.toMutableMap() as MutableMap> @@ -115,11 +109,20 @@ object DocumentLevelMonitorRunner : MonitorRunner() { val docsToQueries = mutableMapOf>() try { - val getIndexRequest = GetIndexRequest().indices(index) - val getIndexResponse: GetIndexResponse = monitorCtx.client!!.suspendUntil { - monitorCtx.client!!.admin().indices().getIndex(getIndexRequest, it) - } - val indices = getIndexResponse.indices() + // Resolve all passed indices to concrete indices + val indices = IndexUtils.resolveAllIndices( + docLevelMonitorInput.indices, + monitorCtx.clusterService!!, + monitorCtx.indexNameExpressionResolver!! + ) + + monitorCtx.docLevelMonitorQueries!!.initDocLevelQueryIndex() + monitorCtx.docLevelMonitorQueries!!.indexDocLevelQueries( + monitor = monitor, + monitorId = monitor.id, + monitorMetadata, + indexTimeout = monitorCtx.indexTimeout!! + ) // cleanup old indices that are not monitored anymore from the same monitor for (ind in updatedLastRunContext.keys) { @@ -131,8 +134,13 @@ object DocumentLevelMonitorRunner : MonitorRunner() { indices.forEach { indexName -> // Prepare lastRunContext for each index val indexLastRunContext = lastRunContext.getOrPut(indexName) { - val indexCreatedRecently = createdRecently(monitor, indexName, periodStart, periodEnd, getIndexResponse) - createRunContext(monitorCtx.clusterService!!, monitorCtx.client!!, indexName, indexCreatedRecently) + val isIndexCreatedRecently = createdRecently( + monitor, + periodStart, + periodEnd, + monitorCtx.clusterService!!.state().metadata.index(indexName) + ) + MonitorMetadataService.createRunContextForIndex(indexName, isIndexCreatedRecently) } // Prepare updatedLastRunContext for each index @@ -160,7 +168,13 @@ object DocumentLevelMonitorRunner : MonitorRunner() { val matchingDocs = getMatchingDocs(monitor, monitorCtx, docExecutionContext, indexName) if (matchingDocs.isNotEmpty()) { - val matchedQueriesForDocs = getMatchedQueries(monitorCtx, matchingDocs.map { it.second }, monitor, indexName) + val matchedQueriesForDocs = getMatchedQueries( + monitorCtx, + matchingDocs.map { it.second }, + monitor, + monitorMetadata, + indexName + ) matchedQueriesForDocs.forEach { hit -> val id = hit.id.replace("_${indexName}_${monitor.id}", "") @@ -214,7 +228,10 @@ object DocumentLevelMonitorRunner : MonitorRunner() { // Don't update monitor if this is a test monitor if (!isTempMonitor) { - updateMonitorMetadata(monitorCtx.client!!, monitorCtx.settings!!, monitorMetadata.copy(lastRunContext = updatedLastRunContext)) + MonitorMetadataService.upsertMetadata( + monitorMetadata.copy(lastRunContext = updatedLastRunContext), + true + ) } // TODO: Update the Document as part of the Trigger and return back the trigger action result @@ -265,6 +282,16 @@ object DocumentLevelMonitorRunner : MonitorRunner() { alerts.add(alert) } + if (findingDocPairs.isEmpty() && monitorResult.error != null) { + val alert = monitorCtx.alertService!!.composeDocLevelAlert( + listOf(), + listOf(), + triggerCtx, + monitorResult.alertError() ?: triggerResult.alertError() + ) + alerts.add(alert) + } + val shouldDefaultToPerExecution = defaultToPerExecutionAction( monitorCtx.maxActionableAlertCount, monitorId = monitor.id, @@ -366,42 +393,22 @@ object DocumentLevelMonitorRunner : MonitorRunner() { throw IOException("Invalid input with document-level-monitor.") } - val docLevelMonitorInput = monitor.inputs[0] as DocLevelMonitorInput - if (docLevelMonitorInput.indices.size > 1) { - throw IOException("Only one index is supported with document-level-monitor.") - } - } - - suspend fun createRunContext( - clusterService: ClusterService, - client: Client, - index: String, - createdRecently: Boolean = false - ): HashMap { - val lastRunContext = HashMap() - lastRunContext["index"] = index - val count = getShardsCount(clusterService, index) - lastRunContext["shards_count"] = count - - for (i: Int in 0 until count) { - val shard = i.toString() - val maxSeqNo: Long = if (createdRecently) -1L else getMaxSeqNo(client, index, shard) - lastRunContext[shard] = maxSeqNo + if ((monitor.inputs[0] as DocLevelMonitorInput).indices.isEmpty()) { + throw IllegalArgumentException("DocLevelMonitorInput has no indices") } - return lastRunContext } // Checks if the index was created from the last execution run or when the monitor was last updated to ensure that // new index is monitored from the beginning of that index private fun createdRecently( monitor: Monitor, - index: String, periodStart: Instant, periodEnd: Instant, - getIndexResponse: GetIndexResponse + indexMetadata: IndexMetadata ): Boolean { val lastExecutionTime = if (periodStart == periodEnd) monitor.lastUpdateTime else periodStart - return getIndexResponse.settings.get(index).getAsLong("index.creation_date", 0L) > lastExecutionTime.toEpochMilli() + val indexCreationDate = indexMetadata.settings.get("index.creation_date")?.toLong() ?: 0L + return indexCreationDate > lastExecutionTime.toEpochMilli() } /** @@ -506,17 +513,27 @@ object DocumentLevelMonitorRunner : MonitorRunner() { monitorCtx: MonitorRunnerExecutionContext, docs: List, monitor: Monitor, + monitorMetadata: MonitorMetadata, index: String ): SearchHits { - val boolQueryBuilder = BoolQueryBuilder().filter(QueryBuilders.matchQuery("index", index)) + val boolQueryBuilder = BoolQueryBuilder().must(QueryBuilders.matchQuery("index", index).operator(Operator.AND)) val percolateQueryBuilder = PercolateQueryBuilderExt("query", docs, XContentType.JSON) if (monitor.id.isNotEmpty()) { - boolQueryBuilder.filter(QueryBuilders.matchQuery("monitor_id", monitor.id)) + boolQueryBuilder.must(QueryBuilders.matchQuery("monitor_id", monitor.id).operator(Operator.AND)) } boolQueryBuilder.filter(percolateQueryBuilder) - val searchRequest = SearchRequest(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) + val queryIndex = monitorMetadata.sourceToQueryIndexMapping[index + monitor.id] + if (queryIndex == null) { + val message = "Failed to resolve concrete queryIndex from sourceIndex during monitor execution!" + + " sourceIndex:$index queryIndex:${ScheduledJob.DOC_LEVEL_QUERIES_INDEX}" + logger.error(message) + throw AlertingException.wrap( + OpenSearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR) + ) + } + val searchRequest = SearchRequest(queryIndex) val searchSourceBuilder = SearchSourceBuilder() searchSourceBuilder.query(boolQueryBuilder) searchRequest.source(searchSourceBuilder) diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt new file mode 100644 index 000000000..be4b2bd7a --- /dev/null +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorMetadataService.kt @@ -0,0 +1,253 @@ +/* + * 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.ExceptionsHelper +import org.opensearch.OpenSearchSecurityException +import org.opensearch.action.DocWriteRequest +import org.opensearch.action.DocWriteResponse +import org.opensearch.action.admin.indices.get.GetIndexRequest +import org.opensearch.action.admin.indices.get.GetIndexResponse +import org.opensearch.action.admin.indices.stats.IndicesStatsAction +import org.opensearch.action.admin.indices.stats.IndicesStatsRequest +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse +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.core.model.DocLevelMonitorInput +import org.opensearch.alerting.core.model.ScheduledJob +import org.opensearch.alerting.model.Monitor +import org.opensearch.alerting.model.MonitorMetadata +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.NamedXContentRegistry +import org.opensearch.common.xcontent.ToXContent +import org.opensearch.common.xcontent.XContentFactory +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentParser +import org.opensearch.common.xcontent.XContentParserUtils +import org.opensearch.common.xcontent.XContentType +import org.opensearch.index.seqno.SequenceNumbers +import org.opensearch.rest.RestStatus +import org.opensearch.transport.RemoteTransportException + +private val log = LogManager.getLogger(MonitorMetadataService::class.java) + +object MonitorMetadataService : + CoroutineScope by CoroutineScope(SupervisorJob() + Dispatchers.Default + CoroutineName("MonitorMetadataService")) { + + 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 upsertMetadata(metadata: MonitorMetadata, updating: Boolean): MonitorMetadata { + 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.monitorId) + .setIfSeqNo(metadata.seqNo) + .setIfPrimaryTerm(metadata.primaryTerm) + .timeout(indexTimeout) + + if (updating) { + indexRequest.id(metadata.id).setIfSeqNo(metadata.seqNo).setIfPrimaryTerm(metadata.primaryTerm) + } 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 MonitorMetadata:${metadata.id} ") + } + } + return metadata.copy( + seqNo = response.seqNo, + primaryTerm = response.primaryTerm + ) + } catch (e: Exception) { + throw AlertingException.wrap(e) + } + } + + suspend fun getOrCreateMetadata( + monitor: Monitor, + createWithRunContext: Boolean = true, + skipIndex: Boolean = false + ): Pair { + try { + val created = true + val metadata = getMetadata(monitor) + return if (metadata != null) { + metadata to !created + } else { + val newMetadata = createNewMetadata(monitor, createWithRunContext = createWithRunContext) + if (skipIndex) { + newMetadata to created + } else { + upsertMetadata(newMetadata, updating = false) to created + } + } + } catch (e: Exception) { + throw AlertingException.wrap(e) + } + } + + suspend fun getMetadata(monitor: Monitor): MonitorMetadata? { + try { + val metadataId = MonitorMetadata.getId(monitor) + val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, metadataId).routing(monitor.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) + MonitorMetadata.parse(xcp) + } else { + null + } + } catch (e: Exception) { + if (e.message?.contains("no such index") == true) { + return null + } else { + throw AlertingException.wrap(e) + } + } + } + + suspend fun recreateRunContext(metadata: MonitorMetadata, monitor: Monitor): MonitorMetadata { + try { + 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) + createFullRunContext(monitorIndex, metadata.lastRunContext as MutableMap>) + else null + if (runContext != null) { + return metadata.copy( + lastRunContext = runContext + ) + } else { + return 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) + (monitor.inputs[0] as DocLevelMonitorInput).indices[0] + else null + val runContext = + if (monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR && createWithRunContext) + createFullRunContext(monitorIndex) + else emptyMap() + return MonitorMetadata( + id = "${monitor.id}-metadata", + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, + monitorId = monitor.id, + lastActionExecutionTimes = emptyList(), + lastRunContext = runContext, + sourceToQueryIndexMapping = mutableMapOf() + ) + } + + private suspend fun createFullRunContext( + index: String?, + existingRunContext: MutableMap>? = null + ): MutableMap> { + val lastRunContext = existingRunContext?.toMutableMap() ?: mutableMapOf() + try { + if (index == null) return mutableMapOf() + val getIndexRequest = GetIndexRequest().indices(index) + val getIndexResponse: GetIndexResponse = client.suspendUntil { + client.admin().indices().getIndex(getIndexRequest, it) + } + val indices = getIndexResponse.indices() + + indices.forEach { indexName -> + if (!lastRunContext.containsKey(indexName)) { + lastRunContext[indexName] = createRunContextForIndex(indexName) + } + } + } catch (e: RemoteTransportException) { + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + throw AlertingException("Failed fetching index stats", RestStatus.INTERNAL_SERVER_ERROR, unwrappedException) + } catch (e: OpenSearchSecurityException) { + throw AlertingException( + "Failed fetching index stats - missing required index permissions: ${e.localizedMessage}", + RestStatus.INTERNAL_SERVER_ERROR, + e + ) + } catch (e: Exception) { + throw AlertingException("Failed fetching index stats", RestStatus.INTERNAL_SERVER_ERROR, e) + } + return lastRunContext + } + + suspend fun createRunContextForIndex(index: String, createdRecently: Boolean = false): MutableMap { + val request = IndicesStatsRequest().indices(index).clear() + val response: IndicesStatsResponse = client.suspendUntil { execute(IndicesStatsAction.INSTANCE, request, it) } + if (response.status != RestStatus.OK) { + val errorMessage = "Failed fetching index stats for index:$index" + throw AlertingException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR, IllegalStateException(errorMessage)) + } + val shards = response.shards.filter { it.shardRouting.primary() && it.shardRouting.active() } + val lastRunContext = HashMap() + lastRunContext["index"] = index + val count = shards.size + lastRunContext["shards_count"] = count + + for (shard in shards) { + lastRunContext[shard.shardRouting.id.toString()] = + if (createdRecently) -1L + else shard.seqNoStats?.globalCheckpoint ?: SequenceNumbers.UNASSIGNED_SEQ_NO + } + return lastRunContext + } +} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt index de8c57993..6b1a490c6 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunner.kt @@ -11,7 +11,6 @@ import org.opensearch.alerting.action.GetDestinationsRequest import org.opensearch.alerting.action.GetDestinationsResponse import org.opensearch.alerting.model.ActionRunResult import org.opensearch.alerting.model.Monitor -import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.model.MonitorRunResult import org.opensearch.alerting.model.Table import org.opensearch.alerting.model.action.Action @@ -63,16 +62,24 @@ abstract class MonitorRunner { throw IllegalStateException("Message content missing in the Destination with id: ${action.destinationId}") } if (!dryrun) { - val roles = MonitorRunnerService.getRolesForMonitor(monitor) - withClosableContext( - InjectorContextElement(monitor.id, monitorCtx.settings!!, monitorCtx.threadPool!!.threadContext, roles) - ) { - actionOutput[Action.MESSAGE_ID] = getConfigAndSendNotification( - action, - monitorCtx, - actionOutput[Action.SUBJECT], - actionOutput[Action.MESSAGE]!! - ) + val client = monitorCtx.client + client!!.threadPool().threadContext.stashContext().use { + withClosableContext( + InjectorContextElement( + monitor.id, + monitorCtx.settings!!, + monitorCtx.threadPool!!.threadContext, + monitor.user?.roles, + monitor.user + ) + ) { + actionOutput[Action.MESSAGE_ID] = getConfigAndSendNotification( + action, + monitorCtx, + actionOutput[Action.SUBJECT], + actionOutput[Action.MESSAGE]!! + ) + } } } ActionRunResult(action.id, action.name, actionOutput, false, MonitorRunnerService.currentTime(), null) @@ -179,8 +186,4 @@ abstract class MonitorRunner { return NotificationActionConfigs(destination, channel) } - - protected fun createMonitorMetadata(monitorId: String): MonitorMetadata { - return MonitorMetadata("$monitorId-metadata", monitorId, emptyList(), emptyMap()) - } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt index 55624d66e..25bb42fc9 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerExecutionContext.kt @@ -13,6 +13,7 @@ import org.opensearch.alerting.settings.DestinationSettings import org.opensearch.alerting.settings.LegacyOpenDistroDestinationSettings 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.settings.Settings import org.opensearch.common.unit.TimeValue @@ -25,6 +26,7 @@ data class MonitorRunnerExecutionContext( var clusterService: ClusterService? = null, var client: Client? = null, var xContentRegistry: NamedXContentRegistry? = null, + var indexNameExpressionResolver: IndexNameExpressionResolver? = null, var scriptService: ScriptService? = null, var settings: Settings? = null, var threadPool: ThreadPool? = null, diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt index 7b2ec5116..31dfa445b 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/MonitorRunnerService.kt @@ -11,10 +11,13 @@ import kotlinx.coroutines.Job import kotlinx.coroutines.SupervisorJob import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager +import org.opensearch.action.ActionListener import org.opensearch.action.bulk.BackoffPolicy +import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.alerting.alerts.AlertIndices import org.opensearch.alerting.alerts.moveAlerts import org.opensearch.alerting.core.JobRunner +import org.opensearch.alerting.core.ScheduledJobIndices import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.model.Alert import org.opensearch.alerting.model.Monitor @@ -33,9 +36,11 @@ 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.alerting.util.IndexUtils import org.opensearch.alerting.util.isBucketLevelMonitor import org.opensearch.alerting.util.isDocLevelMonitor 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 @@ -72,6 +77,11 @@ object MonitorRunnerService : JobRunner, CoroutineScope, AbstractLifecycleCompon return this } + fun registerindexNameExpressionResolver(indexNameExpressionResolver: IndexNameExpressionResolver): MonitorRunnerService { + this.monitorCtx.indexNameExpressionResolver = indexNameExpressionResolver + return this + } + fun registerScriptService(scriptService: ScriptService): MonitorRunnerService { this.monitorCtx.scriptService = scriptService return this @@ -218,6 +228,23 @@ object MonitorRunnerService : JobRunner, CoroutineScope, AbstractLifecycleCompon } suspend fun runJob(job: ScheduledJob, periodStart: Instant, periodEnd: Instant, dryrun: Boolean): MonitorRunResult<*> { + + // Updating the scheduled job index at the start of monitor execution runs for when there is an upgrade the the schema mapping + // has not been updated. + if (!IndexUtils.scheduledJobIndexUpdated && monitorCtx.clusterService != null && monitorCtx.client != null) { + IndexUtils.updateIndexMapping( + ScheduledJob.SCHEDULED_JOBS_INDEX, + ScheduledJobIndices.scheduledJobMappings(), monitorCtx.clusterService!!.state(), monitorCtx.client!!.admin().indices(), + object : ActionListener { + override fun onResponse(response: AcknowledgedResponse) { + } + override fun onFailure(t: Exception) { + logger.error("Failed to update config index schema", t) + } + } + ) + } + val monitor = job as Monitor val runResult = if (monitor.isBucketLevelMonitor()) { BucketLevelMonitorRunner.runMonitor(monitor, monitorCtx, periodStart, periodEnd, dryrun) 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 9b37b3404..4a660d1c1 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/alerts/AlertIndices.kt @@ -6,6 +6,7 @@ package org.opensearch.alerting.alerts import org.apache.logging.log4j.LogManager +import org.opensearch.ExceptionsHelper import org.opensearch.ResourceAlreadyExistsException import org.opensearch.action.ActionListener import org.opensearch.action.admin.cluster.state.ClusterStateRequest @@ -36,6 +37,7 @@ import org.opensearch.alerting.settings.AlertingSettings.Companion.FINDING_HISTO import org.opensearch.alerting.settings.AlertingSettings.Companion.FINDING_HISTORY_RETENTION_PERIOD import org.opensearch.alerting.settings.AlertingSettings.Companion.FINDING_HISTORY_ROLLOVER_PERIOD import org.opensearch.alerting.settings.AlertingSettings.Companion.REQUEST_TIMEOUT +import org.opensearch.alerting.util.AlertingException import org.opensearch.alerting.util.IndexUtils import org.opensearch.client.Client import org.opensearch.cluster.ClusterChangedEvent @@ -293,8 +295,12 @@ class AlertIndices( return try { val createIndexResponse: CreateIndexResponse = client.admin().indices().suspendUntil { create(request, it) } createIndexResponse.isAcknowledged - } catch (e: ResourceAlreadyExistsException) { - true + } catch (t: Exception) { + if (ExceptionsHelper.unwrapCause(t) is ResourceAlreadyExistsException) { + true + } else { + throw AlertingException.wrap(t) + } } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/model/AlertingConfigAccessor.kt b/alerting/src/main/kotlin/org/opensearch/alerting/model/AlertingConfigAccessor.kt index d1d5411f6..903e6618e 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/model/AlertingConfigAccessor.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/model/AlertingConfigAccessor.kt @@ -18,10 +18,7 @@ import org.opensearch.common.bytes.BytesReference import org.opensearch.common.xcontent.LoggingDeprecationHandler import org.opensearch.common.xcontent.NamedXContentRegistry import org.opensearch.common.xcontent.XContentHelper -import org.opensearch.common.xcontent.XContentParser -import org.opensearch.common.xcontent.XContentParserUtils import org.opensearch.common.xcontent.XContentType -import org.opensearch.index.IndexNotFoundException /** * This is an accessor class to retrieve documents/information from the Alerting config index. @@ -29,28 +26,6 @@ import org.opensearch.index.IndexNotFoundException class AlertingConfigAccessor { companion object { - suspend fun getMonitorMetadata(client: Client, xContentRegistry: NamedXContentRegistry, metadataId: String): MonitorMetadata? { - return try { - val jobSource = getAlertingConfigDocumentSource(client, "Monitor Metadata", metadataId) - withContext(Dispatchers.IO) { - val xcp = XContentHelper.createParser( - xContentRegistry, LoggingDeprecationHandler.INSTANCE, - jobSource, XContentType.JSON - ) - XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.nextToken(), xcp) - MonitorMetadata.parse(xcp) - } - } catch (e: IllegalStateException) { - if (e.message?.equals("Monitor Metadata document with id $metadataId not found or source is empty") == true) { - return null - } else throw e - } catch (e: IndexNotFoundException) { - if (e.message?.equals("no such index [.opendistro-alerting-config]") == true) { - return null - } else throw e - } - } - suspend fun getEmailAccountInfo(client: Client, xContentRegistry: NamedXContentRegistry, emailAccountId: String): EmailAccount { val source = getAlertingConfigDocumentSource(client, "Email account", emailAccountId) return withContext(Dispatchers.IO) { 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 a78972a33..b98043461 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/model/MonitorMetadata.kt @@ -5,6 +5,7 @@ package org.opensearch.alerting.model +import org.opensearch.alerting.model.destination.Destination.Companion.NO_ID import org.opensearch.alerting.opensearchapi.instant import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput @@ -13,29 +14,40 @@ import org.opensearch.common.xcontent.ToXContent import org.opensearch.common.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentParser import org.opensearch.common.xcontent.XContentParserUtils +import org.opensearch.index.seqno.SequenceNumbers import java.io.IOException import java.time.Instant data class MonitorMetadata( val id: String, + val seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + val primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, val monitorId: String, val lastActionExecutionTimes: List, - val lastRunContext: Map + val lastRunContext: Map, + // Maps (sourceIndex + monitorId) --> concreteQueryIndex + val sourceToQueryIndexMapping: MutableMap = mutableMapOf() ) : Writeable, ToXContent { @Throws(IOException::class) constructor(sin: StreamInput) : this( id = sin.readString(), + seqNo = sin.readLong(), + primaryTerm = sin.readLong(), monitorId = sin.readString(), lastActionExecutionTimes = sin.readList(ActionExecutionTime::readFrom), - lastRunContext = Monitor.suppressWarning(sin.readMap()) + lastRunContext = Monitor.suppressWarning(sin.readMap()), + sourceToQueryIndexMapping = sin.readMap() as MutableMap ) override fun writeTo(out: StreamOutput) { out.writeString(id) + out.writeLong(seqNo) + out.writeLong(primaryTerm) out.writeString(monitorId) out.writeCollection(lastActionExecutionTimes) out.writeMap(lastRunContext) + out.writeMap(sourceToQueryIndexMapping as MutableMap) } override fun toXContent(builder: XContentBuilder, params: ToXContent.Params): XContentBuilder { @@ -44,6 +56,9 @@ data class MonitorMetadata( builder.field(MONITOR_ID_FIELD, monitorId) .field(LAST_ACTION_EXECUTION_FIELD, lastActionExecutionTimes.toTypedArray()) if (lastRunContext.isNotEmpty()) builder.field(LAST_RUN_CONTEXT_FIELD, lastRunContext) + if (sourceToQueryIndexMapping.isNotEmpty()) { + builder.field(SOURCE_TO_QUERY_INDEX_MAP_FIELD, sourceToQueryIndexMapping as MutableMap) + } if (params.paramAsBoolean("with_type", false)) builder.endObject() return builder.endObject() } @@ -53,13 +68,20 @@ data class MonitorMetadata( const val MONITOR_ID_FIELD = "monitor_id" const val LAST_ACTION_EXECUTION_FIELD = "last_action_execution_times" const val LAST_RUN_CONTEXT_FIELD = "last_run_context" + const val SOURCE_TO_QUERY_INDEX_MAP_FIELD = "source_to_query_index_mapping" @JvmStatic @JvmOverloads @Throws(IOException::class) - fun parse(xcp: XContentParser): MonitorMetadata { + fun parse( + xcp: XContentParser, + id: String = NO_ID, + seqNo: Long = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm: Long = SequenceNumbers.UNASSIGNED_PRIMARY_TERM + ): MonitorMetadata { lateinit var monitorId: String val lastActionExecutionTimes = mutableListOf() var lastRunContext: Map = mapOf() + var sourceToQueryIndexMapping: MutableMap = mutableMapOf() XContentParserUtils.ensureExpectedToken(XContentParser.Token.START_OBJECT, xcp.currentToken(), xcp) while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { @@ -75,14 +97,18 @@ data class MonitorMetadata( } } LAST_RUN_CONTEXT_FIELD -> lastRunContext = xcp.map() + SOURCE_TO_QUERY_INDEX_MAP_FIELD -> sourceToQueryIndexMapping = xcp.map() as MutableMap } } return MonitorMetadata( - "$monitorId-metadata", + if (id != NO_ID) id else "$monitorId-metadata", + seqNo = seqNo, + primaryTerm = primaryTerm, monitorId = monitorId, lastActionExecutionTimes = lastActionExecutionTimes, - lastRunContext = lastRunContext + lastRunContext = lastRunContext, + sourceToQueryIndexMapping = sourceToQueryIndexMapping ) } @@ -91,6 +117,10 @@ data class MonitorMetadata( fun readFrom(sin: StreamInput): MonitorMetadata { return MonitorMetadata(sin) } + + fun getId(monitor: Monitor): String { + return monitor.id + "-metadata" + } } } 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 b5ba20b02..f2ef4837d 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportDeleteMonitorAction.kt @@ -5,19 +5,32 @@ package org.opensearch.alerting.transport +import kotlinx.coroutines.CoroutineName +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.GlobalScope +import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchStatusException 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.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.master.AcknowledgedResponse +import org.opensearch.alerting.MonitorMetadataService import org.opensearch.alerting.action.DeleteMonitorAction import org.opensearch.alerting.action.DeleteMonitorRequest import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.model.Monitor +import org.opensearch.alerting.opensearchapi.suspendUntil import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.alerting.util.AlertingException import org.opensearch.client.Client @@ -34,9 +47,12 @@ 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 java.io.IOException +import kotlin.coroutines.resume +import kotlin.coroutines.resumeWithException +import kotlin.coroutines.suspendCoroutine private val log = LogManager.getLogger(TransportDeleteMonitorAction::class.java) @@ -66,7 +82,8 @@ class TransportDeleteMonitorAction @Inject constructor( if (!validateUserBackendRoles(user, actionListener)) { return } - client.threadPool().threadContext.stashContext().use { + + GlobalScope.launch(Dispatchers.IO + CoroutineName("DeleteMonitorAction")) { DeleteMonitorHandler(client, actionListener, deleteRequest, user, request.monitorId).resolveUserAndStart() } } @@ -78,120 +95,109 @@ class TransportDeleteMonitorAction @Inject constructor( private val user: User?, private val monitorId: String ) { - - fun resolveUserAndStart() { - if (user == null) { - // Security is disabled, so we can delete the destination without issues - deleteMonitor() - } else if (!doFilterForUser(user)) { - // security is enabled and filterby is disabled. - deleteMonitor() - } else { - try { - start() - } catch (ex: IOException) { - actionListener.onFailure(AlertingException.wrap(ex)) + suspend fun resolveUserAndStart() { + try { + val monitor = getMonitor() + + val canDelete = user == null || + !doFilterForUser(user) || + checkUserPermissionsWithResource(user, monitor.user, actionListener, "monitor", monitorId) + + if (canDelete) { + val deleteResponse = deleteMonitor(monitor) + deleteDocLevelMonitorQueriesAndIndices(monitor) + deleteMetadata(monitor) + actionListener.onResponse(deleteResponse) + } else { + actionListener.onFailure( + AlertingException("Not allowed to delete this monitor!", RestStatus.FORBIDDEN, IllegalStateException()) + ) } + } catch (t: Exception) { + actionListener.onFailure(AlertingException.wrap(t)) } } - fun start() { + private suspend fun getMonitor(): Monitor { val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, monitorId) - client.get( - getRequest, - object : ActionListener { - override fun onResponse(response: GetResponse) { - if (!response.isExists) { - actionListener.onFailure( - AlertingException.wrap( - OpenSearchStatusException("Monitor with $monitorId is not found", RestStatus.NOT_FOUND) - ) - ) - return - } - val xcp = XContentHelper.createParser( - xContentRegistry, LoggingDeprecationHandler.INSTANCE, - response.sourceAsBytesRef, XContentType.JSON - ) - val monitor = ScheduledJob.parse(xcp, response.id, response.version) as Monitor - onGetResponse(monitor) - } - override fun onFailure(t: Exception) { - actionListener.onFailure(AlertingException.wrap(t)) - } - } + + val getResponse: GetResponse = client.suspendUntil { get(getRequest, it) } + if (getResponse.isExists == false) { + actionListener.onFailure( + AlertingException.wrap( + OpenSearchStatusException("Monitor with $monitorId is 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 Monitor } - private fun onGetResponse(monitor: Monitor) { - if (!checkUserPermissionsWithResource(user, monitor.user, actionListener, "monitor", monitorId)) { - return - } else { - deleteMonitor() - } + private suspend fun deleteMonitor(monitor: Monitor): DeleteResponse { + return client.suspendUntil { delete(deleteRequest, it) } } - private fun deleteMonitor() { - client.delete( - deleteRequest, - object : ActionListener { - override fun onResponse(response: DeleteResponse) { - val clusterState = clusterService.state() - if (clusterState.routingTable.hasIndex(ScheduledJob.DOC_LEVEL_QUERIES_INDEX)) { - deleteDocLevelMonitorQueries() - } - deleteMetadata() - - actionListener.onResponse(response) - } + private suspend fun deleteMetadata(monitor: Monitor) { + val deleteRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, "${monitor.id}-metadata") + val deleteResponse: DeleteResponse = client.suspendUntil { delete(deleteRequest, it) } + } - override fun onFailure(t: Exception) { - actionListener.onFailure(AlertingException.wrap(t)) + private suspend fun deleteDocLevelMonitorQueriesAndIndices(monitor: Monitor) { + val clusterState = clusterService.state() + 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 } - ) - } - - private fun deleteMetadata() { - val getRequest = GetRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, monitorId) - client.get( - getRequest, - object : ActionListener { - override fun onResponse(response: GetResponse) { - if (response.isExists) { - val deleteMetadataRequest = DeleteRequest(ScheduledJob.SCHEDULED_JOBS_INDEX, "$monitorId") - .setRefreshPolicy(deleteRequest.refreshPolicy) - client.delete( - deleteMetadataRequest, - object : ActionListener { - override fun onResponse(response: DeleteResponse) { - } - - override fun onFailure(t: Exception) { - } + // 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) } ) - } - } - override fun onFailure(t: Exception) { } } - ) - } - - private fun deleteDocLevelMonitorQueries() { - DeleteByQueryRequestBuilder(client, DeleteByQueryAction.INSTANCE) - .source(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) - .filter(QueryBuilders.matchQuery("monitor_id", monitorId)) - .execute( - object : ActionListener { - override fun onResponse(response: BulkByScrollResponse) { - } - - override fun onFailure(t: Exception) { - } - } - ) + } } } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteMonitorAction.kt b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteMonitorAction.kt index 2ea064768..28ad82b8c 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteMonitorAction.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportExecuteMonitorAction.kt @@ -17,6 +17,7 @@ import org.opensearch.action.get.GetResponse import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.MonitorMetadataService import org.opensearch.alerting.MonitorRunnerService import org.opensearch.alerting.action.ExecuteMonitorAction import org.opensearch.alerting.action.ExecuteMonitorRequest @@ -129,9 +130,11 @@ class TransportExecuteMonitorAction @Inject constructor( docLevelMonitorQueries.initDocLevelQueryIndex() log.info("Central Percolation index ${ScheduledJob.DOC_LEVEL_QUERIES_INDEX} created") } + val (metadata, _) = MonitorMetadataService.getOrCreateMetadata(monitor, skipIndex = true) docLevelMonitorQueries.indexDocLevelQueries( monitor, monitor.id, + metadata, WriteRequest.RefreshPolicy.IMMEDIATE, indexTimeout ) 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 407d95a5b..f55b4c3d2 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/transport/TransportIndexMonitorAction.kt @@ -9,12 +9,16 @@ 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.OpenSearchSecurityException import org.opensearch.OpenSearchStatusException +import org.opensearch.ResourceAlreadyExistsException import org.opensearch.action.ActionListener +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.admin.indices.get.GetIndexRequest -import org.opensearch.action.admin.indices.get.GetIndexResponse import org.opensearch.action.get.GetRequest import org.opensearch.action.get.GetResponse import org.opensearch.action.index.IndexRequest @@ -25,7 +29,7 @@ import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction import org.opensearch.action.support.WriteRequest.RefreshPolicy import org.opensearch.action.support.master.AcknowledgedResponse -import org.opensearch.alerting.DocumentLevelMonitorRunner +import org.opensearch.alerting.MonitorMetadataService import org.opensearch.alerting.action.IndexMonitorAction import org.opensearch.alerting.action.IndexMonitorRequest import org.opensearch.alerting.action.IndexMonitorResponse @@ -35,7 +39,6 @@ import org.opensearch.alerting.core.model.DocLevelMonitorInput.Companion.DOC_LEV import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.core.model.ScheduledJob.Companion.SCHEDULED_JOBS_INDEX import org.opensearch.alerting.core.model.SearchInput -import org.opensearch.alerting.model.AlertingConfigAccessor.Companion.getMonitorMetadata import org.opensearch.alerting.model.Monitor import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.opensearchapi.suspendUntil @@ -254,10 +257,30 @@ class TransportIndexMonitorAction @Inject constructor( if (!scheduledJobIndices.scheduledJobIndexExists()) { scheduledJobIndices.initScheduledJobIndex(object : ActionListener { override fun onResponse(response: CreateIndexResponse) { - onCreateMappingsResponse(response) + onCreateMappingsResponse(response.isAcknowledged) } override fun onFailure(t: Exception) { - actionListener.onFailure(AlertingException.wrap(t)) + // 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) { + actionListener.onFailure( + OpenSearchException("Cannot determine that the $SCHEDULED_JOBS_INDEX index is healthy") + ) + } + // Retry mapping of monitor + onCreateMappingsResponse(true) + } + } else { + actionListener.onFailure(AlertingException.wrap(t)) + } } }) } else if (!IndexUtils.scheduledJobIndexUpdated) { @@ -303,6 +326,7 @@ class TransportIndexMonitorAction @Inject constructor( val query = QueryBuilders.boolQuery().filter(QueryBuilders.termQuery("${Monitor.MONITOR_TYPE}.type", Monitor.MONITOR_TYPE)) val searchSource = SearchSourceBuilder().query(query).timeout(requestTimeout) val searchRequest = SearchRequest(SCHEDULED_JOBS_INDEX).source(searchSource) + client.search( searchRequest, object : ActionListener { @@ -358,8 +382,8 @@ class TransportIndexMonitorAction @Inject constructor( } } - private fun onCreateMappingsResponse(response: CreateIndexResponse) { - if (response.isAcknowledged) { + private fun onCreateMappingsResponse(isAcknowledged: Boolean) { + if (isAcknowledged) { log.info("Created $SCHEDULED_JOBS_INDEX with mappings.") prepareMonitorIndexing() IndexUtils.scheduledJobIndexUpdated() @@ -394,8 +418,6 @@ class TransportIndexMonitorAction @Inject constructor( } private suspend fun indexMonitor() { - var metadata = createMetadata() - val indexRequest = IndexRequest(SCHEDULED_JOBS_INDEX) .setRefreshPolicy(request.refreshPolicy) .source(request.monitor.toXContentWithUser(jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) @@ -412,22 +434,16 @@ class TransportIndexMonitorAction @Inject constructor( ) return } - metadata = metadata.copy(monitorId = indexResponse.id, id = "${indexResponse.id}-metadata") - - // In case the metadata fails to be created, the monitor runner should have logic to recreate and index the metadata. - // This is currently being handled in DocumentLevelMonitor as its the only current monitor to use metadata currently. - // This should be enhanced by having a utility class to handle the logic of management and creation of the metadata. - // Issue to track this: https://github.com/opensearch-project/alerting/issues/445 - val metadataIndexRequest = IndexRequest(SCHEDULED_JOBS_INDEX) - .setRefreshPolicy(request.refreshPolicy) - .source(metadata.toXContent(jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) - .id(metadata.id) - .timeout(indexTimeout) - client.suspendUntil { client.index(metadataIndexRequest, it) } - + request.monitor = request.monitor.copy(id = indexResponse.id) + var (metadata, created) = MonitorMetadataService.getOrCreateMetadata(request.monitor) + if (created == false) { + log.warn("Metadata doc id:${metadata.id} exists, but it shouldn't!") + } if (request.monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { - indexDocLevelMonitorQueries(request.monitor, indexResponse.id, request.refreshPolicy) + indexDocLevelMonitorQueries(request.monitor, indexResponse.id, metadata, request.refreshPolicy) } + // When inserting queries in queryIndex we could update sourceToQueryIndexMapping + MonitorMetadataService.upsertMetadata(metadata, updating = true) actionListener.onResponse( IndexMonitorResponse( @@ -441,7 +457,12 @@ class TransportIndexMonitorAction @Inject constructor( } @Suppress("UNCHECKED_CAST") - private suspend fun indexDocLevelMonitorQueries(monitor: Monitor, monitorId: String, refreshPolicy: RefreshPolicy) { + private suspend fun indexDocLevelMonitorQueries( + monitor: Monitor, + monitorId: String, + monitorMetadata: MonitorMetadata, + refreshPolicy: RefreshPolicy + ) { if (!docLevelMonitorQueries.docLevelQueryIndexExists()) { docLevelMonitorQueries.initDocLevelQueryIndex() log.info("Central Percolation index ${ScheduledJob.DOC_LEVEL_QUERIES_INDEX} created") @@ -449,6 +470,7 @@ class TransportIndexMonitorAction @Inject constructor( docLevelMonitorQueries.indexDocLevelQueries( monitor, monitorId, + monitorMetadata, refreshPolicy, indexTimeout ) @@ -506,40 +528,20 @@ class TransportIndexMonitorAction @Inject constructor( ) return } - - val metadata = getMonitorMetadata(client, xContentRegistry, "${request.monitor.id}-metadata") - - if (metadata == null) { - val newMetadata = createMetadata() - val indexMetadataRequest = IndexRequest(SCHEDULED_JOBS_INDEX) - .setRefreshPolicy(request.refreshPolicy) - .source(newMetadata.toXContent(jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) - .id(newMetadata.id) - .timeout(indexTimeout) - client.suspendUntil { client.index(indexMetadataRequest, it) } - } else if (currentMonitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { - val monitorIndex = (request.monitor.inputs[0] as DocLevelMonitorInput).indices[0] - val runContext = createFullRunContext( - monitorIndex, - metadata.lastRunContext as MutableMap> - ) - val updatedMetadata = metadata.copy(lastRunContext = runContext) - val indexMetadataRequest = IndexRequest(SCHEDULED_JOBS_INDEX) - .setRefreshPolicy(request.refreshPolicy) - .source(updatedMetadata.toXContent(jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) - .id(metadata.id) - .timeout(indexTimeout) - client.suspendUntil { client.index(indexMetadataRequest, it) } - } - - if (currentMonitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { + var updatedMetadata: MonitorMetadata + val (metadata, created) = MonitorMetadataService.getOrCreateMetadata(request.monitor) + // Recreate runContext if metadata exists + // Delete and insert all queries from/to queryIndex + if (created == false && currentMonitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) { + updatedMetadata = MonitorMetadataService.recreateRunContext(metadata, currentMonitor) client.suspendUntil { DeleteByQueryRequestBuilder(client, DeleteByQueryAction.INSTANCE) .source(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) .filter(QueryBuilders.matchQuery("monitor_id", currentMonitor.id)) .execute(it) } - indexDocLevelMonitorQueries(request.monitor, currentMonitor.id, request.refreshPolicy) + indexDocLevelMonitorQueries(request.monitor, currentMonitor.id, updatedMetadata, request.refreshPolicy) + MonitorMetadataService.upsertMetadata(updatedMetadata, updating = true) } actionListener.onResponse( IndexMonitorResponse( @@ -552,34 +554,6 @@ class TransportIndexMonitorAction @Inject constructor( } } - private suspend fun createFullRunContext( - index: String?, - existingRunContext: MutableMap>? = null - ): MutableMap> { - if (index == null) return mutableMapOf() - val getIndexRequest = GetIndexRequest().indices(index) - val getIndexResponse: GetIndexResponse = client.suspendUntil { - client.admin().indices().getIndex(getIndexRequest, it) - } - val indices = getIndexResponse.indices() - val lastRunContext = existingRunContext?.toMutableMap() ?: mutableMapOf>() - indices.forEach { indexName -> - if (!lastRunContext.containsKey(indexName)) - lastRunContext[indexName] = DocumentLevelMonitorRunner.createRunContext(clusterService, client, indexName) - } - return lastRunContext - } - - private suspend fun createMetadata(): MonitorMetadata { - val monitorIndex = if (request.monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) - (request.monitor.inputs[0] as DocLevelMonitorInput).indices[0] - else null - val runContext = if (request.monitor.monitorType == Monitor.MonitorType.DOC_LEVEL_MONITOR) createFullRunContext(monitorIndex) - else emptyMap() - return MonitorMetadata("${request.monitorId}-metadata", request.monitorId, emptyList(), runContext) - } - private fun checkShardsFailure(response: IndexResponse): String? { val failureReasons = StringBuilder() if (response.shardInfo.failed > 0) { 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 fd44d525c..516266c59 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/util/AlertingUtils.kt @@ -6,25 +6,14 @@ package org.opensearch.alerting.util import org.apache.logging.log4j.LogManager -import org.opensearch.action.index.IndexRequest -import org.opensearch.action.index.IndexResponse -import org.opensearch.action.support.WriteRequest -import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.model.AggregationResultBucket import org.opensearch.alerting.model.BucketLevelTriggerRunResult import org.opensearch.alerting.model.Monitor -import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.model.action.Action import org.opensearch.alerting.model.action.ActionExecutionPolicy import org.opensearch.alerting.model.action.ActionExecutionScope import org.opensearch.alerting.model.destination.Destination -import org.opensearch.alerting.opensearchapi.suspendUntil -import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.alerting.settings.DestinationSettings -import org.opensearch.client.Client -import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentFactory private val logger = LogManager.getLogger("AlertingUtils") @@ -123,13 +112,3 @@ fun defaultToPerExecutionAction( return false } - -suspend fun updateMonitorMetadata(client: Client, settings: Settings, monitorMetadata: MonitorMetadata): IndexResponse { - val indexRequest = IndexRequest(ScheduledJob.SCHEDULED_JOBS_INDEX) - .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) - .source(monitorMetadata.toXContent(XContentFactory.jsonBuilder(), ToXContent.MapParams(mapOf("with_type" to "true")))) - .id(monitorMetadata.id) - .timeout(AlertingSettings.INDEX_TIMEOUT.get(settings)) - - return client.suspendUntil { client.index(indexRequest, it) } -} diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/util/DocLevelMonitorQueries.kt b/alerting/src/main/kotlin/org/opensearch/alerting/util/DocLevelMonitorQueries.kt index 87b364bfc..c3fb72577 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/util/DocLevelMonitorQueries.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/util/DocLevelMonitorQueries.kt @@ -6,50 +6,86 @@ package org.opensearch.alerting.util import org.apache.logging.log4j.LogManager +import org.opensearch.ExceptionsHelper +import org.opensearch.OpenSearchStatusException import org.opensearch.ResourceAlreadyExistsException +import org.opensearch.action.admin.indices.alias.Alias import org.opensearch.action.admin.indices.create.CreateIndexRequest import org.opensearch.action.admin.indices.create.CreateIndexResponse -import org.opensearch.action.admin.indices.get.GetIndexRequest -import org.opensearch.action.admin.indices.get.GetIndexResponse +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest +import org.opensearch.action.admin.indices.rollover.RolloverRequest +import org.opensearch.action.admin.indices.rollover.RolloverResponse +import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest +import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse +import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest import org.opensearch.action.bulk.BulkRequest import org.opensearch.action.bulk.BulkResponse import org.opensearch.action.index.IndexRequest import org.opensearch.action.support.WriteRequest.RefreshPolicy import org.opensearch.action.support.master.AcknowledgedResponse +import org.opensearch.alerting.MonitorRunnerService.monitorCtx import org.opensearch.alerting.core.model.DocLevelMonitorInput import org.opensearch.alerting.core.model.DocLevelQuery import org.opensearch.alerting.core.model.ScheduledJob import org.opensearch.alerting.model.Monitor +import org.opensearch.alerting.model.MonitorMetadata import org.opensearch.alerting.opensearchapi.suspendUntil 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.XContentType +import org.opensearch.index.mapper.MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING +import org.opensearch.rest.RestStatus private val log = LogManager.getLogger(DocLevelMonitorQueries::class.java) class DocLevelMonitorQueries(private val client: Client, private val clusterService: ClusterService) { companion object { + + const val PROPERTIES = "properties" + const val NESTED = "nested" + const val TYPE = "type" + const val INDEX_PATTERN_SUFFIX = "-000001" + const val QUERY_INDEX_BASE_FIELDS_COUNT = 8 // 3 fields we defined and 5 builtin additional metadata fields @JvmStatic fun docLevelQueriesMappings(): String { return DocLevelMonitorQueries::class.java.classLoader.getResource("mappings/doc-level-queries.json").readText() } + fun docLevelQueriesSettings(): Settings { + return Settings.builder().loadFromSource( + DocLevelMonitorQueries::class.java.classLoader.getResource("settings/doc-level-queries.json").readText(), + XContentType.JSON + ).build() + } } suspend fun initDocLevelQueryIndex(): Boolean { if (!docLevelQueryIndexExists()) { - val indexRequest = CreateIndexRequest(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) + // Since we changed queryIndex to be alias now, for backwards compatibility, we have to delete index with same name + // as our alias, to avoid name clash. + if (clusterService.state().metadata.hasIndex(ScheduledJob.DOC_LEVEL_QUERIES_INDEX)) { + val acknowledgedResponse: AcknowledgedResponse = client.suspendUntil { + admin().indices().delete(DeleteIndexRequest(ScheduledJob.DOC_LEVEL_QUERIES_INDEX), it) + } + if (!acknowledgedResponse.isAcknowledged) { + val errorMessage = "Deletion of old queryIndex [${ScheduledJob.DOC_LEVEL_QUERIES_INDEX}] index is not acknowledged!" + log.error(errorMessage) + throw AlertingException.wrap(OpenSearchStatusException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR)) + } + } + val alias = ScheduledJob.DOC_LEVEL_QUERIES_INDEX + val indexPattern = ScheduledJob.DOC_LEVEL_QUERIES_INDEX + INDEX_PATTERN_SUFFIX + val indexRequest = CreateIndexRequest(indexPattern) .mapping(docLevelQueriesMappings()) - .settings( - Settings.builder().put("index.hidden", true) - .build() - ) + .alias(Alias(alias)) + .settings(docLevelQueriesSettings()) return try { val createIndexResponse: CreateIndexResponse = client.suspendUntil { client.admin().indices().create(indexRequest, it) } createIndexResponse.isAcknowledged - } catch (t: ResourceAlreadyExistsException) { - if (t.message?.contains("already exists") == true) { + } catch (t: Exception) { + if (ExceptionsHelper.unwrapCause(t) is ResourceAlreadyExistsException) { true } else { throw t @@ -61,80 +97,284 @@ class DocLevelMonitorQueries(private val client: Client, private val clusterServ fun docLevelQueryIndexExists(): Boolean { val clusterState = clusterService.state() - return clusterState.routingTable.hasIndex(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) + return clusterState.metadata.hasAlias(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) + } + + /** + * Does a DFS traversal of index mappings tree. + * Calls processLeafFn on every leaf node. + * Populates flattenPaths list with full paths of leaf nodes + * @param node current node which we're visiting + * @param currentPath current node path from root node + * @param processLeafFn leaf processor function which is called on every leaf discovered + * @param flattenPaths list of full paths of all leaf nodes relative to root + */ + fun traverseMappingsAndUpdate( + node: MutableMap, + currentPath: String, + processLeafFn: (String, MutableMap) -> Triple>, + flattenPaths: MutableList + ) { + // If node contains "properties" property then it is internal(non-leaf) node + log.debug("Node in traverse: $node") + // newNodes will hold list of updated leaf properties + var newNodes = ArrayList>(node.size) + node.entries.forEach { + // Compute full path relative to root + val fullPath = if (currentPath.isEmpty()) it.key + else "$currentPath.${it.key}" + val nodeProps = it.value as MutableMap + // If it has type property and type is not "nested" then this is a leaf + if (nodeProps.containsKey(TYPE) && nodeProps[TYPE] != NESTED) { + // At this point we know full path of node, so we add it to output array + flattenPaths.add(fullPath) + // Calls processLeafFn and gets old node name, new node name and new properties of node. + // This is all information we need to update this node + val (oldName, newName, props) = processLeafFn(it.key, it.value as MutableMap) + newNodes.add(Triple(oldName, newName, props)) + } else { + // Internal(non-leaf) node - visit children + traverseMappingsAndUpdate(nodeProps[PROPERTIES] as MutableMap, fullPath, processLeafFn, flattenPaths) + } + } + // Here we can update all processed leaves in tree + newNodes.forEach { + // If we renamed leaf, we have to remove it first + if (it.first != it.second) { + node.remove(it.first) + } + // Put new properties of leaf + node.put(it.second, it.third) + } } suspend fun indexDocLevelQueries( monitor: Monitor, monitorId: String, + monitorMetadata: MonitorMetadata, refreshPolicy: RefreshPolicy = RefreshPolicy.IMMEDIATE, indexTimeout: TimeValue ) { val docLevelMonitorInput = monitor.inputs[0] as DocLevelMonitorInput - val index = docLevelMonitorInput.indices[0] val queries: List = docLevelMonitorInput.queries - val clusterState = clusterService.state() + val indices = IndexUtils.resolveAllIndices( + docLevelMonitorInput.indices, + monitorCtx.clusterService!!, + monitorCtx.indexNameExpressionResolver!! + ) - val getIndexRequest = GetIndexRequest().indices(index) - val getIndexResponse: GetIndexResponse = client.suspendUntil { - client.admin().indices().getIndex(getIndexRequest, it) - } - val indices = getIndexResponse.indices() + val clusterState = clusterService.state() + // Run through each backing index and apply appropriate mappings to query index indices?.forEach { indexName -> if (clusterState.routingTable.hasIndex(indexName)) { val indexMetadata = clusterState.metadata.index(indexName) if (indexMetadata.mapping()?.sourceAsMap?.get("properties") != null) { val properties = ( (indexMetadata.mapping()?.sourceAsMap?.get("properties")) - as Map> + as MutableMap ) - - val updatedProperties = properties.entries.associate { - if (it.value.containsKey("path")) { - val newVal = it.value.toMutableMap() - newVal["path"] = "${it.value["path"]}_${indexName}_$monitorId" - "${it.key}_${indexName}_$monitorId" to newVal - } else { - "${it.key}_${indexName}_$monitorId" to it.value + // Node processor function is used to process leaves of index mappings tree + // + val leafNodeProcessor = + fun(fieldName: String, props: MutableMap): Triple> { + val newProps = props.toMutableMap() + if (props.containsKey("path")) { + newProps["path"] = "${props["path"]}_${indexName}_$monitorId" + } + return Triple(fieldName, "${fieldName}_${indexName}_$monitorId", newProps) } - } + // Traverse and update index mappings here while extracting flatten field paths + val flattenPaths = mutableListOf() + traverseMappingsAndUpdate(properties, "", leafNodeProcessor, flattenPaths) + // Updated mappings ready to be applied on queryIndex + val updatedProperties = properties + // Updates mappings of concrete queryIndex. This can rollover queryIndex if field mapping limit is reached. + var (updateMappingResponse, concreteQueryIndex) = updateQueryIndexMappings( + monitor, + monitorMetadata, + indexName, + updatedProperties + ) - val updateMappingRequest = PutMappingRequest(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) - updateMappingRequest.source(mapOf("properties" to updatedProperties)) - val updateMappingResponse: AcknowledgedResponse = client.suspendUntil { - client.admin().indices().putMapping(updateMappingRequest, it) + if (updateMappingResponse.isAcknowledged) { + doIndexAllQueries(concreteQueryIndex, indexName, monitorId, queries, flattenPaths, refreshPolicy, indexTimeout) } + } + } + } + } - if (updateMappingResponse.isAcknowledged) { - val indexRequests = mutableListOf() - queries.forEach { - var query = it.query - properties.forEach { prop -> - query = query.replace("${prop.key}:", "${prop.key}_${indexName}_$monitorId:") - } - val indexRequest = IndexRequest(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) - .id(it.id + "_${indexName}_$monitorId") - .source( - mapOf( - "query" to mapOf("query_string" to mapOf("query" to query)), - "monitor_id" to monitorId, - "index" to indexName - ) - ) - indexRequests.add(indexRequest) - } - if (indexRequests.isNotEmpty()) { - val bulkResponse: BulkResponse = client.suspendUntil { - client.bulk( - BulkRequest().setRefreshPolicy(refreshPolicy).timeout(indexTimeout).add(indexRequests), it - ) - } - } + private suspend fun doIndexAllQueries( + concreteQueryIndex: String, + sourceIndex: String, + monitorId: String, + queries: List, + flattenPaths: MutableList, + refreshPolicy: RefreshPolicy, + indexTimeout: TimeValue + ) { + val indexRequests = mutableListOf() + queries.forEach { + var query = it.query + flattenPaths.forEach { fieldPath -> + query = query.replace("$fieldPath:", "${fieldPath}_${sourceIndex}_$monitorId:") + } + val indexRequest = IndexRequest(concreteQueryIndex) + .id(it.id + "_${sourceIndex}_$monitorId") + .source( + mapOf( + "query" to mapOf("query_string" to mapOf("query" to query)), + "monitor_id" to monitorId, + "index" to sourceIndex + ) + ) + indexRequests.add(indexRequest) + } + if (indexRequests.isNotEmpty()) { + val bulkResponse: BulkResponse = client.suspendUntil { + client.bulk( + BulkRequest().setRefreshPolicy(refreshPolicy).timeout(indexTimeout).add(indexRequests), it + ) + } + bulkResponse.forEach { bulkItemResponse -> + if (bulkItemResponse.isFailed) { + log.debug(bulkItemResponse.failureMessage) + } + } + } + } + + private suspend fun updateQueryIndexMappings( + monitor: Monitor, + monitorMetadata: MonitorMetadata, + sourceIndex: String, + updatedProperties: MutableMap + ): Pair { + var targetQueryIndex = monitorMetadata.sourceToQueryIndexMapping[sourceIndex + monitor.id] + if (targetQueryIndex == null) { + // queryIndex is alias which will always have only 1 backing index which is writeIndex + // This is due to a fact that _rollover API would maintain only single index under alias + // if you don't add is_write_index setting when creating index initially + targetQueryIndex = getWriteIndexNameForAlias(ScheduledJob.DOC_LEVEL_QUERIES_INDEX) + if (targetQueryIndex == null) { + val message = "Failed to get write index for queryIndex alias:${ScheduledJob.DOC_LEVEL_QUERIES_INDEX}" + log.error(message) + throw AlertingException.wrap( + OpenSearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR) + ) + } + monitorMetadata.sourceToQueryIndexMapping[sourceIndex + monitor.id] = targetQueryIndex + } + val updateMappingRequest = PutMappingRequest(targetQueryIndex) + updateMappingRequest.source(mapOf("properties" to updatedProperties)) + var updateMappingResponse = AcknowledgedResponse(false) + try { + // Adjust max field limit in mappings for query index, if needed. + checkAndAdjustMaxFieldLimit(sourceIndex, targetQueryIndex) + updateMappingResponse = client.suspendUntil { + client.admin().indices().putMapping(updateMappingRequest, it) + } + return Pair(updateMappingResponse, targetQueryIndex) + } catch (e: Exception) { + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + log.debug("exception after rollover queryIndex index: $targetQueryIndex exception: ${unwrappedException.message}") + // If we reached limit for total number of fields in mappings, do a rollover here + if (unwrappedException.message?.contains("Limit of total fields") == true) { + try { + // Do queryIndex rollover + targetQueryIndex = rolloverQueryIndex(monitor) + // Adjust max field limit in mappings for new index. + checkAndAdjustMaxFieldLimit(sourceIndex, targetQueryIndex) + // PUT mappings to newly created index + val updateMappingRequestRetry = PutMappingRequest(targetQueryIndex) + updateMappingRequestRetry.source(mapOf("properties" to updatedProperties)) + updateMappingResponse = client.suspendUntil { + client.admin().indices().putMapping(updateMappingRequestRetry, it) + } + } catch (e: Exception) { + // If we reached limit for total number of fields in mappings after rollover + // it means that source index has more then (FIELD_LIMIT - 3) fields (every query index has 3 fields defined) + // TODO maybe split queries/mappings between multiple query indices? + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + log.debug("exception after rollover queryIndex index: $targetQueryIndex exception: ${unwrappedException.message}") + if (unwrappedException.message?.contains("Limit of total fields") == true) { + val errorMessage = + "Monitor [${monitorMetadata.monitorId}] can't process index [$sourceIndex] due to field mapping limit" + log.error(errorMessage) + throw AlertingException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR, e) + } else { + throw AlertingException.wrap(e) } } + } else { + log.debug("unknown exception during PUT mapping on queryIndex: $targetQueryIndex") + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + throw AlertingException.wrap(unwrappedException) } } + // We did rollover, so try to apply mappings again on new targetQueryIndex + if (targetQueryIndex.isNotEmpty()) { + // add newly created index to monitor's metadata object so that we can fetch it later on, when either applying mappings or running queries + monitorMetadata.sourceToQueryIndexMapping[sourceIndex + monitor.id] = targetQueryIndex + } else { + val failureMessage = "Failed to resolve targetQueryIndex!" + log.error(failureMessage) + throw AlertingException(failureMessage, RestStatus.INTERNAL_SERVER_ERROR, IllegalStateException(failureMessage)) + } + return Pair(updateMappingResponse, targetQueryIndex) + } + + /** + * Adjusts max field limit index setting for query index if source index has higher limit. + * This will prevent max field limit exception, when source index has more fields then query index limit + */ + private suspend fun checkAndAdjustMaxFieldLimit(sourceIndex: String, concreteQueryIndex: String) { + val getSettingsResponse: GetSettingsResponse = client.suspendUntil { + admin().indices().getSettings(GetSettingsRequest().indices(sourceIndex, concreteQueryIndex), it) + } + val sourceIndexLimit = + getSettingsResponse.getSetting(sourceIndex, INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.key)?.toLong() ?: 1000L + val queryIndexLimit = + getSettingsResponse.getSetting(concreteQueryIndex, INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.key)?.toLong() ?: 1000L + // Our query index initially has 3 fields we defined and 5 more builtin metadata fields in mappings so we have to account for that + if (sourceIndexLimit > (queryIndexLimit - QUERY_INDEX_BASE_FIELDS_COUNT)) { + val updateSettingsResponse: AcknowledgedResponse = client.suspendUntil { + admin().indices().updateSettings( + UpdateSettingsRequest(concreteQueryIndex).settings( + Settings.builder().put( + INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.key, sourceIndexLimit + QUERY_INDEX_BASE_FIELDS_COUNT + ) + ), + it + ) + } + } + } + + private suspend fun rolloverQueryIndex(monitor: Monitor): String { + val queryIndex = ScheduledJob.DOC_LEVEL_QUERIES_INDEX + val queryIndexPattern = queryIndex + INDEX_PATTERN_SUFFIX + + val request = RolloverRequest(queryIndex, null) + request.createIndexRequest.index(queryIndexPattern) + .mapping(docLevelQueriesMappings()) + .settings(docLevelQueriesSettings()) + val response: RolloverResponse = client.suspendUntil { + client.admin().indices().rolloverIndex(request, it) + } + if (response.isRolledOver == false) { + val message = "failed to rollover queryIndex:$queryIndex queryIndexPattern:$queryIndexPattern" + log.error(message) + throw AlertingException.wrap( + OpenSearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR) + ) + } + return response.newIndex + } + + private fun getWriteIndexNameForAlias(alias: String): String? { + return this.clusterService.state().metadata().indicesLookup?.get(alias)?.writeIndex?.index?.name } } diff --git a/alerting/src/main/kotlin/org/opensearch/alerting/util/IndexUtils.kt b/alerting/src/main/kotlin/org/opensearch/alerting/util/IndexUtils.kt index 9f299e8c5..39b5c8f15 100644 --- a/alerting/src/main/kotlin/org/opensearch/alerting/util/IndexUtils.kt +++ b/alerting/src/main/kotlin/org/opensearch/alerting/util/IndexUtils.kt @@ -7,16 +7,20 @@ package org.opensearch.alerting.util import org.opensearch.action.ActionListener import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest +import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.alerting.alerts.AlertIndices import org.opensearch.alerting.core.ScheduledJobIndices import org.opensearch.client.IndicesAdminClient import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexMetadata +import org.opensearch.cluster.metadata.IndexNameExpressionResolver +import org.opensearch.cluster.service.ClusterService import org.opensearch.common.xcontent.LoggingDeprecationHandler import org.opensearch.common.xcontent.NamedXContentRegistry import org.opensearch.common.xcontent.XContentParser import org.opensearch.common.xcontent.XContentType +import org.opensearch.index.IndexNotFoundException class IndexUtils { @@ -130,5 +134,26 @@ class IndexUtils { } } } + + @JvmStatic + fun resolveAllIndices(indices: List, clusterService: ClusterService, resolver: IndexNameExpressionResolver): List { + val result = mutableListOf() + + indices.forEach { index -> + val concreteIndices = resolver.concreteIndexNames( + clusterService.state(), + IndicesOptions.lenientExpand(), + true, + index + ) + result.addAll(concreteIndices) + } + + if (result.size == 0) { + throw IndexNotFoundException(indices[0]) + } + + return result + } } } diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/AlertingRestTestCase.kt b/alerting/src/test/kotlin/org/opensearch/alerting/AlertingRestTestCase.kt index 5cb99d04c..010d1d47c 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/AlertingRestTestCase.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/AlertingRestTestCase.kt @@ -74,7 +74,6 @@ import javax.management.MBeanServerInvocationHandler import javax.management.ObjectName import javax.management.remote.JMXConnectorFactory import javax.management.remote.JMXServiceURL -import kotlin.collections.HashMap abstract class AlertingRestTestCase : ODFERestTestCase() { @@ -652,7 +651,7 @@ abstract class AlertingRestTestCase : ODFERestTestCase() { } protected fun refreshIndex(index: String): Response { - val response = client().makeRequest("POST", "/$index/_refresh") + val response = client().makeRequest("POST", "/$index/_refresh?expand_wildcards=all") assertEquals("Unable to refresh index", RestStatus.OK, response.restStatus()) return response } @@ -1135,10 +1134,11 @@ abstract class AlertingRestTestCase : ODFERestTestCase() { client().performRequest(request) } - fun createIndexRoleWithDocLevelSecurity(name: String, index: String, dlsQuery: String) { + fun createIndexRoleWithDocLevelSecurity(name: String, index: String, dlsQuery: String, clusterPermissions: String? = "") { val request = Request("PUT", "/_plugins/_security/api/roles/$name") var entity = "{\n" + "\"cluster_permissions\": [\n" + + "\"$clusterPermissions\"\n" + "],\n" + "\"index_permissions\": [\n" + "{\n" + @@ -1199,10 +1199,10 @@ abstract class AlertingRestTestCase : ODFERestTestCase() { user: String, index: String, role: String, - backendRole: String, + backendRoles: List, clusterPermissions: String? ) { - createUser(user, user, arrayOf(backendRole)) + createUser(user, user, backendRoles.toTypedArray()) createTestIndex(index) createCustomIndexRole(role, index, clusterPermissions) createUserRolesMapping(role, arrayOf(user)) diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/DocumentMonitorRunnerIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/DocumentMonitorRunnerIT.kt index 86b90095f..88a562910 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/DocumentMonitorRunnerIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/DocumentMonitorRunnerIT.kt @@ -9,6 +9,7 @@ import org.opensearch.alerting.alerts.AlertIndices.Companion.ALL_ALERT_INDEX_PAT import org.opensearch.alerting.alerts.AlertIndices.Companion.ALL_FINDING_INDEX_PATTERN import org.opensearch.alerting.core.model.DocLevelMonitorInput import org.opensearch.alerting.core.model.DocLevelQuery +import org.opensearch.alerting.model.Alert import org.opensearch.alerting.model.action.ActionExecutionPolicy import org.opensearch.alerting.model.action.AlertCategory import org.opensearch.alerting.model.action.PerAlertActionScope @@ -19,6 +20,7 @@ import org.opensearch.script.Script import java.time.ZonedDateTime import java.time.format.DateTimeFormatter import java.time.temporal.ChronoUnit.MILLIS +import java.util.Locale class DocumentMonitorRunnerIT : AlertingRestTestCase() { @@ -179,6 +181,37 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { assertTrue("Findings saved for test monitor", findings[1].relatedDocIds.contains("5")) } + fun `test execute monitor input error`() { + val testIndex = createTestIndex() + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3", tags = listOf("test_tag")) + val docLevelInput = DocLevelMonitorInput("description", listOf(testIndex), listOf(docQuery)) + + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val monitor = createMonitor(randomDocumentLevelMonitor(inputs = listOf(docLevelInput), triggers = listOf(trigger))) + assertNotNull(monitor.id) + + deleteIndex(testIndex) + + val response = executeMonitor(monitor.id) + + val output = entityAsMap(response) + assertEquals(monitor.name, output["monitor_name"]) + @Suppress("UNCHECKED_CAST") + val inputResults = output.stringMap("input_results") + assertTrue("Missing monitor error message", (inputResults?.get("error") as String).isNotEmpty()) + + val alerts = searchAlerts(monitor) + assertEquals("Alert not saved", 1, alerts.size) + assertEquals("Alert status is incorrect", Alert.State.ERROR, alerts[0].state) + } + fun `test execute monitor generates alerts and findings with per alert execution for actions`() { val testIndex = createTestIndex() val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) @@ -236,6 +269,8 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { } } + refreshAllIndices() + val alerts = searchAlertsWithFilter(monitor) assertEquals("Alert saved for test monitor", 2, alerts.size) @@ -308,9 +343,12 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { assertTrue("Findings saved for test monitor", findings[1].relatedDocIds.contains("5")) } - fun `test execute monitor with wildcard index that generates alerts and findings`() { - val testIndex = createTestIndex("test1") - val testIndex2 = createTestIndex("test2") + fun `test execute monitor with wildcard index that generates alerts and findings for EQUALS query operator`() { + val testIndexPrefix = "test-index-${randomAlphaOfLength(10).lowercase(Locale.ROOT)}" + val testQueryName = "wildcard-test-query" + val testIndex = createTestIndex("${testIndexPrefix}1") + val testIndex2 = createTestIndex("${testIndexPrefix}2") + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) val testDoc = """{ "message" : "This is an error from IAD region", @@ -318,10 +356,10 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { "test_field" : "us-west-2" }""" - val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") - val docLevelInput = DocLevelMonitorInput("description", listOf("test*"), listOf(docQuery)) + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = testQueryName) + val docLevelInput = DocLevelMonitorInput("description", listOf("$testIndexPrefix*"), listOf(docQuery)) - val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + val trigger = randomDocumentLevelTrigger(condition = Script("query[name=$testQueryName]")) val monitor = createMonitor(randomDocumentLevelMonitor(inputs = listOf(docLevelInput), triggers = listOf(trigger))) assertNotNull(monitor.id) @@ -345,8 +383,52 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { val findings = searchFindings(monitor) assertEquals("Findings saved for test monitor", 2, findings.size) - assertTrue("Findings saved for test monitor", findings[0].relatedDocIds.contains("1")) - assertTrue("Findings saved for test monitor", findings[1].relatedDocIds.contains("5")) + val foundFindings = findings.filter { it.relatedDocIds.contains("1") || it.relatedDocIds.contains("5") } + assertEquals("Didn't find findings for docs 1 and 5", 2, foundFindings.size) + } + + fun `test execute monitor with wildcard index that generates alerts and findings for NOT EQUALS query operator`() { + val testIndexPrefix = "test-index-${randomAlphaOfLength(10).lowercase(Locale.ROOT)}" + val testQueryName = "wildcard-test-query" + val testIndex = createTestIndex("${testIndexPrefix}1") + val testIndex2 = createTestIndex("${testIndexPrefix}2") + + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + + val docQuery = DocLevelQuery(query = "NOT (test_field:\"us-west-1\")", name = testQueryName) + val docLevelInput = DocLevelMonitorInput("description", listOf("$testIndexPrefix*"), listOf(docQuery)) + + val trigger = randomDocumentLevelTrigger(condition = Script("query[name=$testQueryName]")) + val monitor = createMonitor(randomDocumentLevelMonitor(inputs = listOf(docLevelInput), triggers = listOf(trigger))) + assertNotNull(monitor.id) + + indexDoc(testIndex, "1", testDoc) + indexDoc(testIndex2, "5", testDoc) + + val response = executeMonitor(monitor.id) + + val output = entityAsMap(response) + + assertEquals(monitor.name, output["monitor_name"]) + @Suppress("UNCHECKED_CAST") + val searchResult = (output.objectMap("input_results")["results"] as List>).first() + @Suppress("UNCHECKED_CAST") + val matchingDocsToQuery = searchResult[docQuery.id] as List + assertEquals("Incorrect search result", 2, matchingDocsToQuery.size) + assertTrue("Incorrect search result", matchingDocsToQuery.containsAll(listOf("1|$testIndex", "5|$testIndex2"))) + + val alerts = searchAlertsWithFilter(monitor) + assertEquals("Alert saved for test monitor", 2, alerts.size) + + val findings = searchFindings(monitor) + assertEquals("Findings saved for test monitor", 2, findings.size) + val foundFindings = findings.filter { it.relatedDocIds.contains("1") || it.relatedDocIds.contains("5") } + assertEquals("Didn't find findings for docs 1 and 5", 2, foundFindings.size) } fun `test execute monitor with new index added after first execution that generates alerts and findings`() { @@ -375,14 +457,9 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { var findings = searchFindings(monitor) assertEquals("Findings saved for test monitor", 2, findings.size) - assertTrue( - "Findings saved for test monitor expected 1 instead of ${findings[0].relatedDocIds}", - findings[0].relatedDocIds.contains("1") - ) - assertTrue( - "Findings saved for test monitor expected 51 instead of ${findings[1].relatedDocIds}", - findings[1].relatedDocIds.contains("5") - ) + + var foundFindings = findings.filter { it.relatedDocIds.contains("1") || it.relatedDocIds.contains("5") } + assertEquals("Findings saved for test monitor expected 1 and 5", 2, foundFindings.size) // clear previous findings and alerts deleteIndex(ALL_FINDING_INDEX_PATTERN) @@ -410,18 +487,11 @@ class DocumentMonitorRunnerIT : AlertingRestTestCase() { findings = searchFindings(monitor) assertEquals("Findings saved for test monitor", 3, findings.size) - assertTrue( - "Findings saved for test monitor expected 14 instead of ${findings[0].relatedDocIds}", - findings[0].relatedDocIds.contains("14") - ) - assertTrue( - "Findings saved for test monitor expected 51 instead of ${findings[1].relatedDocIds}", - findings[1].relatedDocIds.contains("51") - ) - assertTrue( - "Findings saved for test monitor expected 10 instead of ${findings[2].relatedDocIds}", - findings[2].relatedDocIds.contains("10") - ) + + foundFindings = findings.filter { + it.relatedDocIds.contains("14") || it.relatedDocIds.contains("51") || it.relatedDocIds.contains("10") + } + assertEquals("Findings saved for test monitor expected 14, 51 and 10", 3, foundFindings.size) } fun `test document-level monitor when alias only has write index with 0 docs`() { diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/MonitorDataSourcesIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/MonitorDataSourcesIT.kt new file mode 100644 index 000000000..47b3c1ee5 --- /dev/null +++ b/alerting/src/test/kotlin/org/opensearch/alerting/MonitorDataSourcesIT.kt @@ -0,0 +1,1173 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.alerting + +import org.junit.Assert +import org.opensearch.action.admin.indices.create.CreateIndexRequest +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest +import org.opensearch.action.admin.indices.get.GetIndexRequest +import org.opensearch.action.admin.indices.get.GetIndexResponse +import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest +import org.opensearch.action.admin.indices.refresh.RefreshRequest +import org.opensearch.action.fieldcaps.FieldCapabilitiesRequest +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.action.DeleteMonitorAction +import org.opensearch.alerting.action.DeleteMonitorRequest +import org.opensearch.alerting.action.GetAlertsAction +import org.opensearch.alerting.action.GetAlertsRequest +import org.opensearch.alerting.alerts.AlertIndices +import org.opensearch.alerting.alerts.AlertIndices.Companion.ALL_FINDING_INDEX_PATTERN +import org.opensearch.alerting.core.ScheduledJobIndices +import org.opensearch.alerting.core.model.DocLevelMonitorInput +import org.opensearch.alerting.core.model.DocLevelQuery +import org.opensearch.alerting.core.model.ScheduledJob +import org.opensearch.alerting.core.model.ScheduledJob.Companion.DOC_LEVEL_QUERIES_INDEX +import org.opensearch.alerting.core.model.ScheduledJob.Companion.SCHEDULED_JOBS_INDEX +import org.opensearch.alerting.model.DocumentLevelTriggerRunResult +import org.opensearch.alerting.model.Table +import org.opensearch.alerting.transport.AlertingSingleNodeTestCase +import org.opensearch.alerting.util.DocLevelMonitorQueries +import org.opensearch.common.settings.Settings +import org.opensearch.common.xcontent.XContentType +import org.opensearch.index.mapper.MapperService +import org.opensearch.index.query.QueryBuilders +import org.opensearch.search.builder.SearchSourceBuilder +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter +import java.time.temporal.ChronoUnit.MILLIS + +/** + * For 2.3 this was backported and some of the tests are not consistent + * due to missing features launched in newer versions + */ +class MonitorDataSourcesIT : AlertingSingleNodeTestCase() { + + fun `test execute monitor with dryrun`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + indexDoc(index, "1", testDoc) + val id = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, id, true) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + val table = Table("asc", "id", null, 1, 0, "") + var getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", null)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 0) + getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", id)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 0) + } + + fun `test mappings parsing`() { + + val index1 = "index_123" + val index2 = "index_456" + val index3 = "index_789" + val index4 = "index_012" + val q1 = DocLevelQuery(query = "properties:\"abcd\"", name = "1") + val q2 = DocLevelQuery(query = "type.properties:\"abcd\"", name = "2") + val q3 = DocLevelQuery(query = "type.something.properties:\"abcd\"", name = "3") + val q4 = DocLevelQuery(query = "type.something.properties.lastone:\"abcd\"", name = "4") + + createIndex(index1, Settings.EMPTY) + createIndex(index2, Settings.EMPTY) + createIndex(index3, Settings.EMPTY) + createIndex(index4, Settings.EMPTY) + + val m1 = """{ + "properties": { + "properties": { + "type": "keyword" + } + } + } + """.trimIndent() + client().admin().indices().putMapping(PutMappingRequest(index1).source(m1, XContentType.JSON)).get() + + val m2 = """{ + "properties": { + "type": { + "properties": { + "properties": { "type": "keyword" } + } + } + } + } + """.trimIndent() + client().admin().indices().putMapping(PutMappingRequest(index2).source(m2, XContentType.JSON)).get() + + val m3 = """{ + "properties": { + "type": { + "properties": { + "something": { + "properties" : { + "properties": { "type": "keyword" } + } + } + } + } + } + } + """.trimIndent() + client().admin().indices().putMapping(PutMappingRequest(index3).source(m3, XContentType.JSON)).get() + + val m4 = """{ + "properties": { + "type": { + "properties": { + "something": { + "properties" : { + "properties": { + "properties": { + "lastone": { "type": "keyword" } + } + } + } + } + } + } + } + } + """.trimIndent() + client().admin().indices().putMapping(PutMappingRequest(index4).source(m4, XContentType.JSON)).get() + + val docLevelInput = DocLevelMonitorInput( + "description", + listOf(index1, index2, index3, index4), + listOf(q1, q2, q3, q4) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor) + + val testDoc1 = """{ + "properties": "abcd" + }""" + indexDoc(index1, "1", testDoc1) + val testDoc2 = """{ + "type.properties": "abcd" + }""" + indexDoc(index2, "1", testDoc2) + val testDoc3 = """{ + "type.something.properties": "abcd" + }""" + indexDoc(index3, "1", testDoc3) + val testDoc4 = """{ + "type.something.properties.lastone": "abcd" + }""" + indexDoc(index4, "1", testDoc4) + + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + val id = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + val table = Table("asc", "id", null, 1, 0, "") + var getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", null)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 1) + val findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 4, findings.size) + } + + fun `test execute monitor with non-flattened json doc as source`() { + val docQuery1 = DocLevelQuery(query = "source.device.port:12345 OR source.device.hwd.id:12345", name = "3") + + val docLevelInput = DocLevelMonitorInput( + "description", listOf(index), listOf(docQuery1) + ) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor) + + val mappings = """{ + "properties": { + "source.device.port": { "type": "long" }, + "source.device.hwd.id": { "type": "long" }, + "nested_field": { + "type": "nested", + "properties": { + "test1": { + "type": "keyword" + } + } + }, + "my_join_field": { + "type": "join", + "relations": { + "question": "answer" + } + } + } + }""" + + client().admin().indices().putMapping(PutMappingRequest(index).source(mappings, XContentType.JSON)).get() + val getFieldCapabilitiesResp = client().fieldCaps(FieldCapabilitiesRequest().indices(index).fields("*")).get() + assertTrue(getFieldCapabilitiesResp.getField("source").containsKey("object")) + assertTrue(getFieldCapabilitiesResp.getField("source.device").containsKey("object")) + assertTrue(getFieldCapabilitiesResp.getField("source.device.hwd").containsKey("object")) + // testing both, nested and flatten documents + val testDocuments = mutableListOf() + testDocuments += """{ + "source" : { "device": {"port" : 12345 } }, + "nested_field": { "test1": "some text" } + }""" + testDocuments += """{ + "source.device.port" : "12345" + }""" + testDocuments += """{ + "source.device.port" : 12345 + }""" + testDocuments += """{ + "source" : { "device": {"hwd": { "id": 12345 } } } + }""" + testDocuments += """{ + "source.device.hwd.id" : 12345 + }""" + // Document with join field + testDocuments += """{ + "source" : { "device" : { "hwd": { "id" : 12345 } } }, + "my_join_field": { "name": "question" } + }""" + // Checking if these pointless but valid documents cause any issues + testDocuments += """{ + "source" : {} + }""" + testDocuments += """{ + "source.device" : null + }""" + testDocuments += """{ + "source.device" : {} + }""" + testDocuments += """{ + "source.device.hwd" : {} + }""" + testDocuments += """{ + "source.device.hwd.id" : null + }""" + testDocuments += """{ + "some.multi.val.field" : [12345, 10, 11] + }""" + // Insert all documents + for (i in testDocuments.indices) { + indexDoc(index, "$i", testDocuments[i]) + } + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + val id = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + val table = Table("asc", "id", null, 1, 0, "") + var getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", null)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 1) + val findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 6, findings.size) + assertEquals("Didn't match query", 1, findings[0].docLevelQueries.size) + } + + fun `test execute monitor without create when no monitors exists`() { + val docQuery = DocLevelQuery(query = "test_field:\"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) + ) + var executeMonitorResponse = executeMonitor(monitor, null) + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + + assertIndexNotExists(SCHEDULED_JOBS_INDEX) + + val createMonitorResponse = createMonitor(monitor) + + assertIndexExists(SCHEDULED_JOBS_INDEX) + + indexDoc(index, "1", testDoc) + + executeMonitorResponse = executeMonitor(monitor, createMonitorResponse?.id, dryRun = false) + + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + Assert.assertEquals( + (executeMonitorResponse.monitorRunResult.triggerResults.iterator().next().value as DocumentLevelTriggerRunResult) + .triggeredDocs.size, + 1 + ) + } + + fun `test delete monitor deletes all queries and metadata too`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + indexDoc(index, "1", testDoc) + val monitorId = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, monitorId, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(monitorId) + // Verify queries exist + var searchResponse = client().search( + SearchRequest(DOC_LEVEL_QUERIES_INDEX).source(SearchSourceBuilder().query(QueryBuilders.matchAllQuery())) + ).get() + assertNotEquals(0, searchResponse.hits.hits.size) + } + + fun `test execute monitor with multiple indices in input success`() { + + val testSourceIndex1 = "test_source_index1" + val testSourceIndex2 = "test_source_index2" + + createIndex(testSourceIndex1, Settings.EMPTY) + createIndex(testSourceIndex2, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex1, testSourceIndex2), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor) + client().admin().indices().refresh(RefreshRequest("*")) + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + indexDoc(testSourceIndex1, "1", testDoc) + indexDoc(testSourceIndex2, "1", testDoc) + + val id = monitorResponse.id + var executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + + var findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 2, findings.size) + var foundFindings = findings.filter { it.relatedDocIds.contains("1") } + assertEquals("Didn't find 2 findings", 2, foundFindings.size) + + indexDoc(testSourceIndex1, "2", testDoc) + indexDoc(testSourceIndex2, "2", testDoc) + executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 4, findings.size) + foundFindings = findings.filter { it.relatedDocIds.contains("2") } + assertEquals("Didn't find 2 findings", 2, foundFindings.size) + } + + fun `test execute monitor with multiple indices in input first index gets deleted`() { + // Index #1 does not exist + val testSourceIndex1 = "test_source_index1" + val testSourceIndex2 = "test_source_index2" + + createIndex(testSourceIndex1, Settings.EMPTY) + createIndex(testSourceIndex2, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex1, testSourceIndex2), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor) + client().admin().indices().refresh(RefreshRequest("*")) + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + indexDoc(testSourceIndex2, "1", testDoc) + + client().admin().indices().delete(DeleteIndexRequest(testSourceIndex1)).get() + + val id = monitorResponse.id + var executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + + var findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 1, findings.size) + var foundFindings = findings.filter { it.relatedDocIds.contains("1") } + assertEquals("Didn't find 2 findings", 1, foundFindings.size) + + indexDoc(testSourceIndex2, "2", testDoc) + executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 2, findings.size) + foundFindings = findings.filter { it.relatedDocIds.contains("2") } + assertEquals("Didn't find 2 findings", 1, foundFindings.size) + } + + fun `test execute monitor with multiple indices in input second index gets deleted`() { + // Second index does not exist + val testSourceIndex1 = "test_source_index1" + val testSourceIndex2 = "test_source_index2" + + createIndex(testSourceIndex1, Settings.EMPTY) + createIndex(testSourceIndex2, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex1, testSourceIndex2), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + val monitorResponse = createMonitor(monitor) + client().admin().indices().refresh(RefreshRequest("*")) + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + indexDoc(testSourceIndex1, "1", testDoc) + + client().admin().indices().delete(DeleteIndexRequest(testSourceIndex2)).get() + + val id = monitorResponse.id + var executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + + var findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 1, findings.size) + var foundFindings = findings.filter { it.relatedDocIds.contains("1") } + assertEquals("Didn't find 2 findings", 1, foundFindings.size) + + indexDoc(testSourceIndex1, "2", testDoc) + + executeMonitorResponse = executeMonitor(monitor, id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(id) + findings = searchFindings(id, ALL_FINDING_INDEX_PATTERN, true) + assertEquals("Findings saved for test monitor", 2, findings.size) + foundFindings = findings.filter { it.relatedDocIds.contains("2") } + assertEquals("Didn't find 2 findings", 1, foundFindings.size) + } + + fun `test execute pre-existing monitor and update`() { + val request = CreateIndexRequest(SCHEDULED_JOBS_INDEX).mapping(ScheduledJobIndices.scheduledJobMappings()) + .settings(Settings.builder().put("index.hidden", true).build()) + client().admin().indices().create(request) + val monitorStringWithoutName = """ + { + "monitor": { + "type": "monitor", + "schema_version": 0, + "name": "UayEuXpZtb", + "monitor_type": "doc_level_monitor", + "user": { + "name": "", + "backend_roles": [], + "roles": [], + "custom_attribute_names": [], + "user_requested_tenant": null + }, + "enabled": true, + "enabled_time": 1662753436791, + "schedule": { + "period": { + "interval": 5, + "unit": "MINUTES" + } + }, + "inputs": [{ + "doc_level_input": { + "description": "description", + "indices": [ + "$index" + ], + "queries": [{ + "id": "63efdcce-b5a1-49f4-a25f-6b5f9496a755", + "name": "3", + "query": "test_field:\"us-west-2\"", + "tags": [] + }] + } + }], + "triggers": [{ + "document_level_trigger": { + "id": "OGnTI4MBv6qt0ATc9Phk", + "name": "mrbHRMevYI", + "severity": "1", + "condition": { + "script": { + "source": "return true", + "lang": "painless" + } + }, + "actions": [] + } + }], + "last_update_time": 1662753436791 + } + } + """.trimIndent() + val monitorId = "abc" + indexDoc(SCHEDULED_JOBS_INDEX, monitorId, monitorStringWithoutName) + val getMonitorResponse = getMonitorResponse(monitorId) + Assert.assertNotNull(getMonitorResponse) + Assert.assertNotNull(getMonitorResponse.monitor) + val monitor = getMonitorResponse.monitor + + val testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + indexDoc(index, "1", testDoc) + var executeMonitorResponse = executeMonitor(monitor!!, monitorId, false) + Assert.assertNotNull(executeMonitorResponse) + if (executeMonitorResponse != null) { + Assert.assertNotNull(executeMonitorResponse.monitorRunResult.monitorName) + } + val alerts = searchAlerts(monitorId) + assertEquals(alerts.size, 1) + + val updateMonitorResponse = updateMonitor( + monitor.copy(id = monitorId), + monitorId + ) + Assert.assertNotNull(updateMonitorResponse) + indexDoc(index, "2", testDoc) + executeMonitorResponse = executeMonitor(updateMonitorResponse!!.monitor, monitorId, false) + val table = Table("asc", "id", null, 1, 0, "") + var getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", null)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 1) + getAlertsResponse = client() + .execute(GetAlertsAction.INSTANCE, GetAlertsRequest(table, "ALL", "ALL", monitorId)) + .get() + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 1) + } + + fun `test execute monitor with empty source index`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + val monitorId = monitorResponse.id + var executeMonitorResponse = executeMonitor(monitor, monitorId, false) + + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + + refreshIndex(ALL_FINDING_INDEX_PATTERN) + + var findings = searchFindings(monitorId, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 0, findings.size) + + indexDoc(index, "1", testDoc) + + executeMonitor(monitor, monitorId, false) + + refreshIndex(ALL_FINDING_INDEX_PATTERN) + + findings = searchFindings(monitorId, ALL_FINDING_INDEX_PATTERN) + assertTrue("Findings saved for test monitor", findings[0].relatedDocIds.contains("1")) + } + + fun `test execute GetFindingsAction with monitorId param`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + indexDoc(index, "1", testDoc) + val monitorId = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, monitorId, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(monitorId) + val findings = searchFindings(monitorId, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 1, findings.size) + assertTrue("Findings saved for test monitor", findings[0].relatedDocIds.contains("1")) + // fetch findings - pass monitorId as reference to finding_index + val findingsFromAPI = getFindings(findings.get(0).id) + assertEquals( + "Findings mismatch between manually searched and fetched via GetFindingsAction", + findings.get(0).id, + findingsFromAPI.get(0).id + ) + } + + fun `test execute GetFindingsAction with unknown monitorId`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + indexDoc(index, "1", testDoc) + val monitorId = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, monitorId, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(monitorId) + val findings = searchFindings(monitorId, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 1, findings.size) + assertTrue("Findings saved for test monitor", findings[0].relatedDocIds.contains("1")) + // fetch findings - don't send monitorId or findingIndexName. It should fall back to hardcoded finding index name + try { + getFindings(findings.get(0).id) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetFindings Action error ", + it.contains("Monitor not found") + ) + } + } + } + + fun `test execute GetFindingsAction with unknown findingIndex param`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + indexDoc(index, "1", testDoc) + val monitorId = monitorResponse.id + val executeMonitorResponse = executeMonitor(monitor, monitorId, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + searchAlerts(monitorId) + val findings = searchFindings(monitorId, ALL_FINDING_INDEX_PATTERN) + assertEquals("Findings saved for test monitor", 1, findings.size) + assertTrue("Findings saved for test monitor", findings[0].relatedDocIds.contains("1")) + // fetch findings - don't send monitorId or findingIndexName. It should fall back to hardcoded finding index name + try { + getFindings(findings.get(0).id) + } catch (e: Exception) { + e.message?.let { + assertTrue( + "Exception not returning GetFindings Action error ", + it.contains("no such index") + ) + } + } + } + + fun `test get alerts by list of monitors containing both existent and non-existent ids`() { + val docQuery = DocLevelQuery(query = "test_field:\"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 testTime = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now().truncatedTo(MILLIS)) + val testDoc = """{ + "message" : "This is an error from IAD region", + "test_strict_date_time" : "$testTime", + "test_field" : "us-west-2" + }""" + + monitor = monitorResponse!!.monitor + + val id = monitorResponse.id + + var monitor1 = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger), + ) + val monitorResponse1 = createMonitor(monitor1) + monitor1 = monitorResponse1!!.monitor + val id1 = monitorResponse1.id + indexDoc(index, "1", testDoc) + executeMonitor(monitor1, id1, false) + executeMonitor(monitor, id, false) + val alerts = searchAlerts(id) + assertEquals("Alert saved for test monitor", 1, alerts.size) + val alerts1 = searchAlerts(id) + assertEquals("Alert saved for test monitor", 1, alerts1.size) + val table = Table("asc", "id", null, 1000, 0, "") + var getAlertsResponse = client() + .execute( + GetAlertsAction.INSTANCE, + GetAlertsRequest(table, "ALL", "ALL", null) + ) + .get() + + Assert.assertTrue(getAlertsResponse != null) + Assert.assertTrue(getAlertsResponse.alerts.size == 2) + + var alertsResponseForRequestWithoutCustomIndex = client() + .execute( + GetAlertsAction.INSTANCE, + GetAlertsRequest(table, "ALL", "ALL", null) + ) + .get() + Assert.assertTrue(alertsResponseForRequestWithoutCustomIndex != null) + Assert.assertTrue(alertsResponseForRequestWithoutCustomIndex.alerts.size == 2) + var getAlertsByAlertIds = client() + .execute( + GetAlertsAction.INSTANCE, + GetAlertsRequest(table, "ALL", "ALL", null) + ) + .get() + Assert.assertTrue(getAlertsByAlertIds != null) + Assert.assertTrue(getAlertsByAlertIds.alerts.size == 2) + + var getAlertsByWrongAlertIds = client() + .execute( + GetAlertsAction.INSTANCE, + GetAlertsRequest(table, "ALL", "ALL", null) + ) + .get() + + Assert.assertTrue(getAlertsByWrongAlertIds != null) + } + + fun `test queryIndex rollover and delete monitor success`() { + + val testSourceIndex = "test_source_index" + createIndex(testSourceIndex, Settings.builder().put("index.mapping.total_fields.limit", "10000").build()) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // This doc should create close to 1000 (limit) fields in index mapping. It's easier to add mappings like this then via api + val docPayload: StringBuilder = StringBuilder(100000) + docPayload.append("{") + for (i in 1..3300) { + docPayload.append(""" "id$i.somefield.somefield$i":$i,""") + } + docPayload.append("\"test_field\" : \"us-west-2\" }") + indexDoc(testSourceIndex, "1", docPayload.toString()) + // Create monitor #1 + var monitorResponse = createMonitor(monitor) + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + // Execute monitor #1 + var executeMonitorResponse = executeMonitor(monitor, monitorResponse.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + // Create monitor #2 + var monitorResponse2 = createMonitor(monitor) + assertFalse(monitorResponse2?.id.isNullOrEmpty()) + monitor = monitorResponse2!!.monitor + // Insert doc #2. This one should trigger creation of alerts during monitor exec + val testDoc = """{ + "test_field" : "us-west-2" + }""" + indexDoc(testSourceIndex, "2", testDoc) + // Execute monitor #2 + var executeMonitorResponse2 = executeMonitor(monitor, monitorResponse2.id, false) + Assert.assertEquals(executeMonitorResponse2!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse2.monitorRunResult.triggerResults.size, 1) + + refreshIndex(AlertIndices.ALERT_INDEX) + var alerts = searchAlerts(monitorResponse2.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 1) + + // Both monitors used same queryIndex alias. Since source index has close to limit amount of fields in mappings, + // we expect that creation of second monitor would trigger rollover of queryIndex + var getIndexResponse: GetIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(2, getIndexResponse.indices.size) + assertEquals(DOC_LEVEL_QUERIES_INDEX + "-000001", getIndexResponse.indices[0]) + assertEquals(DOC_LEVEL_QUERIES_INDEX + "-000002", getIndexResponse.indices[1]) + // Now we'll verify that execution of both monitors still works + indexDoc(testSourceIndex, "3", testDoc) + // Exec Monitor #1 + executeMonitorResponse = executeMonitor(monitor, monitorResponse.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + refreshIndex(AlertIndices.ALERT_INDEX) + alerts = searchAlerts(monitorResponse.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 2) + // Exec Monitor #2 + executeMonitorResponse = executeMonitor(monitor, monitorResponse2.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + refreshIndex(AlertIndices.ALERT_INDEX) + alerts = searchAlerts(monitorResponse2.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 2) + // Delete monitor #1 + client().execute( + DeleteMonitorAction.INSTANCE, DeleteMonitorRequest(monitorResponse.id, WriteRequest.RefreshPolicy.IMMEDIATE) + ).get() + // Expect first concrete queryIndex to be deleted since that one was only used by this monitor + getIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(1, getIndexResponse.indices.size) + assertEquals(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "-000002", getIndexResponse.indices[0]) + // Delete monitor #2 + client().execute( + DeleteMonitorAction.INSTANCE, DeleteMonitorRequest(monitorResponse2.id, WriteRequest.RefreshPolicy.IMMEDIATE) + ).get() + // Expect second concrete queryIndex to be deleted since that one was only used by this monitor + getIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(0, getIndexResponse.indices.size) + } + + fun `test queryIndex rollover failure source_index field count over limit`() { + + val testSourceIndex = "test_source_index" + createIndex(testSourceIndex, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // This doc should create 999 fields in mapping, only 1 field less then limit + val docPayload: StringBuilder = StringBuilder(100000) + docPayload.append("{") + for (i in 1..998) { + docPayload.append(""" "id$i":$i,""") + } + docPayload.append("\"test_field\" : \"us-west-2\" }") + indexDoc(testSourceIndex, "1", docPayload.toString()) + // Create monitor and expect failure. + // queryIndex has 3 fields in mappings initially so 999 + 3 > 1000(default limit) + try { + createMonitor(monitor) + } catch (e: Exception) { + assertTrue(e.message?.contains("can't process index [$testSourceIndex] due to field mapping limit") ?: false) + } + } + + fun `test queryIndex not rolling over multiple monitors`() { + val testSourceIndex = "test_source_index" + createIndex(testSourceIndex, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // Create doc with 11 fields + val docPayload: StringBuilder = StringBuilder(1000) + docPayload.append("{") + for (i in 1..10) { + docPayload.append(""" "id$i":$i,""") + } + docPayload.append("\"test_field\" : \"us-west-2\" }") + indexDoc(testSourceIndex, "1", docPayload.toString()) + // Create monitor #1 + var monitorResponse = createMonitor(monitor) + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + // Execute monitor #1 + var executeMonitorResponse = executeMonitor(monitor, monitorResponse.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + // Create monitor #2 + var monitorResponse2 = createMonitor(monitor) + assertFalse(monitorResponse2?.id.isNullOrEmpty()) + monitor = monitorResponse2!!.monitor + // Insert doc #2. This one should trigger creation of alerts during monitor exec + val testDoc = """{ + "test_field" : "us-west-2" + }""" + indexDoc(testSourceIndex, "2", testDoc) + // Execute monitor #2 + var executeMonitorResponse2 = executeMonitor(monitor, monitorResponse2.id, false) + Assert.assertEquals(executeMonitorResponse2!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse2.monitorRunResult.triggerResults.size, 1) + + refreshIndex(AlertIndices.ALERT_INDEX) + var alerts = searchAlerts(monitorResponse2.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 1) + + // Both monitors used same queryIndex. Since source index has well below limit amount of fields in mappings, + // we expect only 1 backing queryIndex + val getIndexResponse: GetIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(1, getIndexResponse.indices.size) + // Now we'll verify that execution of both monitors work + indexDoc(testSourceIndex, "3", testDoc) + // Exec Monitor #1 + executeMonitorResponse = executeMonitor(monitor, monitorResponse.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + refreshIndex(AlertIndices.ALERT_INDEX) + alerts = searchAlerts(monitorResponse.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 2) + // Exec Monitor #2 + executeMonitorResponse = executeMonitor(monitor, monitorResponse2.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + refreshIndex(AlertIndices.ALERT_INDEX) + alerts = searchAlerts(monitorResponse2.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 2) + } + + /** + * 1. Create monitor with input source_index with 9000 fields in mappings - can fit 1 in queryIndex + * 2. Update monitor and change input source_index to a new one with 9000 fields in mappings + * 3. Expect queryIndex rollover resulting in 2 backing indices + * 4. Delete monitor and expect that all backing indices are deleted + * */ + fun `test updating monitor no execution queryIndex rolling over`() { + val testSourceIndex1 = "test_source_index1" + val testSourceIndex2 = "test_source_index2" + createIndex(testSourceIndex1, Settings.builder().put("index.mapping.total_fields.limit", "10000").build()) + createIndex(testSourceIndex2, Settings.builder().put("index.mapping.total_fields.limit", "10000").build()) + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex1), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // This doc should create close to 10000 (limit) fields in index mapping. It's easier to add mappings like this then via api + val docPayload: StringBuilder = StringBuilder(100000) + docPayload.append("{") + for (i in 1..9000) { + docPayload.append(""" "id$i":$i,""") + } + docPayload.append("\"test_field\" : \"us-west-2\" }") + // Indexing docs here as an easier means to set index mappings + indexDoc(testSourceIndex1, "1", docPayload.toString()) + indexDoc(testSourceIndex2, "1", docPayload.toString()) + // Create monitor + var monitorResponse = createMonitor(monitor) + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + // Update monitor and change input + val updatedMonitor = monitor.copy( + inputs = listOf( + DocLevelMonitorInput("description", listOf(testSourceIndex2), listOf(docQuery)) + ) + ) + updateMonitor(updatedMonitor, updatedMonitor.id) + assertFalse(monitorResponse?.id.isNullOrEmpty()) + + // Expect queryIndex to rollover after setting new source_index with close to limit amount of fields in mappings + var getIndexResponse: GetIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(2, getIndexResponse.indices.size) + + deleteMonitor(updatedMonitor.id) + waitUntil { + getIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + return@waitUntil getIndexResponse.indices.isEmpty() + } + assertEquals(0, getIndexResponse.indices.size) + } + + fun `test queryIndex gets increased max fields in mappings`() { + val testSourceIndex = "test_source_index" + createIndex(testSourceIndex, Settings.builder().put("index.mapping.total_fields.limit", "10000").build()) + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // This doc should create 12000 fields in index mapping. It's easier to add mappings like this then via api + val docPayload: StringBuilder = StringBuilder(100000) + docPayload.append("{") + for (i in 1..9998) { + docPayload.append(""" "id$i":$i,""") + } + docPayload.append("\"test_field\" : \"us-west-2\" }") + // Indexing docs here as an easier means to set index mappings + indexDoc(testSourceIndex, "1", docPayload.toString()) + // Create monitor + var monitorResponse = createMonitor(monitor) + assertFalse(monitorResponse?.id.isNullOrEmpty()) + monitor = monitorResponse!!.monitor + + // Expect queryIndex to rollover after setting new source_index with close to limit amount of fields in mappings + var getIndexResponse: GetIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + assertEquals(1, getIndexResponse.indices.size) + val field_max_limit = getIndexResponse + .getSetting(DOC_LEVEL_QUERIES_INDEX + "-000001", MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.key).toInt() + + assertEquals(10000 + DocLevelMonitorQueries.QUERY_INDEX_BASE_FIELDS_COUNT, field_max_limit) + + deleteMonitor(monitorResponse.id) + waitUntil { + getIndexResponse = + client().admin().indices().getIndex(GetIndexRequest().indices(ScheduledJob.DOC_LEVEL_QUERIES_INDEX + "*")).get() + return@waitUntil getIndexResponse.indices.isEmpty() + } + assertEquals(0, getIndexResponse.indices.size) + } + + fun `test queryIndex bwc when index was not an alias`() { + createIndex(DOC_LEVEL_QUERIES_INDEX, Settings.builder().put("index.hidden", true).build()) + assertIndexExists(DOC_LEVEL_QUERIES_INDEX) + + val testSourceIndex = "test_source_index" + createIndex(testSourceIndex, Settings.EMPTY) + + val docQuery = DocLevelQuery(query = "test_field:\"us-west-2\"", name = "3") + val docLevelInput = DocLevelMonitorInput("description", listOf(testSourceIndex), listOf(docQuery)) + val trigger = randomDocumentLevelTrigger(condition = ALWAYS_RUN) + var monitor = randomDocumentLevelMonitor( + inputs = listOf(docLevelInput), + triggers = listOf(trigger) + ) + // This doc should create 999 fields in mapping, only 1 field less then limit + val docPayload = "{\"test_field\" : \"us-west-2\" }" + // Create monitor + try { + var monitorResponse = createMonitor(monitor) + indexDoc(testSourceIndex, "1", docPayload) + var executeMonitorResponse = executeMonitor(monitor, monitorResponse!!.id, false) + Assert.assertEquals(executeMonitorResponse!!.monitorRunResult.monitorName, monitor.name) + Assert.assertEquals(executeMonitorResponse.monitorRunResult.triggerResults.size, 1) + refreshIndex(AlertIndices.ALERT_INDEX) + val alerts = searchAlerts(monitorResponse.id) + Assert.assertTrue(alerts != null) + Assert.assertTrue(alerts.size == 1) + // check if DOC_LEVEL_QUERIES_INDEX alias exists + assertAliasExists(DOC_LEVEL_QUERIES_INDEX) + } catch (e: Exception) { + fail("Exception happend but it shouldn't!") + } + } +} diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/alerts/AlertIndicesIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/alerts/AlertIndicesIT.kt index e1b780491..8f361991e 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/alerts/AlertIndicesIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/alerts/AlertIndicesIT.kt @@ -23,6 +23,8 @@ import org.opensearch.alerting.settings.AlertingSettings import org.opensearch.common.xcontent.XContentType import org.opensearch.common.xcontent.json.JsonXContent.jsonXContent import org.opensearch.rest.RestStatus +import org.opensearch.test.OpenSearchTestCase +import java.util.concurrent.TimeUnit class AlertIndicesIT : AlertingRestTestCase() { @@ -62,7 +64,7 @@ class AlertIndicesIT : AlertingRestTestCase() { executeMonitor(createRandomMonitor()) assertIndexExists(AlertIndices.ALERT_INDEX) assertIndexExists(AlertIndices.ALERT_HISTORY_WRITE_INDEX) - verifyIndexSchemaVersion(ScheduledJob.SCHEDULED_JOBS_INDEX, 5) + verifyIndexSchemaVersion(ScheduledJob.SCHEDULED_JOBS_INDEX, 6) verifyIndexSchemaVersion(AlertIndices.ALERT_INDEX, 4) verifyIndexSchemaVersion(AlertIndices.ALERT_HISTORY_WRITE_INDEX, 4) } @@ -86,7 +88,7 @@ class AlertIndicesIT : AlertingRestTestCase() { val trueMonitor = createMonitor(randomDocumentLevelMonitor(inputs = listOf(docLevelInput), triggers = listOf(trigger))) executeMonitor(trueMonitor.id) assertIndexExists(AlertIndices.FINDING_HISTORY_WRITE_INDEX) - verifyIndexSchemaVersion(ScheduledJob.SCHEDULED_JOBS_INDEX, 5) + verifyIndexSchemaVersion(ScheduledJob.SCHEDULED_JOBS_INDEX, 6) verifyIndexSchemaVersion(AlertIndices.FINDING_HISTORY_WRITE_INDEX, 1) } @@ -235,7 +237,14 @@ class AlertIndicesIT : AlertingRestTestCase() { client().updateSettings(AlertingSettings.ALERT_HISTORY_RETENTION_PERIOD.key, "1s") // Give some time for history to be rolled over and cleared - Thread.sleep(5000) + OpenSearchTestCase.waitUntil({ + val alertIndices = getAlertIndices().size + val docCount = getAlertHistoryDocCount() + if (alertIndices > 2 || docCount > 0) { + return@waitUntil false + } + return@waitUntil true + }, 30, TimeUnit.SECONDS) // Given the max_docs and retention settings above, the history index will rollover and the non-write index will be deleted. // This leaves two indices: alert index and an empty history write index @@ -284,7 +293,14 @@ class AlertIndicesIT : AlertingRestTestCase() { client().updateSettings(AlertingSettings.ALERT_HISTORY_RETENTION_PERIOD.key, "1s") // Give some time for history to be rolled over and cleared - Thread.sleep(5000) + OpenSearchTestCase.waitUntil({ + val alertIndices = getAlertIndices().size + val docCount = getAlertHistoryDocCount() + if (alertIndices > 2 || docCount > 0) { + return@waitUntil false + } + return@waitUntil true + }, 30, TimeUnit.SECONDS) // Given the max_docs and retention settings above, the history index will rollover and the non-write index will be deleted. // This leaves two indices: alert index and an empty history write index diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/model/XContentTests.kt b/alerting/src/test/kotlin/org/opensearch/alerting/model/XContentTests.kt index 17e772890..4d4776835 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/model/XContentTests.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/model/XContentTests.kt @@ -383,7 +383,13 @@ class XContentTests : OpenSearchTestCase() { } fun `test MonitorMetadata`() { - val monitorMetadata = MonitorMetadata("monitorId-metadata", "monitorId", emptyList(), emptyMap()) + val monitorMetadata = MonitorMetadata( + id = "monitorId-metadata", + monitorId = "monitorId", + lastActionExecutionTimes = emptyList(), + lastRunContext = emptyMap(), + sourceToQueryIndexMapping = mutableMapOf() + ) val monitorMetadataString = monitorMetadata.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS).string() val parsedMonitorMetadata = MonitorMetadata.parse(parser(monitorMetadataString)) assertEquals("Round tripping MonitorMetadata doesn't work", monitorMetadata, parsedMonitorMetadata) 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 66804bfb1..229885acb 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/MonitorRestApiIT.kt @@ -60,6 +60,7 @@ import org.opensearch.test.rest.OpenSearchRestTestCase import java.time.Instant import java.time.ZoneId import java.time.temporal.ChronoUnit +import java.util.concurrent.TimeUnit @TestLogging("level:DEBUG", reason = "Debug for tests.") @Suppress("UNCHECKED_CAST") @@ -850,15 +851,27 @@ class MonitorRestApiIT : AlertingRestTestCase() { updatedMonitor.toHttpEntity() ) assertEquals("Update request not successful", RestStatus.OK, updateResponse.restStatus()) - - // Wait 5 seconds for event to be processed and alerts moved - Thread.sleep(5000) - + // Wait until postIndex hook is executed due to monitor update + waitUntil({ + val alerts = searchAlerts(monitor) + if (alerts.size == 1) { + return@waitUntil true + } + return@waitUntil false + }, 60, TimeUnit.SECONDS) val alerts = searchAlerts(monitor) // We have two alerts from above, 1 for each trigger, there should be only 1 left in active index assertEquals("One alert should be in active index", 1, alerts.size) assertEquals("Wrong alert in active index", alertKeep.toJsonString(), alerts.single().toJsonString()) + waitUntil({ + val alerts = searchAlerts(monitor, AlertIndices.ALERT_HISTORY_WRITE_INDEX) + if (alerts.size == 1) { + return@waitUntil true + } + return@waitUntil false + }, 60, TimeUnit.SECONDS) + val historyAlerts = searchAlerts(monitor, AlertIndices.ALERT_HISTORY_WRITE_INDEX) // Only alertDelete should of been moved to history index assertEquals("One alert should be in history index", 1, historyAlerts.size) diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureDestinationRestApiIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureDestinationRestApiIT.kt index 598b4f98c..a95bdc9fe 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureDestinationRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureDestinationRestApiIT.kt @@ -41,7 +41,7 @@ class SecureDestinationRestApiIT : AlertingRestTestCase() { } } - val user = "userOne" + val user = "userA" var userClient: RestClient? = null @Before @@ -139,7 +139,7 @@ class SecureDestinationRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_GET_DESTINATION_ACCESS) ) diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailAccountRestApiIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailAccountRestApiIT.kt index 74bb75ff7..5cf047681 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailAccountRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailAccountRestApiIT.kt @@ -13,6 +13,7 @@ import org.junit.After import org.junit.Before import org.junit.BeforeClass import org.opensearch.alerting.ALERTING_GET_EMAIL_ACCOUNT_ACCESS +import org.opensearch.alerting.ALERTING_NO_ACCESS_ROLE import org.opensearch.alerting.ALERTING_SEARCH_EMAIL_ACCOUNT_ACCESS import org.opensearch.alerting.AlertingPlugin import org.opensearch.alerting.AlertingRestTestCase @@ -20,6 +21,7 @@ import org.opensearch.alerting.TEST_HR_BACKEND_ROLE import org.opensearch.alerting.TEST_HR_INDEX import org.opensearch.alerting.TEST_HR_ROLE import org.opensearch.alerting.makeRequest +import org.opensearch.client.ResponseException import org.opensearch.client.RestClient import org.opensearch.commons.rest.SecureRestClientBuilder import org.opensearch.rest.RestStatus @@ -50,7 +52,7 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { } } - val user = "userOne" + val user = "userB" var userClient: RestClient? = null @Before @@ -76,7 +78,7 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_GET_EMAIL_ACCOUNT_ACCESS) ) @@ -105,7 +107,7 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_SEARCH_EMAIL_ACCOUNT_ACCESS) ) @@ -126,18 +128,16 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 - + */ fun `test get email accounts with an user without get email account role`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) - val emailAccount = createRandomEmailAccountWithGivenName(true, randomAlphaOfLength(5)) - try { userClient?.makeRequest( "GET", @@ -155,19 +155,15 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - fun `test search email accounts with an user without search email account role`() { - createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) - createRandomEmailAccountWithGivenName(true, randomAlphaOfLength(5)) - try { userClient?.makeRequest( "POST", @@ -182,6 +178,4 @@ class SecureEmailAccountRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - - */ } diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailGroupsRestApiIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailGroupsRestApiIT.kt index 72fb317e1..2e52514a4 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailGroupsRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureEmailGroupsRestApiIT.kt @@ -52,7 +52,7 @@ class SecureEmailGroupsRestApiIT : AlertingRestTestCase() { } } - val user = "userOne" + val user = "userC" var userClient: RestClient? = null @Before @@ -78,7 +78,7 @@ class SecureEmailGroupsRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_GET_EMAIL_GROUP_ACCESS) ) @@ -105,7 +105,7 @@ class SecureEmailGroupsRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_SEARCH_EMAIL_GROUP_ACCESS) ) diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureMonitorRestApiIT.kt b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureMonitorRestApiIT.kt index 151be8d4d..8ea7b5409 100644 --- a/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureMonitorRestApiIT.kt +++ b/alerting/src/test/kotlin/org/opensearch/alerting/resthandler/SecureMonitorRestApiIT.kt @@ -20,21 +20,27 @@ import org.opensearch.alerting.ALERTING_FULL_ACCESS_ROLE import org.opensearch.alerting.ALERTING_GET_ALERTS_ACCESS import org.opensearch.alerting.ALERTING_GET_MONITOR_ACCESS import org.opensearch.alerting.ALERTING_INDEX_MONITOR_ACCESS +import org.opensearch.alerting.ALERTING_NO_ACCESS_ROLE +import org.opensearch.alerting.ALERTING_READ_ONLY_ACCESS import org.opensearch.alerting.ALERTING_SEARCH_MONITOR_ONLY_ACCESS import org.opensearch.alerting.ALL_ACCESS_ROLE import org.opensearch.alerting.ALWAYS_RUN import org.opensearch.alerting.AlertingRestTestCase import org.opensearch.alerting.DRYRUN_MONITOR +import org.opensearch.alerting.TERM_DLS_QUERY import org.opensearch.alerting.TEST_HR_BACKEND_ROLE import org.opensearch.alerting.TEST_HR_INDEX import org.opensearch.alerting.TEST_HR_ROLE import org.opensearch.alerting.TEST_NON_HR_INDEX +import org.opensearch.alerting.aggregation.bucketselectorext.BucketSelectorExtAggregationBuilder import org.opensearch.alerting.assertUserNull import org.opensearch.alerting.core.model.SearchInput import org.opensearch.alerting.makeRequest import org.opensearch.alerting.model.Alert import org.opensearch.alerting.randomAction import org.opensearch.alerting.randomAlert +import org.opensearch.alerting.randomBucketLevelMonitor +import org.opensearch.alerting.randomBucketLevelTrigger import org.opensearch.alerting.randomQueryLevelMonitor import org.opensearch.alerting.randomQueryLevelTrigger import org.opensearch.alerting.randomTemplateScript @@ -49,6 +55,9 @@ import org.opensearch.commons.authuser.User import org.opensearch.commons.rest.SecureRestClientBuilder 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 org.opensearch.test.junit.annotations.TestLogging @@ -65,7 +74,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { } } - val user = "userOne" + val user = "userD" var userClient: RestClient? = null @Before @@ -85,11 +94,15 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { } // Create Monitor related security tests - fun `test create monitor with an user with alerting role`() { - createUserWithTestData(user, TEST_HR_INDEX, TEST_HR_ROLE, TEST_HR_BACKEND_ROLE) - createUserRolesMapping(ALERTING_FULL_ACCESS_ROLE, arrayOf(user)) + createUserWithTestDataAndCustomRole( + user, + TEST_HR_INDEX, + TEST_HR_ROLE, + listOf(TEST_HR_BACKEND_ROLE), + getClusterPermissionsFromCustomRole(ALERTING_INDEX_MONITOR_ACCESS) + ) try { // randomMonitor has a dummy user, api ignores the User passed as part of monitor, it picks user info from the logged-in user. val monitor = randomQueryLevelMonitor().copy( @@ -110,13 +123,14 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test create monitor with an user without alerting role`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) try { @@ -138,13 +152,9 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { fun `test create monitor with an user with read-only role`() { - createUserWithTestDataAndCustomRole( - user, - TEST_HR_INDEX, - TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, - getClusterPermissionsFromCustomRole(ALERTING_READ_ONLY_ACCESS) - ) + createUserWithTestData(user, TEST_HR_INDEX, TEST_HR_ROLE, TEST_HR_BACKEND_ROLE) + createUserRolesMapping(ALERTING_READ_ONLY_ACCESS, arrayOf(user)) + try { val monitor = randomQueryLevelMonitor().copy( inputs = listOf( @@ -159,9 +169,9 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { assertEquals("Unexpected status", RestStatus.FORBIDDEN, e.response.restStatus()) } finally { deleteRoleAndRoleMapping(TEST_HR_ROLE) + deleteRoleMapping(ALERTING_READ_ONLY_ACCESS) } } - */ fun `test query monitors with an user with only search monitor cluster permission`() { @@ -169,7 +179,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_SEARCH_MONITOR_ONLY_ACCESS) ) val monitor = createRandomMonitor(true) @@ -186,17 +196,19 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { val hits = xcp.map()["hits"]!! as Map> val numberDocsFound = hits["total"]?.get("value") assertEquals("Monitor not found during search", 1, numberDocsFound) + deleteRoleAndRoleMapping(TEST_HR_ROLE) } /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test query monitors with an user without search monitor cluster permission`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) try { @@ -215,7 +227,6 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - */ fun `test create monitor with an user without index read role`() { @@ -223,7 +234,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_INDEX_MONITOR_ACCESS) ) try { @@ -257,7 +268,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_GET_MONITOR_ACCESS) ) @@ -278,12 +289,13 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test get monitor with an user without get monitor role`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) @@ -303,7 +315,6 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - */ fun getDocs(response: Response?): Any? { val hits = createParser( @@ -414,8 +425,8 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test query monitors with disable filter by`() { - disableFilterBy() // creates monitor as "admin" user. @@ -440,12 +451,17 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { NStringEntity(search, ContentType.APPLICATION_JSON) ) fail("Expected 403 FORBIDDEN response") - } catch (e: AssertionError) { - assertEquals("Unexpected status", "Expected 403 FORBIDDEN response", e.message) + } catch (e: ResponseException) { + assertEquals("Unexpected status", RestStatus.FORBIDDEN, e.response.restStatus()) } - // add alerting roles and search as userOne - must return 1 docs - createUserRolesMapping(ALERTING_FULL_ACCESS_ROLE, arrayOf(user)) + createUserWithTestDataAndCustomRole( + user, + TEST_HR_INDEX, + TEST_HR_ROLE, + listOf(TEST_HR_BACKEND_ROLE), + getClusterPermissionsFromCustomRole(ALERTING_SEARCH_MONITOR_ONLY_ACCESS) + ) try { val userOneSearchResponse = userClient?.makeRequest( "POST", @@ -456,7 +472,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { assertEquals("Search monitor failed", RestStatus.OK, userOneSearchResponse?.restStatus()) assertEquals("Monitor not found during search", 1, getDocs(userOneSearchResponse)) } finally { - deleteRoleMapping(ALERTING_FULL_ACCESS_ROLE) + deleteRoleAndRoleMapping(TEST_HR_ROLE) } } @@ -486,8 +502,8 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { NStringEntity(search, ContentType.APPLICATION_JSON) ) fail("Expected 403 FORBIDDEN response") - } catch (e: AssertionError) { - assertEquals("Unexpected status", "Expected 403 FORBIDDEN response", e.message) + } catch (e: ResponseException) { + assertEquals("Unexpected status", RestStatus.FORBIDDEN, e.response.restStatus()) } // add alerting roles and search as userOne - must return 0 docs @@ -506,14 +522,12 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { } } - */ - fun `test execute monitor with an user with execute monitor access`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_EXECUTE_MONITOR_ACCESS) ) @@ -533,12 +547,13 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test execute monitor with an user without execute monitor access`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) @@ -558,14 +573,13 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - */ fun `test delete monitor with an user with delete monitor access`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_DELETE_MONITOR_ACCESS) ) @@ -587,12 +601,13 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test delete monitor with an user without delete monitor access`() { createUserWithTestDataAndCustomRole( user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_NO_ACCESS_ROLE) ) @@ -636,8 +651,8 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { try { getAlerts(userClient as RestClient, inputMap).asMap() fail("Expected 403 FORBIDDEN response") - } catch (e: AssertionError) { - assertEquals("Unexpected status", "Expected 403 FORBIDDEN response", e.message) + } catch (e: ResponseException) { + assertEquals("Unexpected status", RestStatus.FORBIDDEN, e.response.restStatus()) } // add alerting roles and search as userOne - must return 0 docs @@ -673,10 +688,9 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { try { getAlerts(userClient as RestClient, inputMap).asMap() fail("Expected 403 FORBIDDEN response") - } catch (e: AssertionError) { - assertEquals("Unexpected status", "Expected 403 FORBIDDEN response", e.message) + } catch (e: ResponseException) { + assertEquals("Unexpected status", RestStatus.FORBIDDEN, e.response.restStatus()) } - // add alerting roles and search as userOne - must return 0 docs createUserRolesMapping(ALERTING_FULL_ACCESS_ROLE, arrayOf(user)) try { @@ -687,8 +701,6 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { } } - */ - fun `test get alerts with an user with get alerts role`() { putAlertMappings() @@ -712,7 +724,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { user, TEST_HR_INDEX, TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + listOf(TEST_HR_BACKEND_ROLE), getClusterPermissionsFromCustomRole(ALERTING_GET_ALERTS_ACCESS) ) try { @@ -809,21 +821,24 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { assertEquals("Delete monitor failed", RestStatus.OK, adminDeleteResponse.restStatus()) } finally { deleteRoleAndRoleMapping(TEST_HR_ROLE) + deleteRoleMapping(ALERTING_FULL_ACCESS_ROLE) } } /* TODO: https://github.com/opensearch-project/alerting/issues/300 + */ fun `test execute query-level monitor with user having partial index permissions`() { - createUserWithDocLevelSecurityTestDataAndCustomRole( - user, - TEST_HR_INDEX, + createUser(user, user, arrayOf(TEST_HR_BACKEND_ROLE)) + createTestIndex(TEST_HR_INDEX) + createIndexRoleWithDocLevelSecurity( TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + TEST_HR_INDEX, TERM_DLS_QUERY, - getClusterPermissionsFromCustomRole(ALERTING_FULL_ACCESS_ROLE) + getClusterPermissionsFromCustomRole(ALERTING_INDEX_MONITOR_ACCESS) ) + createUserRolesMapping(TEST_HR_ROLE, arrayOf(user)) // Add a doc that is accessible to the user indexDoc( @@ -831,7 +846,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { """ { "test_field": "a", - "accessible": true + "accessible": true } """.trimIndent() ) @@ -850,7 +865,7 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { val input = SearchInput(indices = listOf(TEST_HR_INDEX), query = SearchSourceBuilder().query(QueryBuilders.matchAllQuery())) val triggerScript = """ // make sure there is exactly one hit - return ctx.results[0].hits.hits.size() == 1 + return ctx.results[0].hits.hits.size() == 1 """.trimIndent() val trigger = randomQueryLevelTrigger(condition = Script(triggerScript)).copy(actions = listOf()) @@ -870,14 +885,15 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { fun `test execute bucket-level monitor with user having partial index permissions`() { - createUserWithDocLevelSecurityTestDataAndCustomRole( - user, - TEST_HR_INDEX, + createUser(user, user, arrayOf(TEST_HR_BACKEND_ROLE)) + createTestIndex(TEST_HR_INDEX) + createIndexRoleWithDocLevelSecurity( TEST_HR_ROLE, - TEST_HR_BACKEND_ROLE, + TEST_HR_INDEX, TERM_DLS_QUERY, - getClusterPermissionsFromCustomRole(ALERTING_FULL_ACCESS_ROLE) + getClusterPermissionsFromCustomRole(ALERTING_INDEX_MONITOR_ACCESS) ) + createUserRolesMapping(TEST_HR_ROLE, arrayOf(user)) // Add a doc that is accessible to the user indexDoc( @@ -937,5 +953,4 @@ class SecureMonitorRestApiIT : AlertingRestTestCase() { deleteRoleAndRoleMapping(TEST_HR_ROLE) } } - */ } diff --git a/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt b/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt new file mode 100644 index 000000000..e12cff10b --- /dev/null +++ b/alerting/src/test/kotlin/org/opensearch/alerting/transport/AlertingSingleNodeTestCase.kt @@ -0,0 +1,226 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +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.get.GetIndexRequest +import org.opensearch.action.admin.indices.get.GetIndexRequestBuilder +import org.opensearch.action.admin.indices.get.GetIndexResponse +import org.opensearch.action.admin.indices.refresh.RefreshAction +import org.opensearch.action.admin.indices.refresh.RefreshRequest +import org.opensearch.action.support.IndicesOptions +import org.opensearch.action.support.WriteRequest +import org.opensearch.alerting.AlertingPlugin +import org.opensearch.alerting.action.DeleteMonitorAction +import org.opensearch.alerting.action.DeleteMonitorRequest +import org.opensearch.alerting.action.ExecuteMonitorAction +import org.opensearch.alerting.action.ExecuteMonitorRequest +import org.opensearch.alerting.action.ExecuteMonitorResponse +import org.opensearch.alerting.action.GetFindingsAction +import org.opensearch.alerting.action.GetFindingsRequest +import org.opensearch.alerting.action.GetFindingsResponse +import org.opensearch.alerting.action.GetMonitorAction +import org.opensearch.alerting.action.GetMonitorRequest +import org.opensearch.alerting.action.IndexMonitorAction +import org.opensearch.alerting.action.IndexMonitorRequest +import org.opensearch.alerting.action.IndexMonitorResponse +import org.opensearch.alerting.alerts.AlertIndices +import org.opensearch.alerting.model.Alert +import org.opensearch.alerting.model.Finding +import org.opensearch.alerting.model.Monitor +import org.opensearch.alerting.model.Table +import org.opensearch.common.settings.Settings +import org.opensearch.common.unit.TimeValue +import org.opensearch.common.xcontent.XContentType +import org.opensearch.common.xcontent.json.JsonXContent +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.plugins.Plugin +import org.opensearch.rest.RestRequest +import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.search.fetch.subphase.FetchSourceContext +import org.opensearch.test.OpenSearchSingleNodeTestCase +import java.time.Instant +import java.util.Locale + +/** + * 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 AlertingSingleNodeTestCase : OpenSearchSingleNodeTestCase() { + + protected val index: String = randomAlphaOfLength(10).lowercase(Locale.ROOT) + + override fun setUp() { + super.setUp() + createTestIndex() + } + + protected fun getAllIndicesFromPattern(pattern: String): List { + val getIndexResponse = ( + client().admin().indices().prepareGetIndex() + .setIndices(pattern) as GetIndexRequestBuilder + ).get() as GetIndexResponse + getIndexResponse + return getIndexResponse.indices().toList() + } + + protected fun executeMonitor(monitor: Monitor, id: String?, dryRun: Boolean = true): ExecuteMonitorResponse? { + val request = ExecuteMonitorRequest(dryRun, TimeValue(Instant.now().toEpochMilli()), id, monitor) + return client().execute(ExecuteMonitorAction.INSTANCE, request).get() + } + + /** A test index that can be used across tests. Feel free to add new fields but don't remove any. */ + protected fun createTestIndex() { + createIndex( + index, Settings.EMPTY, + """ + "properties" : { + "test_strict_date_time" : { "type" : "date", "format" : "strict_date_time" }, + "test_field" : { "type" : "keyword" } + } + """.trimIndent() + ) + } + + protected fun indexDoc(index: String, id: String, doc: String) { + client().prepareIndex(index).setId(id) + .setSource(doc, XContentType.JSON).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get() + } + + protected fun assertIndexExists(index: String) { + val getIndexResponse = + client().admin().indices().getIndex( + GetIndexRequest().indices(index).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN) + ).get() + assertTrue(getIndexResponse.indices.size > 0) + } + + protected fun assertIndexNotExists(index: String) { + val getIndexResponse = + client().admin().indices().getIndex( + GetIndexRequest().indices(index).indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_HIDDEN) + ).get() + assertFalse(getIndexResponse.indices.size > 0) + } + + protected fun assertAliasExists(alias: String) { + val aliasesResponse = client().admin().indices().getAliases(GetAliasesRequest()).get() + val foundAlias = aliasesResponse.aliases.values().forEach { + it.value.forEach { + if (it.alias == alias) { + return + } + } + } + fail("alias doesn't exists, but it should") + } + + protected fun createMonitor(monitor: Monitor): IndexMonitorResponse? { + val request = IndexMonitorRequest( + monitorId = Monitor.NO_ID, + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, + refreshPolicy = WriteRequest.RefreshPolicy.parse("true"), + method = RestRequest.Method.POST, + monitor = monitor + ) + return client().execute(IndexMonitorAction.INSTANCE, request).actionGet() + } + + protected fun updateMonitor(monitor: Monitor, monitorId: String): IndexMonitorResponse? { + val request = IndexMonitorRequest( + monitorId = monitorId, + seqNo = SequenceNumbers.UNASSIGNED_SEQ_NO, + primaryTerm = SequenceNumbers.UNASSIGNED_PRIMARY_TERM, + refreshPolicy = WriteRequest.RefreshPolicy.parse("true"), + method = RestRequest.Method.PUT, + monitor = monitor + ) + return client().execute(IndexMonitorAction.INSTANCE, request).actionGet() + } + + protected fun deleteMonitor(monitorId: String): Boolean { + client().execute( + DeleteMonitorAction.INSTANCE, DeleteMonitorRequest(monitorId, WriteRequest.RefreshPolicy.IMMEDIATE) + ).get() + return true + } + + protected fun searchAlerts(id: String, indices: String = AlertIndices.ALERT_INDEX, refresh: Boolean = true): List { + try { + if (refresh) refreshIndex(indices) + } catch (e: Exception) { + logger.warn("Could not refresh index $indices because: ${e.message}") + return emptyList() + } + val ssb = SearchSourceBuilder() + ssb.version(true) + ssb.query(TermQueryBuilder(Alert.MONITOR_ID_FIELD, id)) + val searchResponse = client().prepareSearch(indices).setRouting(id).setSource(ssb).get() + + return searchResponse.hits.hits.map { + val xcp = createParser(JsonXContent.jsonXContent, it.sourceRef).also { it.nextToken() } + Alert.parse(xcp, it.id, it.version) + } + } + + protected fun refreshIndex(index: String) { + client().execute(RefreshAction.INSTANCE, RefreshRequest(index)).get() + } + + protected fun searchFindings( + id: String, + indices: String = AlertIndices.ALL_FINDING_INDEX_PATTERN, + refresh: Boolean = true + ): List { + if (refresh) refreshIndex(indices) + + val ssb = SearchSourceBuilder() + ssb.version(true) + ssb.query(TermQueryBuilder(Alert.MONITOR_ID_FIELD, id)) + val searchResponse = client().prepareSearch(indices).setRouting(id).setSource(ssb).get() + + return searchResponse.hits.hits.map { + val xcp = createParser(JsonXContent.jsonXContent, it.sourceRef).also { it.nextToken() } + Finding.parse(xcp) + }.filter { finding -> finding.monitorId == id } + } + + protected fun getFindings( + findingId: String + ): List { + + val getFindingsRequest = GetFindingsRequest( + findingId, + Table("asc", "monitor_id", null, 100, 0, null) + ) + val getFindingsResponse: GetFindingsResponse = client().execute(GetFindingsAction.INSTANCE, getFindingsRequest).get() + + return getFindingsResponse.findings.map { it.finding }.toList() + } + + protected fun getMonitorResponse( + monitorId: String, + version: Long = 1L, + fetchSourceContext: FetchSourceContext = FetchSourceContext.FETCH_SOURCE + ) = client().execute( + GetMonitorAction.INSTANCE, + GetMonitorRequest(monitorId, version, RestRequest.Method.GET, fetchSourceContext) + ).get() + + override fun getPlugins(): List> { + return listOf(AlertingPlugin::class.java, ReindexPlugin::class.java, ParentJoinPlugin::class.java) + } + + override fun resetNodeAfterTest(): Boolean { + return false + } +} diff --git a/alerting/src/test/resources/bwc/alerting/1.1.0.0/opensearch-alerting-1.1.0.0.zip b/alerting/src/test/resources/bwc/alerting/1.1.0.0/opensearch-alerting-1.1.0.0.zip new file mode 100644 index 000000000..29afea2cd Binary files /dev/null and b/alerting/src/test/resources/bwc/alerting/1.1.0.0/opensearch-alerting-1.1.0.0.zip differ diff --git a/alerting/src/test/resources/notifications-core/opensearch-notifications-core-2.3.0.0.zip b/alerting/src/test/resources/notifications-core/opensearch-notifications-core-2.3.0.0.zip new file mode 100644 index 000000000..38788c299 Binary files /dev/null and b/alerting/src/test/resources/notifications-core/opensearch-notifications-core-2.3.0.0.zip differ diff --git a/alerting/src/test/resources/notifications/opensearch-notifications-2.3.0.0.zip b/alerting/src/test/resources/notifications/opensearch-notifications-2.3.0.0.zip new file mode 100644 index 000000000..4dfd822a9 Binary files /dev/null and b/alerting/src/test/resources/notifications/opensearch-notifications-2.3.0.0.zip differ diff --git a/core/src/main/kotlin/org/opensearch/alerting/opensearchapi/OpenSearchExtensions.kt b/core/src/main/kotlin/org/opensearch/alerting/opensearchapi/OpenSearchExtensions.kt index a49181292..5698b26a8 100644 --- a/core/src/main/kotlin/org/opensearch/alerting/opensearchapi/OpenSearchExtensions.kt +++ b/core/src/main/kotlin/org/opensearch/alerting/opensearchapi/OpenSearchExtensions.kt @@ -19,7 +19,6 @@ import org.opensearch.client.OpenSearchClient import org.opensearch.common.bytes.BytesReference import org.opensearch.common.settings.Settings import org.opensearch.common.util.concurrent.ThreadContext -import org.opensearch.common.util.concurrent.ThreadContext.StoredContext import org.opensearch.common.xcontent.ToXContent import org.opensearch.common.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentHelper @@ -207,28 +206,13 @@ suspend fun NotificationsPluginInterface.suspendUntil(block: NotificationsPl }) } -/** - * Store a [ThreadContext] and restore a [ThreadContext] when the coroutine resumes on a different thread. - * - * @param threadContext - a [ThreadContext] instance - */ -class ElasticThreadContextElement(private val threadContext: ThreadContext) : ThreadContextElement { - - companion object Key : CoroutineContext.Key - private var context: StoredContext = threadContext.newStoredContext(true) - - override val key: CoroutineContext.Key<*> - get() = Key - - override fun restoreThreadContext(context: CoroutineContext, oldState: Unit) { - this.context = threadContext.stashContext() - } - - override fun updateThreadContext(context: CoroutineContext) = this.context.close() -} - -class InjectorContextElement(id: String, settings: Settings, threadContext: ThreadContext, private val roles: List?) : - ThreadContextElement { +class InjectorContextElement( + id: String, + settings: Settings, + threadContext: ThreadContext, + private val roles: List?, + private val user: User? = null +) : ThreadContextElement { companion object Key : CoroutineContext.Key override val key: CoroutineContext.Key<*> @@ -238,6 +222,8 @@ class InjectorContextElement(id: String, settings: Settings, threadContext: Thre override fun updateThreadContext(context: CoroutineContext) { rolesInjectorHelper.injectRoles(roles) + // This is from where plugins extract backend roles. It should be passed when calling APIs of other plugins + rolesInjectorHelper.injectUserInfo(user) } override fun restoreThreadContext(context: CoroutineContext, oldState: Unit) { diff --git a/core/src/main/resources/mappings/scheduled-jobs.json b/core/src/main/resources/mappings/scheduled-jobs.json index 9771686ff..5778f6eff 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": 5 + "schema_version": 6 }, "properties": { "monitor": { @@ -463,6 +463,10 @@ "last_run_context": { "type": "object", "enabled": false + }, + "source_to_query_index_mapping": { + "type": "object", + "enabled": false } } } diff --git a/core/src/main/resources/settings/doc-level-queries.json b/core/src/main/resources/settings/doc-level-queries.json new file mode 100644 index 000000000..c5cbfa445 --- /dev/null +++ b/core/src/main/resources/settings/doc-level-queries.json @@ -0,0 +1,10 @@ +{ + "index": { + "mapping": { + "total_fields": { + "limit": 10000 + } + }, + "hidden": true + } +} \ No newline at end of file