From 0ea7f53f574641c11261307e35ff721d13178655 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 17 Aug 2023 23:28:12 -0700 Subject: [PATCH 01/44] added response interceptor Signed-off-by: Ronnak Saxena --- .../indexmanagement/IndexManagementPlugin.kt | 5 +- .../rollup/interceptor/ResponseInterceptor.kt | 74 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) create mode 100644 src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt diff --git a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt index bea66041f..b2d132674 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt @@ -113,6 +113,7 @@ import org.opensearch.indexmanagement.rollup.action.start.TransportStartRollupAc import org.opensearch.indexmanagement.rollup.action.stop.StopRollupAction import org.opensearch.indexmanagement.rollup.action.stop.TransportStopRollupAction import org.opensearch.indexmanagement.rollup.actionfilter.FieldCapsFilter +import org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptor import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.model.RollupMetadata @@ -208,6 +209,7 @@ class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, ActionPlugin lateinit var clusterService: ClusterService lateinit var indexNameExpressionResolver: IndexNameExpressionResolver lateinit var rollupInterceptor: RollupInterceptor + lateinit var responseInterceptor: ResponseInterceptor lateinit var fieldCapsFilter: FieldCapsFilter lateinit var indexMetadataProvider: IndexMetadataProvider private val indexMetadataServices: MutableList> = mutableListOf() @@ -391,6 +393,7 @@ class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, ActionPlugin environment ) rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver) + responseInterceptor = ResponseInterceptor(clusterService, settings, indexNameExpressionResolver) val jvmService = JvmService(environment.settings()) val transformRunner = TransformRunner.initialize( client, @@ -612,7 +615,7 @@ class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, ActionPlugin } override fun getTransportInterceptors(namedWriteableRegistry: NamedWriteableRegistry, threadContext: ThreadContext): List { - return listOf(rollupInterceptor) + return listOf(rollupInterceptor, responseInterceptor) } override fun getActionFilters(): List { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt new file mode 100644 index 000000000..fb0fa0a5d --- /dev/null +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -0,0 +1,74 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.indexmanagement.rollup.interceptor + +import org.apache.logging.log4j.LogManager +import org.opensearch.cluster.metadata.IndexNameExpressionResolver +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.io.stream.StreamInput +import org.opensearch.common.settings.Settings +import org.opensearch.transport.TransportInterceptor +import org.opensearch.transport.TransportResponse +import org.opensearch.transport.TransportRequest +import org.opensearch.transport.Transport +import org.opensearch.transport.TransportRequestOptions +import org.opensearch.transport.TransportResponseHandler +import org.opensearch.transport.TransportException + + +class ResponseInterceptor( + val clusterService: ClusterService, + val settings: Settings, + val indexNameExpressionResolver: IndexNameExpressionResolver +) : TransportInterceptor { + private val logger = LogManager.getLogger(javaClass) + + override fun interceptSender(sender: TransportInterceptor.AsyncSender): TransportInterceptor.AsyncSender { + return CustomAsyncSender(sender) + } + + private inner class CustomAsyncSender(private val originalSender: TransportInterceptor.AsyncSender) : TransportInterceptor.AsyncSender { + + override fun sendRequest( + connection: Transport.Connection?, + action: String?, + request: TransportRequest?, + options: TransportRequestOptions?, + handler: TransportResponseHandler? + ) { + val interceptedHandler = CustomResponseHandler(handler) + + originalSender.sendRequest(connection, action, request, options, interceptedHandler) + } + } + + private inner class CustomResponseHandler( + private val originalHandler: TransportResponseHandler? + ) : TransportResponseHandler { + + override fun read(inStream: StreamInput?): T { + val response = originalHandler?.read(inStream) + // Modify the response if necessary + return response!! + } + + override fun handleResponse(response: T?) { + // Handle the response or delegate to the original handler + logger.error("ronsax response interceptoed!! $response") + originalHandler?.handleResponse(response) + } + + override fun handleException(exp: TransportException?) { + // Handle exceptions or delegate to the original handler + originalHandler?.handleException(exp) + } + + override fun executor(): String { + return originalHandler?.executor() ?: "" + } + + } +} \ No newline at end of file From 4e93b1f907e696f60fac9d262f8acfe55c29c104 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 17 Aug 2023 23:44:21 -0700 Subject: [PATCH 02/44] Base case: Query Live and Rollup data with no overlap Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/RollupInterceptor.kt | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index ffd1e4bd7..1ccdf7995 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -85,13 +85,20 @@ class RollupInterceptor( val index = request.shardId().indexName val isRollupIndex = isRollupIndex(index, clusterService.state()) if (isRollupIndex) { - if (request.source().size() != 0) { - throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") - } +// if (request.source().size() != 0) { +// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") +// } val indices = request.indices().map { it.toString() }.toTypedArray() val concreteIndices = indexNameExpressionResolver .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) + val concreteRolledIndexNames = mutableListOf() + for (indexName in concreteIndices) { + if (isRollupIndex(indexName, clusterService.state())) { + concreteRolledIndexNames.add(indexName) + } + } + val filteredConcreteIndices = concreteRolledIndexNames.toTypedArray() // To extract fields from QueryStringQueryBuilder we need concrete source index name. val rollupJob = clusterService.state().metadata.index(index).getRollupJobs()?.get(0) ?: throw IllegalArgumentException("No rollup job associated with target_index") @@ -102,7 +109,7 @@ class RollupInterceptor( val aggregationFieldMappings = getAggregationMetadata(request.source().aggregations()?.aggregatorFactories) val fieldMappings = queryFieldMappings + aggregationFieldMappings - val allMatchingRollupJobs = validateIndicies(concreteIndices, fieldMappings) + val allMatchingRollupJobs = validateIndicies(filteredConcreteIndices, fieldMappings) // only rebuild if there is necessity to rebuild if (fieldMappings.isNotEmpty()) { @@ -142,7 +149,7 @@ class RollupInterceptor( var allMatchingRollupJobs: Map> = mapOf() for (concreteIndex in concreteIndices) { val rollupJobs = clusterService.state().metadata.index(concreteIndex).getRollupJobs() - ?: throw IllegalArgumentException("Not all indices have rollup job") + ?: throw IllegalArgumentException("Not all indices have rollup job, missing on $concreteIndex") val (matchingRollupJobs, issues) = findMatchingRollupJobs(fieldMappings, rollupJobs) if (issues.isNotEmpty() || matchingRollupJobs.isEmpty()) { From 7fd394a61c091704c485001e390a60239229775f Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Fri, 18 Aug 2023 13:38:41 -0700 Subject: [PATCH 03/44] finished base case and added integ test Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 7 +- .../rollup/interceptor/RollupInterceptorIT.kt | 78 +++++++++++++++++++ 2 files changed, 80 insertions(+), 5 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index fb0fa0a5d..8d36f60a6 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -18,7 +18,6 @@ import org.opensearch.transport.TransportRequestOptions import org.opensearch.transport.TransportResponseHandler import org.opensearch.transport.TransportException - class ResponseInterceptor( val clusterService: ClusterService, val settings: Settings, @@ -48,7 +47,6 @@ class ResponseInterceptor( private inner class CustomResponseHandler( private val originalHandler: TransportResponseHandler? ) : TransportResponseHandler { - override fun read(inStream: StreamInput?): T { val response = originalHandler?.read(inStream) // Modify the response if necessary @@ -57,7 +55,7 @@ class ResponseInterceptor( override fun handleResponse(response: T?) { // Handle the response or delegate to the original handler - logger.error("ronsax response interceptoed!! $response") +// logger.error("ronsax response interceptoed!! $response") originalHandler?.handleResponse(response) } @@ -69,6 +67,5 @@ class ResponseInterceptor( override fun executor(): String { return originalHandler?.executor() ?: "" } - } -} \ No newline at end of file +} diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt index e42295418..171e0cf1b 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt @@ -1876,4 +1876,82 @@ class RollupInterceptorIT : RollupRestTestCase() { Assert.assertTrue(e.message!!.contains("Can't parse query_string query without sourceIndex mappings!")) } } + fun `test search a live index and rollup index with no overlap`() { + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("RatecodeID", "RatecodeID"), + Terms("PULocationID", "PULocationID") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ), + RollupMetrics(sourceField = "total_amount", targetField = "total_amount", metrics = listOf(Max(), Min())) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + + // Delete values from live index + var deleteResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + ) + assertTrue(deleteResponse.restStatus() == RestStatus.OK) + // Term query + var req = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(searchResponse.restStatus() == RestStatus.OK) + var responseAggs = searchResponse.asMap()["aggregations"] as Map> + assertEquals( + "Aggregation from searching both indices is wrong", + 9024.0, + responseAggs.getValue("sum_passenger_count")["value"] + ) + } } From f023a3930ed4f0dd220b0630537d102d7f428af4 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 22 Aug 2023 11:36:21 -0700 Subject: [PATCH 04/44] added to response interceptor Signed-off-by: Ronnak Saxena --- .../indexmanagement/IndexManagementPlugin.kt | 4 +- .../rollup/interceptor/ResponseInterceptor.kt | 277 +++++++++++++++++- .../rollup/interceptor/RollupInterceptor.kt | 151 ++++++++-- 3 files changed, 403 insertions(+), 29 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt index b2d132674..444bafc09 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt @@ -392,8 +392,8 @@ class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, ActionPlugin namedWriteableRegistry, environment ) - rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver) - responseInterceptor = ResponseInterceptor(clusterService, settings, indexNameExpressionResolver) + rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver, xContentRegistry) + responseInterceptor = ResponseInterceptor(clusterService, settings, indexNameExpressionResolver, client) val jvmService = JvmService(environment.settings()) val transformRunner = TransformRunner.initialize( client, diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 8d36f60a6..2bad74440 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -6,10 +6,31 @@ package org.opensearch.indexmanagement.rollup.interceptor import org.apache.logging.log4j.LogManager +import org.opensearch.action.ActionListener +import org.opensearch.action.search.SearchRequest +import org.opensearch.action.search.SearchResponse +import org.opensearch.client.Client import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.Settings +import org.opensearch.index.query.QueryBuilders +import org.opensearch.indexmanagement.common.model.dimension.DateHistogram +import org.opensearch.indexmanagement.rollup.model.Rollup +import org.opensearch.indexmanagement.rollup.util.getRollupJobs +import org.opensearch.indexmanagement.rollup.util.isRollupIndex +import org.opensearch.search.DocValueFormat +import org.opensearch.search.aggregations.InternalAggregation +import org.opensearch.search.aggregations.InternalAggregations +import org.opensearch.search.aggregations.metrics.InternalSum +import org.opensearch.search.aggregations.support.ValuesSource +import org.opensearch.search.aggregations.support.ValuesSourceConfig +import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.search.fetch.QueryFetchSearchResult +import org.opensearch.search.internal.ShardSearchRequest +import org.opensearch.search.query.QuerySearchResult +import org.opensearch.search.sort.SortBuilders +import org.opensearch.search.sort.SortOrder import org.opensearch.transport.TransportInterceptor import org.opensearch.transport.TransportResponse import org.opensearch.transport.TransportRequest @@ -17,11 +38,15 @@ import org.opensearch.transport.Transport import org.opensearch.transport.TransportRequestOptions import org.opensearch.transport.TransportResponseHandler import org.opensearch.transport.TransportException +import java.text.SimpleDateFormat +import java.util.* +import java.util.concurrent.CountDownLatch class ResponseInterceptor( val clusterService: ClusterService, val settings: Settings, - val indexNameExpressionResolver: IndexNameExpressionResolver + val indexNameExpressionResolver: IndexNameExpressionResolver, + val client: Client ) : TransportInterceptor { private val logger = LogManager.getLogger(javaClass) @@ -30,6 +55,7 @@ class ResponseInterceptor( } private inner class CustomAsyncSender(private val originalSender: TransportInterceptor.AsyncSender) : TransportInterceptor.AsyncSender { + // Logic for overlap override fun sendRequest( connection: Transport.Connection?, @@ -52,11 +78,254 @@ class ResponseInterceptor( // Modify the response if necessary return response!! } + fun isRewrittenInterceptorRequest(response: QuerySearchResult): Boolean { + val currentAggregations = response.aggregations().expand() + for (agg in currentAggregations) { + if (agg.name == "interceptor_interval_data") { + return true + } + } + return false + } + fun getRollupJob(response: QuerySearchResult): Rollup? { + val originalRequest = response.shardSearchRequest!! + val indices = originalRequest.indices().map { it.toString() }.toTypedArray() + val allIndices = indexNameExpressionResolver + .concreteIndexNames(clusterService.state(), originalRequest.indicesOptions(), *indices) + for (index in allIndices) { + if (isRollupIndex(index, clusterService.state())) { + return clusterService.state().metadata.index(index).getRollupJobs()?.get(0)!! + } + } + return null + } + fun getRollupAndLiveIndices(request:ShardSearchRequest): Pair, Array> { + val liveIndices = mutableListOf() + val rollupIndices = mutableListOf() + val indices = request.indices().map { it.toString() }.toTypedArray() + val concreteIndices = indexNameExpressionResolver + .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) + for (indexName in concreteIndices) { + if (isRollupIndex(indexName, clusterService.state())) { + rollupIndices.add(indexName) + } else { + liveIndices.add(indexName) + } + } + return Pair(rollupIndices.toTypedArray(), liveIndices.toTypedArray()) + } + private fun convertEpochMillisToDateString(epochMillis: Long): String { + val pattern = "yyyy-MM-dd HH:mm:ss" + val dateFormat = SimpleDateFormat(pattern) + val date = Date(epochMillis) + val dateString = dateFormat.format(date) + return dateString + } + private fun convertDateStringToEpochMillis(dateString: String): Long { + val pattern = "yyyy-MM-dd HH:mm:ss" + val dateFormat = SimpleDateFormat(pattern) + + try { + val date = dateFormat.parse(dateString) + return (date.time) + } catch (e: Exception) { + println("Error parsing date: ${e.message}") + } + return 0L + } + private fun convertFixedIntervalStringToMs(fixedInterval:String): Long { + // Possible types are ms, s, m, h, d + val regex = """(\d+)([a-zA-Z]+)""".toRegex() + val matchResult = regex.find(fixedInterval) + ?: throw IllegalArgumentException("Invalid interval format: $fixedInterval") + + val numericValue = matchResult.groupValues[1].toLong() + val intervalType = matchResult.groupValues[2] + + val milliseconds = when (intervalType) { + "ms" -> numericValue + "s" -> numericValue * 1000L + "m" -> numericValue * 60 * 1000L + "h" -> numericValue * 60 * 60 * 1000L + "d" -> numericValue * 24 * 60 * 60 * 1000L + "w" -> numericValue * 7 * 24 * 60 * 60 * 1000L + else -> throw IllegalArgumentException("Unsupported interval type: $intervalType") + } + + return milliseconds + } +// Returns Pair(startRange: Long, endRange: Long) +// Note startRange is inclusive and endRange is exclusive, they are longs becuase the type is epoch milliseconds + fun findOverlap(response: QuerySearchResult): Pair { + val job:Rollup = getRollupJob(response)!! // maybe throw a try catch later + var dateSourceField: String = "" + var dateTargetField: String = "" + var rollupInterval: String? = "" + for (dim in job.dimensions) { + if (dim is DateHistogram) { + dateSourceField = dim.sourceField + dateTargetField = dim.targetField + rollupInterval = dim.fixedInterval + break + } + } + // Keep existing query and add 3 fake match alls to avoid infinite loop + val request:ShardSearchRequest = response.shardSearchRequest!! + val oldQuery = request.source().query() + val fakeQuery = QueryBuilders.boolQuery() + .must(oldQuery ?: QueryBuilders.matchAllQuery()) + .must(QueryBuilders.matchAllQuery()) + .must(QueryBuilders.matchAllQuery()) + .must(QueryBuilders.matchAllQuery()) + // TODO scale this for multiple indices!!!! + val (rollupIndices, liveIndices) = getRollupAndLiveIndices(request) + val rollupIndex = rollupIndices[0] + val liveIndex = liveIndices[0] + // Build search request to find the minimum date in the live data index + var sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) + var searchSourceBuilder = SearchSourceBuilder() + .sort(sort) + .size(1) + .query(fakeQuery) + val minLiveDateRequest = SearchRequest() + .indices(liveIndex) + .source(searchSourceBuilder) + + var minLiveDateResponse: SearchResponse? = null + var latch = CountDownLatch(1) + client.search( + minLiveDateRequest, + object : ActionListener { + override fun onResponse(searchResponse: SearchResponse) { + minLiveDateResponse = searchResponse + latch.countDown() + } + + override fun onFailure(e: Exception) { + logger.error("ronsax minLiveDate request failed ", e) + latch.countDown() + } + }) + latch.await() + + // Build search request to find the maximum date on the rolled data index + sort = SortBuilders.fieldSort("${dateTargetField}.date_histogram").order(SortOrder.DESC) + searchSourceBuilder = SearchSourceBuilder() + .sort(sort) + .query(fakeQuery) + .size(1) +// .query(QueryBuilders.wrapperQuery(queryJson)) + // Need to avoid infinite interceptor loop + val maxRolledDateRequest = SearchRequest() + .indices(rollupIndex) + .source(searchSourceBuilder) + var maxRolledDateResponse: SearchResponse? = null + latch = CountDownLatch(1) + client.search( + maxRolledDateRequest, + object : ActionListener { + override fun onResponse(searchResponse: SearchResponse) { + maxRolledDateResponse = searchResponse + latch.countDown() + } + + override fun onFailure(e: Exception) { + logger.error("ronsax maxLiveDate request failed ", e) + latch.countDown() + } + }) + latch.await() + + if (minLiveDateResponse != null && maxRolledDateResponse != null) { + // Rollup data ends at maxRolledDate + fixedInterval + val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("${dateTargetField}.date_histogram") as Long + val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) + val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String + val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) + if (liveDataStartPoint <= rollupDataEndPoint) { + // Find intersection timestamp + val intersectionTime = maxRolledDate + val shardRequestIndex = request.shardId().indexName + if (shardRequestIndex == liveIndex) { + // Start at intersection timestamp, end at inf + return Pair(intersectionTime, Long.MAX_VALUE) + } else if (shardRequestIndex == rollupIndex) { + // Start at 0, end at intersection time + return Pair(0L, intersectionTime) + } + } + } else { + logger.error("Not able to make client calls in response interceptor") + } + // No overlap so start and end include everything + return Pair(0L, Long.MAX_VALUE) + } + +// Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) +// fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { +// /* +// PSUEDOCODE +// 1. Look at first bucket to see which aggs where in initial request +// 2. Store in a map of {aggregationName: [aggType, runningSum/Min/Max/Avg to change] +// 3. Iterate through all buckets +// 4. if bucket in range start <= timeStamp < end +// 5. update all computation values in map +// 6. init new InternalAggregations object +// 7. iterate throguh all key, vals in map and construct an internalAggregation object for each of them, add to InternalAggregations object +// 8. return InternalAggregations object +// */ +// return InternalAggregations() +// } + @Suppress("UNCHECKED_CAST") override fun handleResponse(response: T?) { - // Handle the response or delegate to the original handler -// logger.error("ronsax response interceptoed!! $response") - originalHandler?.handleResponse(response) + // Handle the response if it came from intercpetor + when (response) { + // live index + is QuerySearchResult -> { + if (response.hasAggs() && isRewrittenInterceptorRequest(response)) { + // Check for overlap + val latch2 = CountDownLatch(1) + val (startTime, endTime) = findOverlap(response) + latch2.countDown() + latch2.await() + logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") + // Modify agg to be original result without overlap computed in + // TODO handle overlap here + response.aggregations() + // TODO create a copy of the QuerySearchResult with aggregations modified +// val newQuerySerach = QuerySearchResult() +// val responseForHandler = newQuerySerach as T + + originalHandler?.handleResponse(response) + } else { + originalHandler?.handleResponse(response) + } + } + // rollup index + is QueryFetchSearchResult -> { + val queryResult = response.queryResult() + if (queryResult.hasAggs() && isRewrittenInterceptorRequest(queryResult)) { + // Check for overlap + val latch2 = CountDownLatch(1) + val (startTime, endTime) = findOverlap(queryResult) + latch2.countDown() + latch2.await() + logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") + // Modify agg to be original result without overlap computed in + + // TODO handle overlap here + val r1: T = QueryFetchSearchResult(response.queryResult(), response.fetchResult()) as T + originalHandler?.handleResponse(r1) + } else { + originalHandler?.handleResponse(response) + } + } + else -> { + // Delegate to original handler + originalHandler?.handleResponse(response) + } + } } override fun handleException(exp: TransportException?) { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 1ccdf7995..1117328a8 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -10,7 +10,16 @@ import org.opensearch.action.support.IndicesOptions import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.bytes.BytesReference +import org.opensearch.common.io.stream.BytesStreamOutput import org.opensearch.common.settings.Settings +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.common.xcontent.XContentFactory +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.XContentType +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.index.query.BoolQueryBuilder import org.opensearch.index.query.BoostingQueryBuilder import org.opensearch.index.query.ConstantScoreQueryBuilder @@ -18,11 +27,13 @@ import org.opensearch.index.query.DisMaxQueryBuilder import org.opensearch.index.query.MatchAllQueryBuilder import org.opensearch.index.query.MatchPhraseQueryBuilder import org.opensearch.index.query.QueryBuilder +import org.opensearch.index.query.QueryBuilders import org.opensearch.index.query.QueryStringQueryBuilder import org.opensearch.index.query.RangeQueryBuilder import org.opensearch.index.query.TermQueryBuilder import org.opensearch.index.query.TermsQueryBuilder import org.opensearch.index.search.MatchQuery +import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.common.model.dimension.Dimension import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping @@ -35,7 +46,7 @@ import org.opensearch.indexmanagement.rollup.util.isRollupIndex import org.opensearch.indexmanagement.rollup.util.populateFieldMappings import org.opensearch.indexmanagement.rollup.util.rewriteSearchSourceBuilder import org.opensearch.indexmanagement.util.IndexUtils -import org.opensearch.search.aggregations.AggregationBuilder +import org.opensearch.search.aggregations.* import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval import org.opensearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder @@ -46,6 +57,8 @@ import org.opensearch.search.aggregations.metrics.MinAggregationBuilder import org.opensearch.search.aggregations.metrics.SumAggregationBuilder import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder import org.opensearch.search.internal.ShardSearchRequest +import org.opensearch.search.aggregations.pipeline.SumBucketPipelineAggregationBuilder +import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.tasks.Task import org.opensearch.transport.TransportChannel import org.opensearch.transport.TransportInterceptor @@ -55,7 +68,8 @@ import org.opensearch.transport.TransportRequestHandler class RollupInterceptor( val clusterService: ClusterService, val settings: Settings, - val indexNameExpressionResolver: IndexNameExpressionResolver + val indexNameExpressionResolver: IndexNameExpressionResolver, + val xContentRegistry: NamedXContentRegistry ) : TransportInterceptor { private val logger = LogManager.getLogger(javaClass) @@ -71,6 +85,83 @@ class RollupInterceptor( searchAllJobs = it } } + // Returns Pair + private fun originalSearchContainsRollup(request: ShardSearchRequest): Pair { + val indices = request.indices().map { it.toString() }.toTypedArray() + val allIndices = indexNameExpressionResolver + .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) + for (index in allIndices) { + if (isRollupIndex(index, clusterService.state())) { + val rollupJob = clusterService.state().metadata.index(index).getRollupJobs()?.get(0) + return Pair(true, rollupJob) + } + } + return Pair(false, null) + } + // Need to determine if this was an internal client call to avoid infinite loop from interceptor, -> query string doesn't include "query" + fun isInternalSearchRequest(request: ShardSearchRequest): Boolean { + if (request.source().query() != null) { + val jsonRequest: String = request.source().query().toString() + // Detected dummy field from internal search request + if (jsonRequest.contains(",\n" + + " {\n" + + " \"match_all\" : {\n" + + " \"boost\" : 1.0\n" + + " }\n" + + " },\n" + + " {\n" + + " \"match_all\" : {\n" + + " \"boost\" : 1.0\n" + + " }\n" + + " },\n" + + " {\n" + + " \"match_all\" : {\n" + + " \"boost\" : 1.0\n" + + " }\n" + + " }")) { + return true + } + } + return false + } + fun copyAggregations(oldAggs: AggregatorFactories.Builder): AggregatorFactories.Builder { + val xContentBuilder = XContentFactory.jsonBuilder() +// val bytesReference: BytesReference = BytesReference.bytes(oldAggs) + val bytesReference = BytesReference.bytes(oldAggs.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS)) + val parser = XContentHelper.createParser( + xContentRegistry, + LoggingDeprecationHandler.INSTANCE, bytesReference, XContentType.JSON + ) + return AggregatorFactories.parseAggregators(parser) + } + + // Wrap original aggregations into buckets based on fixed interval to remove overlap in response interceptor + fun breakRequestIntoBuckets(request: ShardSearchRequest, rollupJob: Rollup) { + val oldAggs = copyAggregations(request.source().aggregations()) + logger.error("ronsax oldAggs is now $oldAggs") + var dateSourceField: String = "" +// var dateTargetField: String = "" + var rollupInterval: String = "" + for (dim in rollupJob.dimensions) { + if (dim is DateHistogram) { + dateSourceField = dim.sourceField +// dateTargetField = dim.targetField + rollupInterval = dim.fixedInterval!! + break + } + } + // Wraps all existing aggs in bucket aggregation + // Notifies the response interceptor that was rewritten since agg name is interceptor_interval_data + // Edge case if User selected this as the aggregation name :/ + val intervalAgg = AggregationBuilders.dateHistogram("interceptor_interval_data") + .field(dateSourceField) + .calendarInterval(DateHistogramInterval(rollupInterval)) + .format("epoch_millis") + .subAggregations(oldAggs) + request.source().aggregation(intervalAgg) + return + } + @Suppress("SpreadOperator") override fun interceptHandler( @@ -82,38 +173,52 @@ class RollupInterceptor( return object : TransportRequestHandler { override fun messageReceived(request: T, channel: TransportChannel, task: Task) { if (searchEnabled && request is ShardSearchRequest) { - val index = request.shardId().indexName - val isRollupIndex = isRollupIndex(index, clusterService.state()) - if (isRollupIndex) { -// if (request.source().size() != 0) { -// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") -// } - + val (containsRollup, rollupJob) = originalSearchContainsRollup(request) + val shardRequestIndex = request.shardId().indexName + val isRollupIndex = isRollupIndex(shardRequestIndex, clusterService.state()) + // Only modifies rollup searches and avoids internal client calls + if (containsRollup || isRollupIndex) { val indices = request.indices().map { it.toString() }.toTypedArray() val concreteIndices = indexNameExpressionResolver .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) val concreteRolledIndexNames = mutableListOf() + val concreteLiveIndexNames = mutableListOf() for (indexName in concreteIndices) { if (isRollupIndex(indexName, clusterService.state())) { concreteRolledIndexNames.add(indexName) + } else { + concreteLiveIndexNames.add(indexName) } } - val filteredConcreteIndices = concreteRolledIndexNames.toTypedArray() - // To extract fields from QueryStringQueryBuilder we need concrete source index name. - val rollupJob = clusterService.state().metadata.index(index).getRollupJobs()?.get(0) - ?: throw IllegalArgumentException("No rollup job associated with target_index") - val queryFieldMappings = getQueryMetadata( - request.source().query(), - getConcreteSourceIndex(rollupJob.sourceIndex, indexNameExpressionResolver, clusterService.state()) - ) - val aggregationFieldMappings = getAggregationMetadata(request.source().aggregations()?.aggregatorFactories) - val fieldMappings = queryFieldMappings + aggregationFieldMappings + val concreteRollupIndicesArray = concreteRolledIndexNames.toTypedArray() + val concreteLiveIndicesArray = concreteLiveIndexNames.toTypedArray() + // Check before rewriting rollup because it deletes dummy field + val requestCalledInInterceptor = isInternalSearchRequest(request) + // Rewrite the request to fit rollup format + if (isRollupIndex) { +// if (!requestCalledInInterceptor && request.source().size() != 0) { +// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") +// } + // To extract fields from QueryStringQueryBuilder we need concrete source index name. + val queryFieldMappings = getQueryMetadata( + request.source().query(), + getConcreteSourceIndex(rollupJob!!.sourceIndex, indexNameExpressionResolver, clusterService.state()) + ) + val aggregationFieldMappings = getAggregationMetadata(request.source().aggregations()?.aggregatorFactories) + val fieldMappings = queryFieldMappings + aggregationFieldMappings - val allMatchingRollupJobs = validateIndicies(filteredConcreteIndices, fieldMappings) + val allMatchingRollupJobs = validateIndicies(concreteRollupIndicesArray, fieldMappings) - // only rebuild if there is necessity to rebuild - if (fieldMappings.isNotEmpty()) { - rewriteShardSearchForRollupJobs(request, allMatchingRollupJobs) + // only rebuild if there is necessity to rebuild + if (fieldMappings.isNotEmpty()) { + rewriteShardSearchForRollupJobs(request, allMatchingRollupJobs) + } + } + // Avoid infinite interceptor loop + // Need to break into buckets for agg search on live and rollup indices + if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor) { + // Break apart request to remove overlapping parts + breakRequestIntoBuckets(request, rollupJob!!) } } } From 4aa4202d0fc1973a75df536dbee66262fffb30a5 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 22 Aug 2023 16:17:37 -0700 Subject: [PATCH 05/44] can rewrite request to bucket pipeline Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 10 +- .../rollup/interceptor/RollupInterceptor.kt | 109 ++++++++++++------ 2 files changed, 74 insertions(+), 45 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 2bad74440..e5a30511e 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -275,7 +275,9 @@ class ResponseInterceptor( // 7. iterate throguh all key, vals in map and construct an internalAggregation object for each of them, add to InternalAggregations object // 8. return InternalAggregations object // */ -// return InternalAggregations() +// intervalAggregations.get<>() +// val result = listOf(intervalAggregations) +// return InternalAggregations(intervalAggregations, null) // } @Suppress("UNCHECKED_CAST") override fun handleResponse(response: T?) { @@ -285,10 +287,7 @@ class ResponseInterceptor( is QuerySearchResult -> { if (response.hasAggs() && isRewrittenInterceptorRequest(response)) { // Check for overlap - val latch2 = CountDownLatch(1) val (startTime, endTime) = findOverlap(response) - latch2.countDown() - latch2.await() logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") // Modify agg to be original result without overlap computed in // TODO handle overlap here @@ -307,10 +306,7 @@ class ResponseInterceptor( val queryResult = response.queryResult() if (queryResult.hasAggs() && isRewrittenInterceptorRequest(queryResult)) { // Check for overlap - val latch2 = CountDownLatch(1) val (startTime, endTime) = findOverlap(queryResult) - latch2.countDown() - latch2.await() logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") // Modify agg to be original result without overlap computed in diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 1117328a8..e5bb7e60c 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -6,32 +6,18 @@ package org.opensearch.indexmanagement.rollup.interceptor import org.apache.logging.log4j.LogManager +import org.opensearch.Version import org.opensearch.action.support.IndicesOptions import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.bytes.BytesReference import org.opensearch.common.io.stream.BytesStreamOutput +import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput +import org.opensearch.common.io.stream.NamedWriteableRegistry import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.LoggingDeprecationHandler -import org.opensearch.common.xcontent.XContentFactory -import org.opensearch.common.xcontent.XContentHelper -import org.opensearch.common.xcontent.XContentType import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.core.xcontent.ToXContent -import org.opensearch.core.xcontent.XContentBuilder -import org.opensearch.index.query.BoolQueryBuilder -import org.opensearch.index.query.BoostingQueryBuilder -import org.opensearch.index.query.ConstantScoreQueryBuilder -import org.opensearch.index.query.DisMaxQueryBuilder -import org.opensearch.index.query.MatchAllQueryBuilder -import org.opensearch.index.query.MatchPhraseQueryBuilder -import org.opensearch.index.query.QueryBuilder -import org.opensearch.index.query.QueryBuilders -import org.opensearch.index.query.QueryStringQueryBuilder -import org.opensearch.index.query.RangeQueryBuilder -import org.opensearch.index.query.TermQueryBuilder -import org.opensearch.index.query.TermsQueryBuilder +import org.opensearch.index.query.* import org.opensearch.index.search.MatchQuery import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.common.model.dimension.Dimension @@ -40,30 +26,23 @@ import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping.Companion.UNKNOWN_MAPPING import org.opensearch.indexmanagement.rollup.query.QueryStringQueryUtil import org.opensearch.indexmanagement.rollup.settings.RollupSettings -import org.opensearch.indexmanagement.rollup.util.getDateHistogram -import org.opensearch.indexmanagement.rollup.util.getRollupJobs -import org.opensearch.indexmanagement.rollup.util.isRollupIndex -import org.opensearch.indexmanagement.rollup.util.populateFieldMappings -import org.opensearch.indexmanagement.rollup.util.rewriteSearchSourceBuilder +import org.opensearch.indexmanagement.rollup.util.* import org.opensearch.indexmanagement.util.IndexUtils +import org.opensearch.search.SearchModule import org.opensearch.search.aggregations.* import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval import org.opensearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder import org.opensearch.search.aggregations.bucket.terms.TermsAggregationBuilder -import org.opensearch.search.aggregations.metrics.AvgAggregationBuilder -import org.opensearch.search.aggregations.metrics.MaxAggregationBuilder -import org.opensearch.search.aggregations.metrics.MinAggregationBuilder -import org.opensearch.search.aggregations.metrics.SumAggregationBuilder -import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder +import org.opensearch.search.aggregations.metrics.* import org.opensearch.search.internal.ShardSearchRequest -import org.opensearch.search.aggregations.pipeline.SumBucketPipelineAggregationBuilder -import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.tasks.Task import org.opensearch.transport.TransportChannel import org.opensearch.transport.TransportInterceptor import org.opensearch.transport.TransportRequest import org.opensearch.transport.TransportRequestHandler +import java.nio.ByteBuffer +import java.util.* class RollupInterceptor( val clusterService: ClusterService, @@ -124,15 +103,69 @@ class RollupInterceptor( } return false } + // Returns true if request was already modified into "interceptor_interval_data" bucket aggregation + fun isRequestRewritten(request: ShardSearchRequest): Boolean { + val currentAggs = request.source().aggregations().aggregatorFactories + for (agg in currentAggs) { + if (agg.name == "interceptor_interval_data") { + return true + } + } + return false + } fun copyAggregations(oldAggs: AggregatorFactories.Builder): AggregatorFactories.Builder { - val xContentBuilder = XContentFactory.jsonBuilder() + +// val out = BytesStreamOutput() +// oldAggs.writeTo(out) +// +// val sin = StreamInput.wrap(out.bytes().toBytesRef().bytes) +// val loadedAggregators = AggregatorFactories(sin) +// // Create a new AggregatorFactories.Builder and add the loaded aggregators manually +// val newAggsBuilder = AggregatorFactories.builder() +// for (aggregator in loadedAggregators) { +// newAggsBuilder.addAggregator(aggregator) +// } +// +// return newAggsBuilder +// try { +// val xContentBuilder = XContentFactory.jsonBuilder() // val bytesReference: BytesReference = BytesReference.bytes(oldAggs) - val bytesReference = BytesReference.bytes(oldAggs.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS)) - val parser = XContentHelper.createParser( - xContentRegistry, - LoggingDeprecationHandler.INSTANCE, bytesReference, XContentType.JSON - ) - return AggregatorFactories.parseAggregators(parser) +// val bytesReference = BytesReference.bytes(oldAggs.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS)) +// val string = +// oldAggs.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS) +// .string() +// val parser = XContentType.JSON.xContent().createParser( +// NamedXContentRegistry(SearchModule(Settings.EMPTY, emptyList()).namedXContents), LoggingDeprecationHandler.INSTANCE, string) +// return AggregatorFactories.parseAggregators(parser) +// } catch(e:Exception) { +// logger.error(e) +// } +// return oldAggs + try { + val aggsStr = BytesStreamOutput().use { out -> + out.writeVersion(Version.CURRENT) + oldAggs.writeTo(out) + val bytes = BytesReference.toBytes(out.bytes()) + Base64.getUrlEncoder().withoutPadding().encodeToString(bytes) + + } + val bytesReference = BytesReference.fromByteBuffer( + ByteBuffer.wrap( + Base64.getUrlDecoder().decode( + aggsStr + ) + ) + ) + val wrapperStreamInput = NamedWriteableAwareStreamInput( + bytesReference.streamInput(), + NamedWriteableRegistry(SearchModule(Settings.EMPTY, emptyList()).namedWriteables) + ) + wrapperStreamInput.setVersion(wrapperStreamInput.readVersion()) + return AggregatorFactories.Builder(wrapperStreamInput) + } catch (e: Exception) { + logger.error(e) + return oldAggs + } } // Wrap original aggregations into buckets based on fixed interval to remove overlap in response interceptor @@ -216,7 +249,7 @@ class RollupInterceptor( } // Avoid infinite interceptor loop // Need to break into buckets for agg search on live and rollup indices - if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor) { + if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor && !isRequestRewritten(request)) { // Break apart request to remove overlapping parts breakRequestIntoBuckets(request, rollupJob!!) } From c3b5477f0a9cd496eecd27ecc05f75388443197f Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 23 Aug 2023 09:52:08 -0700 Subject: [PATCH 06/44] trying to rewrite aggregations in a helper function Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 2 +- .../rollup/interceptor/RollupInterceptor.kt | 12 ++++++------ .../indexmanagement/rollup/util/RollupUtils.kt | 8 ++++++++ 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index e5a30511e..c61cfbca3 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -291,7 +291,7 @@ class ResponseInterceptor( logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") // Modify agg to be original result without overlap computed in // TODO handle overlap here - response.aggregations() + val aggs = response.aggregations().expand() // TODO create a copy of the QuerySearchResult with aggregations modified // val newQuerySerach = QuerySearchResult() // val responseForHandler = newQuerySerach as T diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index e5bb7e60c..002ffd3a2 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -227,6 +227,12 @@ class RollupInterceptor( val concreteLiveIndicesArray = concreteLiveIndexNames.toTypedArray() // Check before rewriting rollup because it deletes dummy field val requestCalledInInterceptor = isInternalSearchRequest(request) + // Avoid infinite interceptor loop + // Need to break into buckets for agg search on live and rollup indices + if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor && !isRequestRewritten(request)) { + // Break apart request to remove overlapping parts + breakRequestIntoBuckets(request, rollupJob!!) + } // Rewrite the request to fit rollup format if (isRollupIndex) { // if (!requestCalledInInterceptor && request.source().size() != 0) { @@ -247,12 +253,6 @@ class RollupInterceptor( rewriteShardSearchForRollupJobs(request, allMatchingRollupJobs) } } - // Avoid infinite interceptor loop - // Need to break into buckets for agg search on live and rollup indices - if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor && !isRequestRewritten(request)) { - // Break apart request to remove overlapping parts - breakRequestIntoBuckets(request, rollupJob!!) - } } } actualHandler.messageReceived(request, channel, task) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index f6d749fbb..a9c109b1a 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -464,3 +464,11 @@ fun parseRollup(response: GetResponse, xContentRegistry: NamedXContentRegistry = return xcp.parseWithType(response.id, response.seqNo, response.primaryTerm, Rollup.Companion::parse) } +// Changes aggregations in search source builder to new original aggregation +//fun SearchSourceBuilder.rewriteAggregations(aggregationBuilder: AggregationBuilder): AggregationBuilder { +// val aggFactory = AggregatorFactories.builder().also { factories -> +// aggregationBuilder.subAggregations.forEach { +// factories.addAggregator(this.rewriteAggregations(it)) +// } +// } + From 5d20241217da583e30117e465da996ff322f610e Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 23 Aug 2023 14:34:02 -0700 Subject: [PATCH 07/44] able to create new aggreations but getting shardIndex is not set error Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 16 +- .../rollup/interceptor/RollupInterceptor.kt | 142 ++++++++++-------- .../rollup/util/RollupUtils.kt | 42 +++++- 3 files changed, 124 insertions(+), 76 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index c61cfbca3..2c12cf453 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -173,11 +173,11 @@ class ResponseInterceptor( // Keep existing query and add 3 fake match alls to avoid infinite loop val request:ShardSearchRequest = response.shardSearchRequest!! val oldQuery = request.source().query() - val fakeQuery = QueryBuilders.boolQuery() - .must(oldQuery ?: QueryBuilders.matchAllQuery()) - .must(QueryBuilders.matchAllQuery()) - .must(QueryBuilders.matchAllQuery()) - .must(QueryBuilders.matchAllQuery()) +// val fakeQuery = QueryBuilders.boolQuery() +// .must(oldQuery ?: QueryBuilders.matchAllQuery()) +// .must(QueryBuilders.matchAllQuery()) +// .must(QueryBuilders.matchAllQuery()) +// .must(QueryBuilders.matchAllQuery()) // TODO scale this for multiple indices!!!! val (rollupIndices, liveIndices) = getRollupAndLiveIndices(request) val rollupIndex = rollupIndices[0] @@ -187,7 +187,7 @@ class ResponseInterceptor( var searchSourceBuilder = SearchSourceBuilder() .sort(sort) .size(1) - .query(fakeQuery) + .query(oldQuery) val minLiveDateRequest = SearchRequest() .indices(liveIndex) .source(searchSourceBuilder) @@ -213,9 +213,8 @@ class ResponseInterceptor( sort = SortBuilders.fieldSort("${dateTargetField}.date_histogram").order(SortOrder.DESC) searchSourceBuilder = SearchSourceBuilder() .sort(sort) - .query(fakeQuery) + .query(oldQuery) .size(1) -// .query(QueryBuilders.wrapperQuery(queryJson)) // Need to avoid infinite interceptor loop val maxRolledDateRequest = SearchRequest() .indices(rollupIndex) @@ -291,7 +290,6 @@ class ResponseInterceptor( logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") // Modify agg to be original result without overlap computed in // TODO handle overlap here - val aggs = response.aggregations().expand() // TODO create a copy of the QuerySearchResult with aggregations modified // val newQuerySerach = QuerySearchResult() // val responseForHandler = newQuerySerach as T diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 002ffd3a2..6f55931c3 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -78,36 +78,49 @@ class RollupInterceptor( return Pair(false, null) } // Need to determine if this was an internal client call to avoid infinite loop from interceptor, -> query string doesn't include "query" - fun isInternalSearchRequest(request: ShardSearchRequest): Boolean { - if (request.source().query() != null) { - val jsonRequest: String = request.source().query().toString() - // Detected dummy field from internal search request - if (jsonRequest.contains(",\n" + - " {\n" + - " \"match_all\" : {\n" + - " \"boost\" : 1.0\n" + - " }\n" + - " },\n" + - " {\n" + - " \"match_all\" : {\n" + - " \"boost\" : 1.0\n" + - " }\n" + - " },\n" + - " {\n" + - " \"match_all\" : {\n" + - " \"boost\" : 1.0\n" + - " }\n" + - " }")) { - return true - } +// fun isRequestFromResponseInterceptor(request: ShardSearchRequest): Boolean { +// if (request.source().query() != null) { +// val jsonRequest: String = request.source().query().toString() +// // Detected dummy field from internal search request +// if (jsonRequest.contains(",\n" + +// " {\n" + +// " \"match_all\" : {\n" + +// " \"boost\" : 1.0\n" + +// " }\n" + +// " },\n" + +// " {\n" + +// " \"match_all\" : {\n" + +// " \"boost\" : 1.0\n" + +// " }\n" + +// " },\n" + +// " {\n" + +// " \"match_all\" : {\n" + +// " \"boost\" : 1.0\n" + +// " }\n" + +// " }")) { +// return true +// } +// } +// return false +// } + // Returns true if request was already modified into "interceptor_interval_data" bucket aggregation + fun isRequestRewrittenIntoBuckets(request: ShardSearchRequest): Boolean { + val currentAggs = request.source().aggregations().aggregatorFactories + if (currentAggs != null) { + for (agg in currentAggs) { + if (agg.name == "interceptor_interval_data") { + return true + } + } } return false } - // Returns true if request was already modified into "interceptor_interval_data" bucket aggregation - fun isRequestRewritten(request: ShardSearchRequest): Boolean { - val currentAggs = request.source().aggregations().aggregatorFactories - for (agg in currentAggs) { - if (agg.name == "interceptor_interval_data") { + // Helper fn to avoid rewritting a rollup request an extra time + fun isReqeustRollupFormat(request: ShardSearchRequest): Boolean { + if (request.source().query() != null) { + val jsonRequest: String = request.source().query().toString() + // Detected dummy field from internal search request + if (jsonRequest.contains("rollup._id")) { return true } } @@ -191,7 +204,8 @@ class RollupInterceptor( .calendarInterval(DateHistogramInterval(rollupInterval)) .format("epoch_millis") .subAggregations(oldAggs) - request.source().aggregation(intervalAgg) + // Changes aggreagtion in source to new agg + request.source(request.source().changeAggregations(listOf(intervalAgg))) return } @@ -211,47 +225,21 @@ class RollupInterceptor( val isRollupIndex = isRollupIndex(shardRequestIndex, clusterService.state()) // Only modifies rollup searches and avoids internal client calls if (containsRollup || isRollupIndex) { - val indices = request.indices().map { it.toString() }.toTypedArray() - val concreteIndices = indexNameExpressionResolver - .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) - val concreteRolledIndexNames = mutableListOf() - val concreteLiveIndexNames = mutableListOf() - for (indexName in concreteIndices) { - if (isRollupIndex(indexName, clusterService.state())) { - concreteRolledIndexNames.add(indexName) - } else { - concreteLiveIndexNames.add(indexName) - } - } - val concreteRollupIndicesArray = concreteRolledIndexNames.toTypedArray() - val concreteLiveIndicesArray = concreteLiveIndexNames.toTypedArray() + val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) // Check before rewriting rollup because it deletes dummy field - val requestCalledInInterceptor = isInternalSearchRequest(request) +// val requestCalledInInterceptor = isRequestFromResponseInterceptor(request) // Avoid infinite interceptor loop // Need to break into buckets for agg search on live and rollup indices - if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !requestCalledInInterceptor && !isRequestRewritten(request)) { + if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts breakRequestIntoBuckets(request, rollupJob!!) } - // Rewrite the request to fit rollup format - if (isRollupIndex) { + // Rewrite the request to fit rollup format if not already done previously + if (isRollupIndex && !isReqeustRollupFormat(request)) { // if (!requestCalledInInterceptor && request.source().size() != 0) { // throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") // } - // To extract fields from QueryStringQueryBuilder we need concrete source index name. - val queryFieldMappings = getQueryMetadata( - request.source().query(), - getConcreteSourceIndex(rollupJob!!.sourceIndex, indexNameExpressionResolver, clusterService.state()) - ) - val aggregationFieldMappings = getAggregationMetadata(request.source().aggregations()?.aggregatorFactories) - val fieldMappings = queryFieldMappings + aggregationFieldMappings - - val allMatchingRollupJobs = validateIndicies(concreteRollupIndicesArray, fieldMappings) - - // only rebuild if there is necessity to rebuild - if (fieldMappings.isNotEmpty()) { - rewriteShardSearchForRollupJobs(request, allMatchingRollupJobs) - } + rewriteRollupRequest(request, rollupJob!!, concreteRollupIndicesArray) } } } @@ -259,6 +247,40 @@ class RollupInterceptor( } } } + // Returns Pair (concreteRollupIndices: Array, concreteLiveIndicesArray: Array) + fun getConcreteIndices(request: ShardSearchRequest): Pair, Array> { + val indices = request.indices().map { it.toString() }.toTypedArray() + val concreteIndices = indexNameExpressionResolver + .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) + val concreteRollupIndexNames = mutableListOf() + val concreteLiveIndexNames = mutableListOf() + for (indexName in concreteIndices) { + if (isRollupIndex(indexName, clusterService.state())) { + concreteRollupIndexNames.add(indexName) + } else { + concreteLiveIndexNames.add(indexName) + } + } + val concreteRollupIndicesArray = concreteRollupIndexNames.toTypedArray() + val concreteLiveIndicesArray = concreteLiveIndexNames.toTypedArray() + return Pair(concreteRollupIndicesArray, concreteLiveIndicesArray) + } + fun rewriteRollupRequest(request: ShardSearchRequest, rollupJob: Rollup, concreteRollupIndicesArray: Array) { + // To extract fields from QueryStringQueryBuilder we need concrete source index name. + val queryFieldMappings = getQueryMetadata( + request.source().query(), + getConcreteSourceIndex(rollupJob.sourceIndex, indexNameExpressionResolver, clusterService.state()) + ) + val aggregationFieldMappings = getAggregationMetadata(request.source().aggregations()?.aggregatorFactories) + val fieldMappings = queryFieldMappings + aggregationFieldMappings + + val allMatchingRollupJobs = validateIndicies(concreteRollupIndicesArray, fieldMappings) + + // only rebuild if there is necessity to rebuild + if (fieldMappings.isNotEmpty()) { + rewriteShardSearchForRollupJobs(request, allMatchingRollupJobs) + } + } fun getConcreteSourceIndex(sourceIndex: String, resolver: IndexNameExpressionResolver, clusterState: ClusterState): String { val concreteIndexNames = resolver.concreteIndexNames(clusterState, IndicesOptions.LENIENT_EXPAND_OPEN, sourceIndex) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index a9c109b1a..50f6a0fac 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -62,6 +62,7 @@ import org.opensearch.search.aggregations.metrics.ScriptedMetricAggregationBuild import org.opensearch.search.aggregations.metrics.SumAggregationBuilder import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder import org.opensearch.search.builder.SearchSourceBuilder +import java.util.* const val DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT = "strict_date_optional_time" const val DATE_FIELD_EPOCH_MILLIS_FORMAT = "epoch_millis" @@ -464,11 +465,38 @@ fun parseRollup(response: GetResponse, xContentRegistry: NamedXContentRegistry = return xcp.parseWithType(response.id, response.seqNo, response.primaryTerm, Rollup.Companion::parse) } -// Changes aggregations in search source builder to new original aggregation -//fun SearchSourceBuilder.rewriteAggregations(aggregationBuilder: AggregationBuilder): AggregationBuilder { -// val aggFactory = AggregatorFactories.builder().also { factories -> -// aggregationBuilder.subAggregations.forEach { -// factories.addAggregator(this.rewriteAggregations(it)) -// } -// } +// Changes aggregations in search source builder to new original aggregation (Change query too?) +fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collection): SearchSourceBuilder { + val ssb = SearchSourceBuilder() + aggregationBuilderCollection.forEach {ssb.aggregation(it)} + if (this.explain() != null) ssb.explain(this.explain()) + if (this.ext() != null) ssb.ext(this.ext()) + ssb.fetchSource(this.fetchSource()) + this.docValueFields()?.forEach { ssb.docValueField(it.field, it.format) } + ssb.storedFields(this.storedFields()) + if (this.from() >= 0) ssb.from(this.from()) + ssb.highlighter(this.highlighter()) + this.indexBoosts()?.forEach { ssb.indexBoost(it.index, it.boost) } + if (this.minScore() != null) ssb.minScore(this.minScore()) + if (this.postFilter() != null) ssb.postFilter(this.postFilter()) + ssb.profile(this.profile()) + if (this.query() != null) ssb.query(this.query()) + this.rescores()?.forEach { ssb.addRescorer(it) } + this.scriptFields()?.forEach { ssb.scriptField(it.fieldName(), it.script(), it.ignoreFailure()) } + if (this.searchAfter() != null) ssb.searchAfter(this.searchAfter()) + if (this.slice() != null) ssb.slice(this.slice()) + if (this.size() >= 0) ssb.size(this.size()) + this.sorts()?.forEach { ssb.sort(it) } + if (this.stats() != null) ssb.stats(this.stats()) + if (this.suggest() != null) ssb.suggest(this.suggest()) + if (this.terminateAfter() >= 0) ssb.terminateAfter(this.terminateAfter()) + if (this.timeout() != null) ssb.timeout(this.timeout()) + ssb.trackScores(this.trackScores()) + this.trackTotalHitsUpTo()?.let { ssb.trackTotalHitsUpTo(it) } + if (this.version() != null) ssb.version(this.version()) + if (this.seqNoAndPrimaryTerm() != null) ssb.seqNoAndPrimaryTerm(this.seqNoAndPrimaryTerm()) + if (this.collapse() != null) ssb.collapse(this.collapse()) + return ssb +} + From 743263b059668b35173f33b04a2386e755b09bd5 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 23 Aug 2023 16:16:27 -0700 Subject: [PATCH 08/44] Can find start and end times for rollup and live index Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 58 ++++++++--------- .../rollup/interceptor/RollupInterceptor.kt | 64 ++++++++----------- .../rollup/util/RollupUtils.kt | 4 +- 3 files changed, 54 insertions(+), 72 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 2c12cf453..6ad9257f9 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -14,17 +14,10 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.Settings -import org.opensearch.index.query.QueryBuilders import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.util.getRollupJobs import org.opensearch.indexmanagement.rollup.util.isRollupIndex -import org.opensearch.search.DocValueFormat -import org.opensearch.search.aggregations.InternalAggregation -import org.opensearch.search.aggregations.InternalAggregations -import org.opensearch.search.aggregations.metrics.InternalSum -import org.opensearch.search.aggregations.support.ValuesSource -import org.opensearch.search.aggregations.support.ValuesSourceConfig import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest @@ -99,7 +92,7 @@ class ResponseInterceptor( } return null } - fun getRollupAndLiveIndices(request:ShardSearchRequest): Pair, Array> { + fun getRollupAndLiveIndices(request: ShardSearchRequest): Pair, Array> { val liveIndices = mutableListOf() val rollupIndices = mutableListOf() val indices = request.indices().map { it.toString() }.toTypedArray() @@ -114,14 +107,14 @@ class ResponseInterceptor( } return Pair(rollupIndices.toTypedArray(), liveIndices.toTypedArray()) } - private fun convertEpochMillisToDateString(epochMillis: Long): String { + fun convertEpochMillisToDateString(epochMillis: Long): String { val pattern = "yyyy-MM-dd HH:mm:ss" val dateFormat = SimpleDateFormat(pattern) val date = Date(epochMillis) val dateString = dateFormat.format(date) return dateString } - private fun convertDateStringToEpochMillis(dateString: String): Long { + fun convertDateStringToEpochMillis(dateString: String): Long { val pattern = "yyyy-MM-dd HH:mm:ss" val dateFormat = SimpleDateFormat(pattern) @@ -133,7 +126,7 @@ class ResponseInterceptor( } return 0L } - private fun convertFixedIntervalStringToMs(fixedInterval:String): Long { + fun convertFixedIntervalStringToMs(fixedInterval: String): Long { // Possible types are ms, s, m, h, d val regex = """(\d+)([a-zA-Z]+)""".toRegex() val matchResult = regex.find(fixedInterval) @@ -157,8 +150,8 @@ class ResponseInterceptor( // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are longs becuase the type is epoch milliseconds - fun findOverlap(response: QuerySearchResult): Pair { - val job:Rollup = getRollupJob(response)!! // maybe throw a try catch later + fun findOverlap(response: QuerySearchResult): Pair { + val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later var dateSourceField: String = "" var dateTargetField: String = "" var rollupInterval: String? = "" @@ -171,7 +164,7 @@ class ResponseInterceptor( } } // Keep existing query and add 3 fake match alls to avoid infinite loop - val request:ShardSearchRequest = response.shardSearchRequest!! + val request: ShardSearchRequest = response.shardSearchRequest!! val oldQuery = request.source().query() // val fakeQuery = QueryBuilders.boolQuery() // .must(oldQuery ?: QueryBuilders.matchAllQuery()) @@ -187,7 +180,6 @@ class ResponseInterceptor( var searchSourceBuilder = SearchSourceBuilder() .sort(sort) .size(1) - .query(oldQuery) val minLiveDateRequest = SearchRequest() .indices(liveIndex) .source(searchSourceBuilder) @@ -206,11 +198,12 @@ class ResponseInterceptor( logger.error("ronsax minLiveDate request failed ", e) latch.countDown() } - }) + } + ) latch.await() // Build search request to find the maximum date on the rolled data index - sort = SortBuilders.fieldSort("${dateTargetField}.date_histogram").order(SortOrder.DESC) + sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) searchSourceBuilder = SearchSourceBuilder() .sort(sort) .query(oldQuery) @@ -233,14 +226,15 @@ class ResponseInterceptor( logger.error("ronsax maxLiveDate request failed ", e) latch.countDown() } - }) + } + ) latch.await() if (minLiveDateResponse != null && maxRolledDateResponse != null) { // Rollup data ends at maxRolledDate + fixedInterval - val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("${dateTargetField}.date_histogram") as Long + val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long // broken for rollup index val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) - val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String + val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String // broken for rollup index val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) if (liveDataStartPoint <= rollupDataEndPoint) { // Find intersection timestamp @@ -287,37 +281,39 @@ class ResponseInterceptor( if (response.hasAggs() && isRewrittenInterceptorRequest(response)) { // Check for overlap val (startTime, endTime) = findOverlap(response) - logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") + logger.error("ronsax: live index: start $startTime and end $endTime") // Modify agg to be original result without overlap computed in // TODO handle overlap here // TODO create a copy of the QuerySearchResult with aggregations modified // val newQuerySerach = QuerySearchResult() // val responseForHandler = newQuerySerach as T - +// response.shardIndex = response.shardSearchRequest?.shardId()?.id ?: -1 originalHandler?.handleResponse(response) } else { originalHandler?.handleResponse(response) } } - // rollup index + // when just 1 rollup index is in request, keep for testing is QueryFetchSearchResult -> { val queryResult = response.queryResult() if (queryResult.hasAggs() && isRewrittenInterceptorRequest(queryResult)) { // Check for overlap val (startTime, endTime) = findOverlap(queryResult) - logger.error("ronsax: index: ${response.shardIndex} start $startTime and end $endTime") - // Modify agg to be original result without overlap computed in + logger.error("ronsax: rollup index: start $startTime and end $endTime") +// response.shardIndex = response.shardSearchRequest?.shardId()?.id ?: -1 // TODO handle overlap here - val r1: T = QueryFetchSearchResult(response.queryResult(), response.fetchResult()) as T - originalHandler?.handleResponse(r1) + // TODO change response object +// val r1 = QueryFetchSearchResult(response.queryResult(), response.fetchResult()) +// r1.shardIndex = response.shardIndex +// val r2: T = r1 as T + originalHandler?.handleResponse(response) } else { originalHandler?.handleResponse(response) } - } - else -> { - // Delegate to original handler - originalHandler?.handleResponse(response) + } else -> { + // Delegate to original handler + originalHandler?.handleResponse(response) } } } diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 6f55931c3..9f1f0bd51 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -17,7 +17,17 @@ import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput import org.opensearch.common.io.stream.NamedWriteableRegistry import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.index.query.* +import org.opensearch.index.query.QueryBuilder +import org.opensearch.index.query.TermQueryBuilder +import org.opensearch.index.query.TermsQueryBuilder +import org.opensearch.index.query.RangeQueryBuilder +import org.opensearch.index.query.MatchAllQueryBuilder +import org.opensearch.index.query.BoolQueryBuilder +import org.opensearch.index.query.BoostingQueryBuilder +import org.opensearch.index.query.ConstantScoreQueryBuilder +import org.opensearch.index.query.DisMaxQueryBuilder +import org.opensearch.index.query.MatchPhraseQueryBuilder +import org.opensearch.index.query.QueryStringQueryBuilder import org.opensearch.index.search.MatchQuery import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.common.model.dimension.Dimension @@ -26,15 +36,26 @@ import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping.Companion.UNKNOWN_MAPPING import org.opensearch.indexmanagement.rollup.query.QueryStringQueryUtil import org.opensearch.indexmanagement.rollup.settings.RollupSettings -import org.opensearch.indexmanagement.rollup.util.* +import org.opensearch.indexmanagement.rollup.util.isRollupIndex +import org.opensearch.indexmanagement.rollup.util.getRollupJobs +import org.opensearch.indexmanagement.rollup.util.changeAggregations +import org.opensearch.indexmanagement.rollup.util.populateFieldMappings +import org.opensearch.indexmanagement.rollup.util.getDateHistogram +import org.opensearch.indexmanagement.rollup.util.rewriteSearchSourceBuilder import org.opensearch.indexmanagement.util.IndexUtils import org.opensearch.search.SearchModule -import org.opensearch.search.aggregations.* +import org.opensearch.search.aggregations.AggregatorFactories +import org.opensearch.search.aggregations.AggregationBuilder +import org.opensearch.search.aggregations.AggregationBuilders import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval import org.opensearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder import org.opensearch.search.aggregations.bucket.terms.TermsAggregationBuilder -import org.opensearch.search.aggregations.metrics.* +import org.opensearch.search.aggregations.metrics.SumAggregationBuilder +import org.opensearch.search.aggregations.metrics.MaxAggregationBuilder +import org.opensearch.search.aggregations.metrics.AvgAggregationBuilder +import org.opensearch.search.aggregations.metrics.MinAggregationBuilder +import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.tasks.Task import org.opensearch.transport.TransportChannel @@ -42,8 +63,7 @@ import org.opensearch.transport.TransportInterceptor import org.opensearch.transport.TransportRequest import org.opensearch.transport.TransportRequestHandler import java.nio.ByteBuffer -import java.util.* - +import java.util.Base64 class RollupInterceptor( val clusterService: ClusterService, val settings: Settings, @@ -127,40 +147,12 @@ class RollupInterceptor( return false } fun copyAggregations(oldAggs: AggregatorFactories.Builder): AggregatorFactories.Builder { - -// val out = BytesStreamOutput() -// oldAggs.writeTo(out) -// -// val sin = StreamInput.wrap(out.bytes().toBytesRef().bytes) -// val loadedAggregators = AggregatorFactories(sin) -// // Create a new AggregatorFactories.Builder and add the loaded aggregators manually -// val newAggsBuilder = AggregatorFactories.builder() -// for (aggregator in loadedAggregators) { -// newAggsBuilder.addAggregator(aggregator) -// } -// -// return newAggsBuilder -// try { -// val xContentBuilder = XContentFactory.jsonBuilder() -// val bytesReference: BytesReference = BytesReference.bytes(oldAggs) -// val bytesReference = BytesReference.bytes(oldAggs.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS)) -// val string = -// oldAggs.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS) -// .string() -// val parser = XContentType.JSON.xContent().createParser( -// NamedXContentRegistry(SearchModule(Settings.EMPTY, emptyList()).namedXContents), LoggingDeprecationHandler.INSTANCE, string) -// return AggregatorFactories.parseAggregators(parser) -// } catch(e:Exception) { -// logger.error(e) -// } -// return oldAggs try { val aggsStr = BytesStreamOutput().use { out -> out.writeVersion(Version.CURRENT) oldAggs.writeTo(out) val bytes = BytesReference.toBytes(out.bytes()) Base64.getUrlEncoder().withoutPadding().encodeToString(bytes) - } val bytesReference = BytesReference.fromByteBuffer( ByteBuffer.wrap( @@ -184,14 +176,11 @@ class RollupInterceptor( // Wrap original aggregations into buckets based on fixed interval to remove overlap in response interceptor fun breakRequestIntoBuckets(request: ShardSearchRequest, rollupJob: Rollup) { val oldAggs = copyAggregations(request.source().aggregations()) - logger.error("ronsax oldAggs is now $oldAggs") var dateSourceField: String = "" -// var dateTargetField: String = "" var rollupInterval: String = "" for (dim in rollupJob.dimensions) { if (dim is DateHistogram) { dateSourceField = dim.sourceField -// dateTargetField = dim.targetField rollupInterval = dim.fixedInterval!! break } @@ -209,7 +198,6 @@ class RollupInterceptor( return } - @Suppress("SpreadOperator") override fun interceptHandler( action: String, diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index 50f6a0fac..a4faeb0f6 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -468,7 +468,7 @@ fun parseRollup(response: GetResponse, xContentRegistry: NamedXContentRegistry = // Changes aggregations in search source builder to new original aggregation (Change query too?) fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collection): SearchSourceBuilder { val ssb = SearchSourceBuilder() - aggregationBuilderCollection.forEach {ssb.aggregation(it)} + aggregationBuilderCollection.forEach { ssb.aggregation(it) } if (this.explain() != null) ssb.explain(this.explain()) if (this.ext() != null) ssb.ext(this.ext()) ssb.fetchSource(this.fetchSource()) @@ -498,5 +498,3 @@ fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collect if (this.collapse() != null) ssb.collapse(this.collapse()) return ssb } - - From 1f07f214210c553b9a94b70d60a28fa82fdf4a77 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 23 Aug 2023 18:46:51 -0700 Subject: [PATCH 09/44] Handles overlap between 1 live index and 1 rollup index for sum aggregation Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 106 ++++++++++++++---- .../rollup/interceptor/RollupInterceptor.kt | 29 ----- 2 files changed, 84 insertions(+), 51 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 6ad9257f9..bcefbd7ee 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -18,22 +18,28 @@ import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.util.getRollupJobs import org.opensearch.indexmanagement.rollup.util.isRollupIndex +import org.opensearch.search.DocValueFormat +import org.opensearch.search.aggregations.InternalAggregation +import org.opensearch.search.aggregations.InternalAggregations +import org.opensearch.search.aggregations.InternalOrder.Aggregation +import org.opensearch.search.aggregations.bucket.histogram.InternalDateHistogram +import org.opensearch.search.aggregations.metrics.InternalMax +import org.opensearch.search.aggregations.metrics.InternalMin +import org.opensearch.search.aggregations.metrics.InternalNumericMetricsAggregation +import org.opensearch.search.aggregations.metrics.InternalSum import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult import org.opensearch.search.sort.SortBuilders import org.opensearch.search.sort.SortOrder -import org.opensearch.transport.TransportInterceptor -import org.opensearch.transport.TransportResponse -import org.opensearch.transport.TransportRequest -import org.opensearch.transport.Transport -import org.opensearch.transport.TransportRequestOptions -import org.opensearch.transport.TransportResponseHandler -import org.opensearch.transport.TransportException +import org.opensearch.transport.* import java.text.SimpleDateFormat +import java.time.ZonedDateTime import java.util.* import java.util.concurrent.CountDownLatch +import kotlin.math.max +import kotlin.math.min class ResponseInterceptor( val clusterService: ClusterService, @@ -254,9 +260,36 @@ class ResponseInterceptor( // No overlap so start and end include everything return Pair(0L, Long.MAX_VALUE) } + fun zonedDateTimeToMillis(zonedDateTime: ZonedDateTime): Long { + return zonedDateTime.toInstant().toEpochMilli() + } + // Depending on which metric the aggregation is return a different start value +// fun computeRunningValue(agg: InternalNumericMetricsAggregation.SingleValue, currentValue: Double): Double { +// when (agg) { +// is InternalSum -> { +// return agg.value + currentValue +// } +// is InternalMax -> { +// return max(agg.value, currentValue) +// } +// is InternalMin -> { +// return min(agg.value, currentValue) +// } +// else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") +// } +// } + // Depending on which metric the aggregation is return a different start value +// fun getAggComputationStartValue(agg: InternalNumericMetricsAggregation.SingleValue): Double { +// when (agg) { +// is InternalSum -> return agg.value +// is InternalMax -> return agg.value +// is InternalMin -> return agg.value +// else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") +// } +// } // Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) -// fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { + fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { // /* // PSUEDOCODE // 1. Look at first bucket to see which aggs where in initial request @@ -268,10 +301,46 @@ class ResponseInterceptor( // 7. iterate throguh all key, vals in map and construct an internalAggregation object for each of them, add to InternalAggregations object // 8. return InternalAggregations object // */ -// intervalAggregations.get<>() -// val result = listOf(intervalAggregations) -// return InternalAggregations(intervalAggregations, null) -// } +// // Create an empty map to hold the sum values TODO add other metrics later + val sumValues = mutableMapOf() +// +// // Iterate through each aggregation and bucket + val interceptorAgg = intervalAggregations.asMap().get("interceptor_interval_data") as InternalDateHistogram + for (bucket in interceptorAgg.buckets) { + val zdt = bucket.key as ZonedDateTime + val timestamp: Long = zonedDateTimeToMillis(zdt) + // Only consider buckets within the specified range + // Start is inclusive and end is exclusive + if (timestamp >= start && timestamp < end) { + for (originalAgg in bucket.aggregations) { + val aggName = originalAgg.name + if (sumValues.containsKey(aggName)) { + // Compute running sum + if (originalAgg is InternalSum) { + sumValues[aggName] = sumValues[aggName]!! + originalAgg.value + } +// sumValues[aggName] = computeRunningValue(originalAgg!!, currentValue!!) + } else { + if (originalAgg is InternalSum) { + sumValues[aggName] = originalAgg.value + } +// sumValues[aggName] = getAggComputationStartValue(originalAgg) + } + } + } + + } + + // Create a new InternalAggregations with sum values + val allAggregations = mutableListOf() + for ((aggName, sumValue) in sumValues) { + val sumAgg = InternalSum(aggName, sumValue, DocValueFormat.RAW, null) + allAggregations.add(sumAgg) + } + + + return InternalAggregations(allAggregations, null) + } @Suppress("UNCHECKED_CAST") override fun handleResponse(response: T?) { // Handle the response if it came from intercpetor @@ -281,13 +350,8 @@ class ResponseInterceptor( if (response.hasAggs() && isRewrittenInterceptorRequest(response)) { // Check for overlap val (startTime, endTime) = findOverlap(response) - logger.error("ronsax: live index: start $startTime and end $endTime") // Modify agg to be original result without overlap computed in - // TODO handle overlap here - // TODO create a copy of the QuerySearchResult with aggregations modified -// val newQuerySerach = QuerySearchResult() -// val responseForHandler = newQuerySerach as T -// response.shardIndex = response.shardSearchRequest?.shardId()?.id ?: -1 + response.aggregations(computeAggregationsWithoutOverlap(response.aggregations().expand(), startTime, endTime)) originalHandler?.handleResponse(response) } else { originalHandler?.handleResponse(response) @@ -299,10 +363,8 @@ class ResponseInterceptor( if (queryResult.hasAggs() && isRewrittenInterceptorRequest(queryResult)) { // Check for overlap val (startTime, endTime) = findOverlap(queryResult) - logger.error("ronsax: rollup index: start $startTime and end $endTime") -// response.shardIndex = response.shardSearchRequest?.shardId()?.id ?: -1 - - // TODO handle overlap here + // Modify agg to be original result without overlap computed in + queryResult.aggregations(computeAggregationsWithoutOverlap(queryResult.aggregations().expand(), startTime, endTime)) // TODO change response object // val r1 = QueryFetchSearchResult(response.queryResult(), response.fetchResult()) // r1.shardIndex = response.shardIndex diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 9f1f0bd51..e456af404 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -97,32 +97,6 @@ class RollupInterceptor( } return Pair(false, null) } - // Need to determine if this was an internal client call to avoid infinite loop from interceptor, -> query string doesn't include "query" -// fun isRequestFromResponseInterceptor(request: ShardSearchRequest): Boolean { -// if (request.source().query() != null) { -// val jsonRequest: String = request.source().query().toString() -// // Detected dummy field from internal search request -// if (jsonRequest.contains(",\n" + -// " {\n" + -// " \"match_all\" : {\n" + -// " \"boost\" : 1.0\n" + -// " }\n" + -// " },\n" + -// " {\n" + -// " \"match_all\" : {\n" + -// " \"boost\" : 1.0\n" + -// " }\n" + -// " },\n" + -// " {\n" + -// " \"match_all\" : {\n" + -// " \"boost\" : 1.0\n" + -// " }\n" + -// " }")) { -// return true -// } -// } -// return false -// } // Returns true if request was already modified into "interceptor_interval_data" bucket aggregation fun isRequestRewrittenIntoBuckets(request: ShardSearchRequest): Boolean { val currentAggs = request.source().aggregations().aggregatorFactories @@ -214,10 +188,7 @@ class RollupInterceptor( // Only modifies rollup searches and avoids internal client calls if (containsRollup || isRollupIndex) { val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) - // Check before rewriting rollup because it deletes dummy field -// val requestCalledInInterceptor = isRequestFromResponseInterceptor(request) // Avoid infinite interceptor loop - // Need to break into buckets for agg search on live and rollup indices if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts breakRequestIntoBuckets(request, rollupJob!!) From b4dbc2647add68d8674a58046d24b7e0b54c3641 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 23 Aug 2023 23:44:46 -0700 Subject: [PATCH 10/44] added min max aggregations and fixed intersection time calculation Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 165 +++++++++++------- .../rollup/interceptor/RollupInterceptor.kt | 1 + 2 files changed, 99 insertions(+), 67 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index bcefbd7ee..bbfdaccda 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -21,22 +21,28 @@ import org.opensearch.indexmanagement.rollup.util.isRollupIndex import org.opensearch.search.DocValueFormat import org.opensearch.search.aggregations.InternalAggregation import org.opensearch.search.aggregations.InternalAggregations -import org.opensearch.search.aggregations.InternalOrder.Aggregation import org.opensearch.search.aggregations.bucket.histogram.InternalDateHistogram import org.opensearch.search.aggregations.metrics.InternalMax import org.opensearch.search.aggregations.metrics.InternalMin -import org.opensearch.search.aggregations.metrics.InternalNumericMetricsAggregation import org.opensearch.search.aggregations.metrics.InternalSum import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult +import org.opensearch.index.query.QueryBuilders import org.opensearch.search.sort.SortBuilders import org.opensearch.search.sort.SortOrder -import org.opensearch.transport.* -import java.text.SimpleDateFormat +import org.opensearch.transport.Transport +import org.opensearch.transport.TransportException +import org.opensearch.transport.TransportInterceptor +import org.opensearch.transport.TransportRequest +import org.opensearch.transport.TransportRequestOptions +import org.opensearch.transport.TransportResponse +import org.opensearch.transport.TransportResponseHandler import java.time.ZonedDateTime -import java.util.* +import java.time.LocalDateTime +import java.time.ZoneOffset +import java.time.format.DateTimeFormatter import java.util.concurrent.CountDownLatch import kotlin.math.max import kotlin.math.min @@ -113,24 +119,12 @@ class ResponseInterceptor( } return Pair(rollupIndices.toTypedArray(), liveIndices.toTypedArray()) } - fun convertEpochMillisToDateString(epochMillis: Long): String { - val pattern = "yyyy-MM-dd HH:mm:ss" - val dateFormat = SimpleDateFormat(pattern) - val date = Date(epochMillis) - val dateString = dateFormat.format(date) - return dateString - } fun convertDateStringToEpochMillis(dateString: String): Long { val pattern = "yyyy-MM-dd HH:mm:ss" - val dateFormat = SimpleDateFormat(pattern) - - try { - val date = dateFormat.parse(dateString) - return (date.time) - } catch (e: Exception) { - println("Error parsing date: ${e.message}") - } - return 0L + val formatter = DateTimeFormatter.ofPattern(pattern) + val localDateTime = LocalDateTime.parse(dateString, formatter) + val epochMillis = localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli() + return epochMillis } fun convertFixedIntervalStringToMs(fixedInterval: String): Long { // Possible types are ms, s, m, h, d @@ -153,6 +147,43 @@ class ResponseInterceptor( return milliseconds } + fun getIntersectionTime(liveDataStartPoint: Long, rollupIndex: String, dateTargetField: String): Long { + // Build search request to find the minimum rollup timestamp >= liveDataStartPoint + val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.ASC) + val query = QueryBuilders.boolQuery() + .must(QueryBuilders.rangeQuery(dateTargetField).gte(liveDataStartPoint)) + val searchSourceBuilder = SearchSourceBuilder() + .sort(sort) + .query(query) + .size(1) + // Need to avoid infinite interceptor loop + val req = SearchRequest() + .indices(rollupIndex) + .source(searchSourceBuilder) + var res: SearchResponse? = null + val latch = CountDownLatch(1) + client.search( + req, + object : ActionListener { + override fun onResponse(searchResponse: SearchResponse) { + res = searchResponse + latch.countDown() + } + + override fun onFailure(e: Exception) { + logger.error("request to find intersection time failed :(", e) + latch.countDown() + } + } + ) + latch.await() + try { + return res!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long + } catch (e: Exception) { + logger.error("Not able to retrieve intersection time from response: $e") + } + return 0L // dummy :P + } // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are longs becuase the type is epoch milliseconds @@ -172,11 +203,6 @@ class ResponseInterceptor( // Keep existing query and add 3 fake match alls to avoid infinite loop val request: ShardSearchRequest = response.shardSearchRequest!! val oldQuery = request.source().query() -// val fakeQuery = QueryBuilders.boolQuery() -// .must(oldQuery ?: QueryBuilders.matchAllQuery()) -// .must(QueryBuilders.matchAllQuery()) -// .must(QueryBuilders.matchAllQuery()) -// .must(QueryBuilders.matchAllQuery()) // TODO scale this for multiple indices!!!! val (rollupIndices, liveIndices) = getRollupAndLiveIndices(request) val rollupIndex = rollupIndices[0] @@ -238,13 +264,13 @@ class ResponseInterceptor( if (minLiveDateResponse != null && maxRolledDateResponse != null) { // Rollup data ends at maxRolledDate + fixedInterval - val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long // broken for rollup index + val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) - val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String // broken for rollup index + val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) if (liveDataStartPoint <= rollupDataEndPoint) { // Find intersection timestamp - val intersectionTime = maxRolledDate + val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndex, dateTargetField) val shardRequestIndex = request.shardId().indexName if (shardRequestIndex == liveIndex) { // Start at intersection timestamp, end at inf @@ -263,30 +289,38 @@ class ResponseInterceptor( fun zonedDateTimeToMillis(zonedDateTime: ZonedDateTime): Long { return zonedDateTime.toInstant().toEpochMilli() } + // Depending on which metric the aggregation is computer data differently + fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Double): Pair { + when (agg) { + is InternalSum -> { + return Pair(agg.value + currentValue, agg.type) + } + is InternalMax -> { + return Pair(max(agg.value,currentValue), agg.type) + } + is InternalMin -> { + return Pair(min(agg.value,currentValue), agg.type) + } + else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") + } + } // Depending on which metric the aggregation is return a different start value -// fun computeRunningValue(agg: InternalNumericMetricsAggregation.SingleValue, currentValue: Double): Double { -// when (agg) { -// is InternalSum -> { -// return agg.value + currentValue -// } -// is InternalMax -> { -// return max(agg.value, currentValue) -// } -// is InternalMin -> { -// return min(agg.value, currentValue) -// } -// else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") -// } -// } - // Depending on which metric the aggregation is return a different start value -// fun getAggComputationStartValue(agg: InternalNumericMetricsAggregation.SingleValue): Double { -// when (agg) { -// is InternalSum -> return agg.value -// is InternalMax -> return agg.value -// is InternalMin -> return agg.value -// else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") -// } -// } + fun getAggComputationStartValue(agg: org.opensearch.search.aggregations.Aggregation): Pair { + when (agg) { + is InternalSum -> return Pair(agg.value, agg.type) + is InternalMax -> return Pair(agg.value, agg.type) + is InternalMin -> return Pair(agg.value, agg.type) + else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") + } + } + fun createNewMetricAgg(aggName: String, aggValue: Double, aggType: String): InternalAggregation { + when (aggType) { + "sum" -> return InternalSum(aggName, aggValue, DocValueFormat.RAW, null) + "min" -> return InternalMin(aggName, aggValue, DocValueFormat.RAW, null) + "max" -> return InternalMax(aggName, aggValue, DocValueFormat.RAW, null) + else -> throw IllegalArgumentException("Could recreate an aggregation for type $aggType") + } + } // Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { @@ -301,8 +335,9 @@ class ResponseInterceptor( // 7. iterate throguh all key, vals in map and construct an internalAggregation object for each of them, add to InternalAggregations object // 8. return InternalAggregations object // */ -// // Create an empty map to hold the sum values TODO add other metrics later - val sumValues = mutableMapOf() + // Create an empty map to hold the agg values + // {aggName: String: Pair} + val sumValues = mutableMapOf>() // // // Iterate through each aggregation and bucket val interceptorAgg = intervalAggregations.asMap().get("interceptor_interval_data") as InternalDateHistogram @@ -315,16 +350,11 @@ class ResponseInterceptor( for (originalAgg in bucket.aggregations) { val aggName = originalAgg.name if (sumValues.containsKey(aggName)) { - // Compute running sum - if (originalAgg is InternalSum) { - sumValues[aggName] = sumValues[aggName]!! + originalAgg.value - } -// sumValues[aggName] = computeRunningValue(originalAgg!!, currentValue!!) + // Compute running calculation + val (currentValue, _) = sumValues[aggName]!! + sumValues[aggName] = computeRunningValue(originalAgg!!, currentValue) } else { - if (originalAgg is InternalSum) { - sumValues[aggName] = originalAgg.value - } -// sumValues[aggName] = getAggComputationStartValue(originalAgg) + sumValues[aggName] = getAggComputationStartValue(originalAgg) } } } @@ -333,9 +363,10 @@ class ResponseInterceptor( // Create a new InternalAggregations with sum values val allAggregations = mutableListOf() - for ((aggName, sumValue) in sumValues) { - val sumAgg = InternalSum(aggName, sumValue, DocValueFormat.RAW, null) - allAggregations.add(sumAgg) + for ((aggName, data) in sumValues) { + val (value, type) = data + val newAgg = createNewMetricAgg(aggName, value, type) + allAggregations.add(newAgg) } @@ -343,7 +374,7 @@ class ResponseInterceptor( } @Suppress("UNCHECKED_CAST") override fun handleResponse(response: T?) { - // Handle the response if it came from intercpetor + // Handle the response if it came from interceptor when (response) { // live index is QuerySearchResult -> { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index e456af404..f1927773b 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -195,6 +195,7 @@ class RollupInterceptor( } // Rewrite the request to fit rollup format if not already done previously if (isRollupIndex && !isReqeustRollupFormat(request)) { + // TODO fix logic to allow response interceptor client calls to have a size of 1 // if (!requestCalledInInterceptor && request.source().size() != 0) { // throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") // } From 1ee93c646d3631fd88888cc106fa6d38a7be86fe Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 24 Aug 2023 10:27:54 -0700 Subject: [PATCH 11/44] changed variable name in computeAggregationsWithoutOverlap Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index bbfdaccda..1c6b66ef9 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -323,6 +323,11 @@ class ResponseInterceptor( } // Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) + /** + * kughbniohujbnuhygjbjuhygj + * @param intervalAggregations + * @return Int + */ fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { // /* // PSUEDOCODE @@ -337,7 +342,7 @@ class ResponseInterceptor( // */ // Create an empty map to hold the agg values // {aggName: String: Pair} - val sumValues = mutableMapOf>() + val aggValues = mutableMapOf>() // // // Iterate through each aggregation and bucket val interceptorAgg = intervalAggregations.asMap().get("interceptor_interval_data") as InternalDateHistogram @@ -349,12 +354,12 @@ class ResponseInterceptor( if (timestamp >= start && timestamp < end) { for (originalAgg in bucket.aggregations) { val aggName = originalAgg.name - if (sumValues.containsKey(aggName)) { + if (aggValues.containsKey(aggName)) { // Compute running calculation - val (currentValue, _) = sumValues[aggName]!! - sumValues[aggName] = computeRunningValue(originalAgg!!, currentValue) + val (currentValue, _) = aggValues[aggName]!! + aggValues[aggName] = computeRunningValue(originalAgg!!, currentValue) } else { - sumValues[aggName] = getAggComputationStartValue(originalAgg) + aggValues[aggName] = getAggComputationStartValue(originalAgg) } } } @@ -363,7 +368,7 @@ class ResponseInterceptor( // Create a new InternalAggregations with sum values val allAggregations = mutableListOf() - for ((aggName, data) in sumValues) { + for ((aggName, data) in aggValues) { val (value, type) = data val newAgg = createNewMetricAgg(aggName, value, type) allAggregations.add(newAgg) From 4203a607dd421ee030ed04b5e56bf04c31b1dce2 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Mon, 28 Aug 2023 10:07:54 -0700 Subject: [PATCH 12/44] Added integ tests for nonoverlapping case Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 11 +- .../interceptor/ResponseInterceptorIT.kt | 220 ++++++++++++++++++ .../rollup/interceptor/RollupInterceptorIT.kt | 78 ------- 3 files changed, 224 insertions(+), 85 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 1c6b66ef9..a93ae32ae 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -262,13 +262,13 @@ class ResponseInterceptor( ) latch.await() - if (minLiveDateResponse != null && maxRolledDateResponse != null) { + if (minLiveDateResponse != null && maxRolledDateResponse != null && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { // Rollup data ends at maxRolledDate + fixedInterval val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) - if (liveDataStartPoint <= rollupDataEndPoint) { + if (liveDataStartPoint < rollupDataEndPoint) { // Find intersection timestamp val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndex, dateTargetField) val shardRequestIndex = request.shardId().indexName @@ -296,10 +296,10 @@ class ResponseInterceptor( return Pair(agg.value + currentValue, agg.type) } is InternalMax -> { - return Pair(max(agg.value,currentValue), agg.type) + return Pair(max(agg.value, currentValue), agg.type) } is InternalMin -> { - return Pair(min(agg.value,currentValue), agg.type) + return Pair(min(agg.value, currentValue), agg.type) } else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") } @@ -363,7 +363,6 @@ class ResponseInterceptor( } } } - } // Create a new InternalAggregations with sum values @@ -373,8 +372,6 @@ class ResponseInterceptor( val newAgg = createNewMetricAgg(aggName, value, type) allAggregations.add(newAgg) } - - return InternalAggregations(allAggregations, null) } @Suppress("UNCHECKED_CAST") diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt new file mode 100644 index 000000000..ae547ed88 --- /dev/null +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -0,0 +1,220 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.indexmanagement.rollup.interceptor + +import org.apache.http.entity.ContentType +import org.apache.http.entity.StringEntity +import org.opensearch.indexmanagement.common.model.dimension.DateHistogram +import org.opensearch.indexmanagement.common.model.dimension.Terms +import org.opensearch.indexmanagement.makeRequest +import org.opensearch.indexmanagement.rollup.RollupRestTestCase +import org.opensearch.indexmanagement.rollup.model.Rollup +import org.opensearch.indexmanagement.rollup.model.RollupMetadata +import org.opensearch.indexmanagement.rollup.model.RollupMetrics +import org.opensearch.indexmanagement.rollup.model.metric.Average +import org.opensearch.indexmanagement.rollup.model.metric.Max +import org.opensearch.indexmanagement.rollup.model.metric.Min +import org.opensearch.indexmanagement.rollup.model.metric.Sum +import org.opensearch.indexmanagement.rollup.model.metric.ValueCount +import org.opensearch.indexmanagement.waitFor +import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule +import org.opensearch.rest.RestStatus +import java.time.Instant +import java.time.temporal.ChronoUnit + +@Suppress("UNCHECKED_CAST") +class ResponseInterceptorIT : RollupRestTestCase() { + fun `test search a live index and rollup index with no overlap`() { + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + + // Delete half the values from rollup data + r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "gte": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteRollupResponse = client().makeRequest( + "POST", + "target_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) + // Search both and check if time series data is the same + // Term query + var req = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(searchResponse.restStatus() == RestStatus.OK) + var responseAggs = searchResponse.asMap()["aggregations"] as Map> + assertEquals( + "Aggregation from searching both indices is wrong", + 9024.0, + responseAggs.getValue("sum_passenger_count")["value"] + ) + } + fun `test search a live index with no data and rollup index with data`() { + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + + // Delete values from live index + var deleteResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + ) + assertTrue(deleteResponse.restStatus() == RestStatus.OK) + // Term query + var req = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(searchResponse.restStatus() == RestStatus.OK) + var responseAggs = searchResponse.asMap()["aggregations"] as Map> + assertEquals( + "Aggregation from searching both indices is wrong", + 9024.0, + responseAggs.getValue("sum_passenger_count")["value"] + ) + } +} diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt index 171e0cf1b..e42295418 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt @@ -1876,82 +1876,4 @@ class RollupInterceptorIT : RollupRestTestCase() { Assert.assertTrue(e.message!!.contains("Can't parse query_string query without sourceIndex mappings!")) } } - fun `test search a live index and rollup index with no overlap`() { - generateNYCTaxiData("source_rollup_search") - val rollup = Rollup( - id = "basic_term_query_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search", - targetIndex = "target_rollup_search", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("RatecodeID", "RatecodeID"), - Terms("PULocationID", "PULocationID") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ), - RollupMetrics(sourceField = "total_amount", targetField = "total_amount", metrics = listOf(Max(), Min())) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - - refreshAllIndices() - - // Delete values from live index - var deleteResponse = client().makeRequest( - "POST", - "source_rollup_search/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) - ) - assertTrue(deleteResponse.restStatus() == RestStatus.OK) - // Term query - var req = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResponse.restStatus() == RestStatus.OK) - var responseAggs = searchResponse.asMap()["aggregations"] as Map> - assertEquals( - "Aggregation from searching both indices is wrong", - 9024.0, - responseAggs.getValue("sum_passenger_count")["value"] - ) - } } From 72e0aadc8734048e7bfd780be0a817d3906380ce Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 29 Aug 2023 01:39:27 -0700 Subject: [PATCH 13/44] added avg and value count aggregation Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 178 +++++++++++----- .../rollup/interceptor/RollupInterceptor.kt | 40 +++- .../rollup/util/RollupUtils.kt | 4 + .../interceptor/ResponseInterceptorIT.kt | 200 +++++++++++++++++- 4 files changed, 368 insertions(+), 54 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index a93ae32ae..800b0b282 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -22,14 +22,14 @@ import org.opensearch.search.DocValueFormat import org.opensearch.search.aggregations.InternalAggregation import org.opensearch.search.aggregations.InternalAggregations import org.opensearch.search.aggregations.bucket.histogram.InternalDateHistogram -import org.opensearch.search.aggregations.metrics.InternalMax -import org.opensearch.search.aggregations.metrics.InternalMin -import org.opensearch.search.aggregations.metrics.InternalSum import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult import org.opensearch.index.query.QueryBuilders +import org.opensearch.script.Script +import org.opensearch.script.ScriptType +import org.opensearch.search.aggregations.metrics.* import org.opensearch.search.sort.SortBuilders import org.opensearch.search.sort.SortOrder import org.opensearch.transport.Transport @@ -147,7 +147,7 @@ class ResponseInterceptor( return milliseconds } - fun getIntersectionTime(liveDataStartPoint: Long, rollupIndex: String, dateTargetField: String): Long { + fun getIntersectionTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { // Build search request to find the minimum rollup timestamp >= liveDataStartPoint val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.ASC) val query = QueryBuilders.boolQuery() @@ -158,8 +158,8 @@ class ResponseInterceptor( .size(1) // Need to avoid infinite interceptor loop val req = SearchRequest() - .indices(rollupIndex) .source(searchSourceBuilder) + rollupIndices.forEach { req.indices(it) } var res: SearchResponse? = null val latch = CountDownLatch(1) client.search( @@ -205,63 +205,70 @@ class ResponseInterceptor( val oldQuery = request.source().query() // TODO scale this for multiple indices!!!! val (rollupIndices, liveIndices) = getRollupAndLiveIndices(request) - val rollupIndex = rollupIndices[0] - val liveIndex = liveIndices[0] - // Build search request to find the minimum date in the live data index - var sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) + val shardRequestIndex = request.shardId().indexName + val isShardIndexRollup = isRollupIndex(shardRequestIndex, clusterService.state()) + // Build search request to find the maximum date on the rolled data index + var sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) var searchSourceBuilder = SearchSourceBuilder() .sort(sort) + .query(oldQuery) .size(1) - val minLiveDateRequest = SearchRequest() - .indices(liveIndex) + // Need to avoid infinite interceptor loop + val maxRolledDateRequest = SearchRequest() .source(searchSourceBuilder) - - var minLiveDateResponse: SearchResponse? = null + rollupIndices.forEach { maxRolledDateRequest.indices(it) } // add all rollup indices to this request + var maxRolledDateResponse: SearchResponse? = null var latch = CountDownLatch(1) client.search( - minLiveDateRequest, + maxRolledDateRequest, object : ActionListener { override fun onResponse(searchResponse: SearchResponse) { - minLiveDateResponse = searchResponse + maxRolledDateResponse = searchResponse latch.countDown() } override fun onFailure(e: Exception) { - logger.error("ronsax minLiveDate request failed ", e) + logger.error("ronsax maxLiveDate request failed ", e) latch.countDown() } } ) latch.await() - - // Build search request to find the maximum date on the rolled data index - sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) + // Build search request to find the minimum date in the live data index + sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) searchSourceBuilder = SearchSourceBuilder() .sort(sort) - .query(oldQuery) .size(1) - // Need to avoid infinite interceptor loop - val maxRolledDateRequest = SearchRequest() - .indices(rollupIndex) + val minLiveDateRequest = SearchRequest() .source(searchSourceBuilder) - var maxRolledDateResponse: SearchResponse? = null + /* + If the shard index is a rollup index I want to find the minimum value of all the live indices to compute the overlap + If the shard index is a live index I only care about the minimum value of the current shard index + */ + if (isShardIndexRollup) { + liveIndices.forEach { minLiveDateRequest.indices(it) } + } else { // shard index is live index + minLiveDateRequest.indices(shardRequestIndex) + } + + var minLiveDateResponse: SearchResponse? = null latch = CountDownLatch(1) client.search( - maxRolledDateRequest, + minLiveDateRequest, object : ActionListener { override fun onResponse(searchResponse: SearchResponse) { - maxRolledDateResponse = searchResponse + minLiveDateResponse = searchResponse latch.countDown() } override fun onFailure(e: Exception) { - logger.error("ronsax maxLiveDate request failed ", e) + logger.error("ronsax minLiveDate request failed ", e) latch.countDown() } } ) latch.await() - + // if they overlap find part to exclude if (minLiveDateResponse != null && maxRolledDateResponse != null && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { // Rollup data ends at maxRolledDate + fixedInterval val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long @@ -270,14 +277,13 @@ class ResponseInterceptor( val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) if (liveDataStartPoint < rollupDataEndPoint) { // Find intersection timestamp - val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndex, dateTargetField) - val shardRequestIndex = request.shardId().indexName - if (shardRequestIndex == liveIndex) { - // Start at intersection timestamp, end at inf - return Pair(intersectionTime, Long.MAX_VALUE) - } else if (shardRequestIndex == rollupIndex) { + val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndices, dateTargetField) + if (isShardIndexRollup) { // Start at 0, end at intersection time return Pair(0L, intersectionTime) + } else { // Shard index is live + // Start at intersection timestamp, end at inf + return Pair(intersectionTime, Long.MAX_VALUE) } } } else { @@ -290,41 +296,93 @@ class ResponseInterceptor( return zonedDateTime.toInstant().toEpochMilli() } // Depending on which metric the aggregation is computer data differently - fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Double): Pair { + fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any): Pair { when (agg) { is InternalSum -> { - return Pair(agg.value + currentValue, agg.type) + return Pair(agg.value + (currentValue as Double), agg.type) } is InternalMax -> { - return Pair(max(agg.value, currentValue), agg.type) + return Pair(max(agg.value, (currentValue as Double)), agg.type) } is InternalMin -> { - return Pair(min(agg.value, currentValue), agg.type) + return Pair(min(agg.value, (currentValue as Double)), agg.type) + } + is InternalValueCount -> { // Live data uses this + return Pair(agg.value + (currentValue as Long), agg.type) + } + is InternalScriptedMetric -> { + // Rollup InternalValueCount + return Pair((agg.aggregation() as Long) + (currentValue as Long), "value_count") } - else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") + else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches: ${agg.name}") } } // Depending on which metric the aggregation is return a different start value - fun getAggComputationStartValue(agg: org.opensearch.search.aggregations.Aggregation): Pair { + fun getAggComputationStartValue(agg: org.opensearch.search.aggregations.Aggregation): Pair { when (agg) { is InternalSum -> return Pair(agg.value, agg.type) is InternalMax -> return Pair(agg.value, agg.type) is InternalMin -> return Pair(agg.value, agg.type) - else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches") + is InternalValueCount -> return Pair(agg.value, agg.type) // Live data + is InternalScriptedMetric -> return Pair(agg.aggregation(), "value_count") // Rollup data + else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches: ${agg.name}") } } - fun createNewMetricAgg(aggName: String, aggValue: Double, aggType: String): InternalAggregation { + fun createNewMetricAgg(aggName: String, aggValue: Any, aggType: String): InternalAggregation { when (aggType) { - "sum" -> return InternalSum(aggName, aggValue, DocValueFormat.RAW, null) - "min" -> return InternalMin(aggName, aggValue, DocValueFormat.RAW, null) - "max" -> return InternalMax(aggName, aggValue, DocValueFormat.RAW, null) - else -> throw IllegalArgumentException("Could recreate an aggregation for type $aggType") + "sum" -> return InternalSum(aggName, (aggValue as Double), DocValueFormat.RAW, null) + "min" -> return InternalMin(aggName, (aggValue as Double), DocValueFormat.RAW, null) + "max" -> return InternalMax(aggName, (aggValue as Double), DocValueFormat.RAW, null) + "value_count" -> return InternalValueCount(aggName, (aggValue as Long), null) +// "scripted_metric" -> { +// val script = Script( +// ScriptType.INLINE, +// "painless", +// "long valueCount = 0; for (vc in states) { valueCount += vc } return valueCount;", +// emptyMap(), // options +// emptyMap() // params +// ) +// return ScriptedMetricAggregationBuilder(aggName). +// } +// "avg" -> return InternalAvg(agg) // TODO look at how to make this bad boy + else -> throw IllegalArgumentException("Could not recreate an aggregation for type $aggType") } } + // Create original avg aggregation + fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, addedAggregations : MutableSet): InternalAvg { + // Sum calc + if (modifiedName.contains(".rollup.avg.sum")) { + // Won't double count + addedAggregations += modifiedName + val originalName = modifiedName.removeSuffix(".rollup.avg.sum") + val avgSum: Double = value as Double + for ((aggName, data) in aggValues) { + // Found value count component to create InternalAvg object + if (!addedAggregations.contains(aggName) && aggName.contains(originalName)) { + addedAggregations += aggName + val (avgCount, _) = data + return InternalAvg(originalName, avgSum, (avgCount as Long), DocValueFormat.RAW, null) + } + } + } else { // Value count calc + // Won't double count + addedAggregations += modifiedName + val originalName = modifiedName.removeSuffix(".rollup.avg.value_count") + val avgCount = value as Long + for ((aggName, data) in aggValues) { + // Found sum component to create InternalAvg object + if (!addedAggregations.contains(aggName) && aggName.contains(originalName)) { + addedAggregations += aggName + val (avgSum, _) = data + return InternalAvg(originalName, (avgSum as Double), avgCount, DocValueFormat.RAW, null) + } + } + } + throw Exception("Can't calculate avg agg for rollup index") + } // Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) /** - * kughbniohujbnuhygjbjuhygj * @param intervalAggregations * @return Int */ @@ -341,8 +399,8 @@ class ResponseInterceptor( // 8. return InternalAggregations object // */ // Create an empty map to hold the agg values - // {aggName: String: Pair} - val aggValues = mutableMapOf>() + // {aggName: String: Pair} + val aggValues = mutableMapOf>() // // // Iterate through each aggregation and bucket val interceptorAgg = intervalAggregations.asMap().get("interceptor_interval_data") as InternalDateHistogram @@ -367,10 +425,26 @@ class ResponseInterceptor( // Create a new InternalAggregations with sum values val allAggregations = mutableListOf() + val addedAggregations = mutableSetOf() // avoid repeating the same aggregations for ((aggName, data) in aggValues) { - val (value, type) = data - val newAgg = createNewMetricAgg(aggName, value, type) - allAggregations.add(newAgg) + if (addedAggregations.contains(aggName)) continue + // special case to compute value_count for rollup indices + else if (aggName.contains(".rollup.value_count")) { + val (value, _) = data + val originalName = aggName.removeSuffix(".rollup.value_count") + allAggregations.add(InternalValueCount(originalName, value as Long, null)) + addedAggregations += aggName + } + // special case to compute avg agg using sum and value_count calculation + else if (aggName.contains(".rollup.avg.sum") || aggName.contains(".rollup.avg.value_count")) { + val (value, _) = data + allAggregations.add(initRollupAvgAgg(aggName, value, aggValues, addedAggregations)) + } else { // Sum, Min, or Max agg + val (value, type) = data + val newAgg = createNewMetricAgg(aggName, value, type) + allAggregations.add(newAgg) + addedAggregations += aggName + } } return InternalAggregations(allAggregations, null) } diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index f1927773b..5b6d44f32 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -146,10 +146,48 @@ class RollupInterceptor( return oldAggs } } + // Need to modify aggs for rollup docs with avg and value count aggs + fun modifyRollupAggs(aggFacts: MutableCollection): AggregatorFactories.Builder { + val build = AggregatorFactories.builder() + for (agg in aggFacts) { + when (agg) { + is SumAggregationBuilder -> { + build.addAggregator(agg) + } + is MaxAggregationBuilder -> { + build.addAggregator(agg) + } + + is MinAggregationBuilder -> { + build.addAggregator(agg) + } + + is ValueCountAggregationBuilder -> { + // I want to append .rollup.value_count to the name so its identified in response interceptor + val newValueCount = ValueCountAggregationBuilder("${agg.name}.rollup.value_count") + newValueCount.field(agg.field()) + build.addAggregator(newValueCount) + } + is AvgAggregationBuilder -> { + // Going to split this into a value_count and a sum agg to put together in response interceptor + // Need to do this since .count and .sum are private in InternalAvg + val avgValueCount = ValueCountAggregationBuilder("${agg.name}.rollup.avg.value_count") + avgValueCount.field(agg.field()) + build.addAggregator(avgValueCount) + val avgSumCount = SumAggregationBuilder("${agg.name}.rollup.avg.sum") + avgSumCount.field(agg.field()) + build.addAggregator(avgSumCount) + } + + else -> throw IllegalArgumentException("The ${agg.type} aggregation is not currently supported in rollups") + } + } + return build + } // Wrap original aggregations into buckets based on fixed interval to remove overlap in response interceptor fun breakRequestIntoBuckets(request: ShardSearchRequest, rollupJob: Rollup) { - val oldAggs = copyAggregations(request.source().aggregations()) + val oldAggs = modifyRollupAggs(request.source().aggregations().aggregatorFactories) var dateSourceField: String = "" var rollupInterval: String = "" for (dim in rollupJob.dimensions) { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index a4faeb0f6..01a950f8c 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -498,3 +498,7 @@ fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collect if (this.collapse() != null) ssb.collapse(this.collapse()) return ssb } + +// fun InternalAvg.getSumAndCount(): Pair { +// return Pair(this.getSum(), this. +// } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index ae547ed88..0c4072e29 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -116,7 +116,6 @@ class ResponseInterceptorIT : RollupRestTestCase() { assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) // Search both and check if time series data is the same - // Term query var req = """ { "size": 0, @@ -141,6 +140,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { responseAggs.getValue("sum_passenger_count")["value"] ) } + // Edge Case fun `test search a live index with no data and rollup index with data`() { generateNYCTaxiData("source_rollup_search") val rollup = Rollup( @@ -217,4 +217,202 @@ class ResponseInterceptorIT : RollupRestTestCase() { responseAggs.getValue("sum_passenger_count")["value"] ) } + fun `test search a live index and rollup index with data overlap`() { + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + // Rollup index is complete overlap of live data + // Search both and check if time series data is the same + var req = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(searchResponse.restStatus() == RestStatus.OK) + var responseAggs = searchResponse.asMap()["aggregations"] as Map> + assertEquals( + "Aggregation from searching both indices is wrong", + 9024.0, + responseAggs.getValue("sum_passenger_count")["value"] + ) + } + fun `test min, max, value_count, sum, and avg aggs on data`() { + // TODO add avg computation later + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + + // Delete values from live index + var deleteResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + ) + assertTrue(deleteResponse.restStatus() == RestStatus.OK) + // Term query + var req = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "total_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(searchResponse.restStatus() == RestStatus.OK) + var responseAggs = searchResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + 9024.0, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + 6.0, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + 9024.0, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count", + 9024.0, + responseAggs.getValue("total_passenger_count")["value"] + ) + } } From 9b977d2750b06b66a405220f0b62ce34f45bfa23 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 29 Aug 2023 01:47:11 -0700 Subject: [PATCH 14/44] fixed ktlint and integ test Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 11 ++++++---- .../interceptor/ResponseInterceptorIT.kt | 20 ++++++++++++++----- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 800b0b282..2bda8d310 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -27,9 +27,12 @@ import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult import org.opensearch.index.query.QueryBuilders -import org.opensearch.script.Script -import org.opensearch.script.ScriptType -import org.opensearch.search.aggregations.metrics.* +import org.opensearch.search.aggregations.metrics.InternalAvg +import org.opensearch.search.aggregations.metrics.InternalMax +import org.opensearch.search.aggregations.metrics.InternalMin +import org.opensearch.search.aggregations.metrics.InternalScriptedMetric +import org.opensearch.search.aggregations.metrics.InternalSum +import org.opensearch.search.aggregations.metrics.InternalValueCount import org.opensearch.search.sort.SortBuilders import org.opensearch.search.sort.SortOrder import org.opensearch.transport.Transport @@ -349,7 +352,7 @@ class ResponseInterceptor( } } // Create original avg aggregation - fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, addedAggregations : MutableSet): InternalAvg { + fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, addedAggregations: MutableSet): InternalAvg { // Sum calc if (modifiedName.contains(".rollup.avg.sum")) { // Won't double count diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 0c4072e29..376d7a51f 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -383,7 +383,12 @@ class ResponseInterceptorIT : RollupRestTestCase() { "field": "passenger_count" } }, - "total_passenger_count": { + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { "value_count": { "field": "passenger_count" } @@ -406,13 +411,18 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) assertEquals( "min agg is wrong", - 9024.0, + 0.0, responseAggs.getValue("min_passenger_count")["value"] ) assertEquals( - "value_count", - 9024.0, - responseAggs.getValue("total_passenger_count")["value"] + "value_count is wrong", + 5000, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + 1.8048, + responseAggs.getValue("avg_passenger_count")["value"] ) } } From 8ed31e7ba01ea16039fb74d6c7e4d3f25dac6239 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 29 Aug 2023 10:09:40 -0700 Subject: [PATCH 15/44] changed test and build workflow Signed-off-by: Ronnak Saxena --- .github/workflows/test-and-build-workflow.yml | 2 +- .../indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/test-and-build-workflow.yml b/.github/workflows/test-and-build-workflow.yml index d324148e8..639ea274f 100644 --- a/.github/workflows/test-and-build-workflow.yml +++ b/.github/workflows/test-and-build-workflow.yml @@ -2,7 +2,7 @@ name: Test and Build Workflow on: pull_request: branches: - - "*" + - "**" push: branches: - "*" diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 376d7a51f..20e7835f9 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -360,7 +360,6 @@ class ResponseInterceptorIT : RollupRestTestCase() { StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) ) assertTrue(deleteResponse.restStatus() == RestStatus.OK) - // Term query var req = """ { "size": 0, From f0d364f3e9e20d21dd7eb5f921afbb38546db219 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 29 Aug 2023 17:37:21 -0700 Subject: [PATCH 16/44] added integ test for multiple live indices Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 62 +++- .../interceptor/ResponseInterceptorIT.kt | 323 ++++++++++++++---- 2 files changed, 294 insertions(+), 91 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 2bda8d310..d0c27b9c2 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -187,6 +187,44 @@ class ResponseInterceptor( } return 0L // dummy :P } + // Calculated the end time for the current shard index if it is a rollup index with data overlapp + fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { + // Build search request to find the maximum rollup timestamp <= liveDataStartPoint + val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) + val query = QueryBuilders.boolQuery() + .must(QueryBuilders.rangeQuery(dateTargetField).lte(liveDataStartPoint)) + val searchSourceBuilder = SearchSourceBuilder() + .sort(sort) + .query(query) + .size(1) + // Need to avoid infinite interceptor loop + val req = SearchRequest() + .source(searchSourceBuilder) + rollupIndices.forEach { req.indices(it) } + var res: SearchResponse? = null + val latch = CountDownLatch(1) + client.search( + req, + object : ActionListener { + override fun onResponse(searchResponse: SearchResponse) { + res = searchResponse + latch.countDown() + } + + override fun onFailure(e: Exception) { + logger.error("request to find intersection time failed :(", e) + latch.countDown() + } + } + ) + latch.await() + try { + return res!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long + } catch (e: Exception) { + logger.error("Not able to retrieve intersection time from response: $e") + } + return 0L // dummy :P + } // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are longs becuase the type is epoch milliseconds @@ -280,13 +318,14 @@ class ResponseInterceptor( val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) if (liveDataStartPoint < rollupDataEndPoint) { // Find intersection timestamp - val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndices, dateTargetField) +// val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndices, dateTargetField) if (isShardIndexRollup) { - // Start at 0, end at intersection time - return Pair(0L, intersectionTime) - } else { // Shard index is live - // Start at intersection timestamp, end at inf - return Pair(intersectionTime, Long.MAX_VALUE) + // Start at 0, end at live data + val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) + return Pair(0L, endTime) + } else { + // Include all live data + return Pair(0L, Long.MAX_VALUE) } } } else { @@ -337,17 +376,6 @@ class ResponseInterceptor( "min" -> return InternalMin(aggName, (aggValue as Double), DocValueFormat.RAW, null) "max" -> return InternalMax(aggName, (aggValue as Double), DocValueFormat.RAW, null) "value_count" -> return InternalValueCount(aggName, (aggValue as Long), null) -// "scripted_metric" -> { -// val script = Script( -// ScriptType.INLINE, -// "painless", -// "long valueCount = 0; for (vc in states) { valueCount += vc } return valueCount;", -// emptyMap(), // options -// emptyMap() // params -// ) -// return ScriptedMetricAggregationBuilder(aggName). -// } -// "avg" -> return InternalAvg(agg) // TODO look at how to make this bad boy else -> throw IllegalArgumentException("Could not recreate an aggregation for type $aggType") } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 20e7835f9..e040525fa 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -67,7 +67,50 @@ class ResponseInterceptorIT : RollupRestTestCase() { val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) } - + // Get expected aggregation values by searching live data before deletion + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] refreshAllIndices() // Split data at 1546304400000 or Jan 01 2019 01:00:00 // Delete half the values from live data simulating an ism job deleting old data @@ -115,30 +158,34 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) - // Search both and check if time series data is the same - var req = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResponse.restStatus() == RestStatus.OK) - var responseAggs = searchResponse.asMap()["aggregations"] as Map> + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> assertEquals( - "Aggregation from searching both indices is wrong", - 9024.0, + "sum agg is wrong", + expectedSum, responseAggs.getValue("sum_passenger_count")["value"] ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) } // Edge Case fun `test search a live index with no data and rollup index with data`() { @@ -183,17 +230,8 @@ class ResponseInterceptorIT : RollupRestTestCase() { } refreshAllIndices() - - // Delete values from live index - var deleteResponse = client().makeRequest( - "POST", - "source_rollup_search/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) - ) - assertTrue(deleteResponse.restStatus() == RestStatus.OK) - // Term query - var req = """ + // Get expected aggregation values by searching live data before deletion + var aggReq = """ { "size": 0, "query": { @@ -204,18 +242,75 @@ class ResponseInterceptorIT : RollupRestTestCase() { "sum": { "field": "passenger_count" } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } } } } """.trimIndent() - var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResponse.restStatus() == RestStatus.OK) - var responseAggs = searchResponse.asMap()["aggregations"] as Map> + var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Delete values from live index + var deleteResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + ) + assertTrue(deleteResponse.restStatus() == RestStatus.OK) + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> assertEquals( - "Aggregation from searching both indices is wrong", - 9024.0, + "sum agg is wrong", + expectedSum, responseAggs.getValue("sum_passenger_count")["value"] ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) } fun `test search a live index and rollup index with data overlap`() { generateNYCTaxiData("source_rollup_search") @@ -257,6 +352,50 @@ class ResponseInterceptorIT : RollupRestTestCase() { val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) } + // Get expected aggregation values by searching live data before deletion + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) + var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] refreshAllIndices() // Split data at 1546304400000 or Jan 01 2019 01:00:00 @@ -284,32 +423,36 @@ class ResponseInterceptorIT : RollupRestTestCase() { assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) // Rollup index is complete overlap of live data // Search both and check if time series data is the same - var req = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResponse.restStatus() == RestStatus.OK) - var responseAggs = searchResponse.asMap()["aggregations"] as Map> + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> assertEquals( - "Aggregation from searching both indices is wrong", - 9024.0, + "sum agg is wrong", + expectedSum, responseAggs.getValue("sum_passenger_count")["value"] ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) } - fun `test min, max, value_count, sum, and avg aggs on data`() { - // TODO add avg computation later + fun `test search multiple live data indices and a rollup data index with overlap`() { generateNYCTaxiData("source_rollup_search") val rollup = Rollup( id = "basic_term_query_rollup_search", @@ -351,16 +494,34 @@ class ResponseInterceptorIT : RollupRestTestCase() { } refreshAllIndices() - - // Delete values from live index - var deleteResponse = client().makeRequest( + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( "POST", "source_rollup_search/_delete_by_query", mapOf("refresh" to "true"), - StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + StringEntity(r, ContentType.APPLICATION_JSON) ) - assertTrue(deleteResponse.restStatus() == RestStatus.OK) - var req = """ + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + + // Insert more live data + generateNYCTaxiData("source_rollup_search2") + // Expected values would discard the overlapping rollup index completely + var aggReq = """ { "size": 0, "query": { @@ -395,33 +556,47 @@ class ResponseInterceptorIT : RollupRestTestCase() { } } """.trimIndent() - var searchResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResponse.restStatus() == RestStatus.OK) - var responseAggs = searchResponse.asMap()["aggregations"] as Map> + var searchResponse = client().makeRequest("POST", "/source_rollup_search,source_rollup_search2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + + // Search all 3 indices to check if overlap was removed + var searchAllResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search,source_rollup_search2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> assertEquals( "sum agg is wrong", - 9024.0, + expectedSum, responseAggs.getValue("sum_passenger_count")["value"] ) assertEquals( "max agg is wrong", - 6.0, + expectedMax, responseAggs.getValue("max_passenger_count")["value"] ) assertEquals( "min agg is wrong", - 0.0, + expectedMin, responseAggs.getValue("min_passenger_count")["value"] ) assertEquals( "value_count is wrong", - 5000, + expectedCount, responseAggs.getValue("count_passenger_count")["value"] ) assertEquals( "avg is wrong", - 1.8048, + expectedAvg, responseAggs.getValue("avg_passenger_count")["value"] ) } + fun `test search aliased live indices data and rollup data`() { + /* add later */ + } } From b52fbdcd8e51e6757c754331af861f129e89aafd Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 30 Aug 2023 13:17:48 -0700 Subject: [PATCH 17/44] added test case for alias live indices Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 150 ++++++++++++++++++ 1 file changed, 150 insertions(+) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index e040525fa..bc802adb9 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -598,5 +598,155 @@ class ResponseInterceptorIT : RollupRestTestCase() { } fun `test search aliased live indices data and rollup data`() { /* add later */ + // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 + generateNYCTaxiData("nyc-taxi-data-1") + generateNYCTaxiData("nyc-taxi-data-2") + generateNYCTaxiData("nyc-taxi-data-3") + // Add them to alias nyc-taxi-data + val createAliasReq = """ + { + "actions": [ + { + "add": { + "index": "nyc-taxi-data-1", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-2", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-3", + "alias": "nyc-taxi-data" + } + } + ] + } + """.trimIndent() + val createAliasRes = client().makeRequest( + "POST", + "_aliases", + mapOf(), + StringEntity(createAliasReq, ContentType.APPLICATION_JSON) + ) + assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) + // Rollup alias into rollup-nyc-taxi-data + val rollup = Rollup( + id = "basic_term_query_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "nyc-taxi-data", + targetIndex = "rollup-nyc-taxi-data", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + refreshAllIndices() + // Find expected values by searching nyc-taxi-data + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data + var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) } } From 820dc24e8bd1eff84e78d81e610952494657cf22 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 30 Aug 2023 14:14:07 -0700 Subject: [PATCH 18/44] cleaned up code and moved functions to utils file Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 133 ++---------------- .../rollup/interceptor/RollupInterceptor.kt | 28 +--- .../rollup/util/RollupUtils.kt | 40 +++++- 3 files changed, 52 insertions(+), 149 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index d0c27b9c2..b83cf403b 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -16,17 +16,19 @@ import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.Settings import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.rollup.model.Rollup -import org.opensearch.indexmanagement.rollup.util.getRollupJobs -import org.opensearch.indexmanagement.rollup.util.isRollupIndex import org.opensearch.search.DocValueFormat import org.opensearch.search.aggregations.InternalAggregation import org.opensearch.search.aggregations.InternalAggregations import org.opensearch.search.aggregations.bucket.histogram.InternalDateHistogram import org.opensearch.search.builder.SearchSourceBuilder -import org.opensearch.search.fetch.QueryFetchSearchResult import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult import org.opensearch.index.query.QueryBuilders +import org.opensearch.indexmanagement.rollup.util.convertDateStringToEpochMillis +import org.opensearch.indexmanagement.rollup.util.convertFixedIntervalStringToMs +import org.opensearch.indexmanagement.rollup.util.getRollupJobs +import org.opensearch.indexmanagement.rollup.util.isRollupIndex +import org.opensearch.indexmanagement.rollup.util.zonedDateTimeToMillis import org.opensearch.search.aggregations.metrics.InternalAvg import org.opensearch.search.aggregations.metrics.InternalMax import org.opensearch.search.aggregations.metrics.InternalMin @@ -43,9 +45,6 @@ import org.opensearch.transport.TransportRequestOptions import org.opensearch.transport.TransportResponse import org.opensearch.transport.TransportResponseHandler import java.time.ZonedDateTime -import java.time.LocalDateTime -import java.time.ZoneOffset -import java.time.format.DateTimeFormatter import java.util.concurrent.CountDownLatch import kotlin.math.max import kotlin.math.min @@ -63,8 +62,6 @@ class ResponseInterceptor( } private inner class CustomAsyncSender(private val originalSender: TransportInterceptor.AsyncSender) : TransportInterceptor.AsyncSender { - // Logic for overlap - override fun sendRequest( connection: Transport.Connection?, action: String?, @@ -83,7 +80,6 @@ class ResponseInterceptor( ) : TransportResponseHandler { override fun read(inStream: StreamInput?): T { val response = originalHandler?.read(inStream) - // Modify the response if necessary return response!! } fun isRewrittenInterceptorRequest(response: QuerySearchResult): Boolean { @@ -122,71 +118,7 @@ class ResponseInterceptor( } return Pair(rollupIndices.toTypedArray(), liveIndices.toTypedArray()) } - fun convertDateStringToEpochMillis(dateString: String): Long { - val pattern = "yyyy-MM-dd HH:mm:ss" - val formatter = DateTimeFormatter.ofPattern(pattern) - val localDateTime = LocalDateTime.parse(dateString, formatter) - val epochMillis = localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli() - return epochMillis - } - fun convertFixedIntervalStringToMs(fixedInterval: String): Long { - // Possible types are ms, s, m, h, d - val regex = """(\d+)([a-zA-Z]+)""".toRegex() - val matchResult = regex.find(fixedInterval) - ?: throw IllegalArgumentException("Invalid interval format: $fixedInterval") - - val numericValue = matchResult.groupValues[1].toLong() - val intervalType = matchResult.groupValues[2] - - val milliseconds = when (intervalType) { - "ms" -> numericValue - "s" -> numericValue * 1000L - "m" -> numericValue * 60 * 1000L - "h" -> numericValue * 60 * 60 * 1000L - "d" -> numericValue * 24 * 60 * 60 * 1000L - "w" -> numericValue * 7 * 24 * 60 * 60 * 1000L - else -> throw IllegalArgumentException("Unsupported interval type: $intervalType") - } - - return milliseconds - } - fun getIntersectionTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { - // Build search request to find the minimum rollup timestamp >= liveDataStartPoint - val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.ASC) - val query = QueryBuilders.boolQuery() - .must(QueryBuilders.rangeQuery(dateTargetField).gte(liveDataStartPoint)) - val searchSourceBuilder = SearchSourceBuilder() - .sort(sort) - .query(query) - .size(1) - // Need to avoid infinite interceptor loop - val req = SearchRequest() - .source(searchSourceBuilder) - rollupIndices.forEach { req.indices(it) } - var res: SearchResponse? = null - val latch = CountDownLatch(1) - client.search( - req, - object : ActionListener { - override fun onResponse(searchResponse: SearchResponse) { - res = searchResponse - latch.countDown() - } - override fun onFailure(e: Exception) { - logger.error("request to find intersection time failed :(", e) - latch.countDown() - } - } - ) - latch.await() - try { - return res!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long - } catch (e: Exception) { - logger.error("Not able to retrieve intersection time from response: $e") - } - return 0L // dummy :P - } // Calculated the end time for the current shard index if it is a rollup index with data overlapp fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { // Build search request to find the maximum rollup timestamp <= liveDataStartPoint @@ -227,7 +159,7 @@ class ResponseInterceptor( } // Returns Pair(startRange: Long, endRange: Long) -// Note startRange is inclusive and endRange is exclusive, they are longs becuase the type is epoch milliseconds +// Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds fun findOverlap(response: QuerySearchResult): Pair { val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later var dateSourceField: String = "" @@ -241,14 +173,12 @@ class ResponseInterceptor( break } } - // Keep existing query and add 3 fake match alls to avoid infinite loop val request: ShardSearchRequest = response.shardSearchRequest!! val oldQuery = request.source().query() - // TODO scale this for multiple indices!!!! val (rollupIndices, liveIndices) = getRollupAndLiveIndices(request) val shardRequestIndex = request.shardId().indexName val isShardIndexRollup = isRollupIndex(shardRequestIndex, clusterService.state()) - // Build search request to find the maximum date on the rolled data index + // Build search request to find the maximum date in all rollup indices var sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) var searchSourceBuilder = SearchSourceBuilder() .sort(sort) @@ -269,13 +199,13 @@ class ResponseInterceptor( } override fun onFailure(e: Exception) { - logger.error("ronsax maxLiveDate request failed ", e) + logger.error("maxLiveDate request failed in response interceptor", e) latch.countDown() } } ) latch.await() - // Build search request to find the minimum date in the live data index + // Build search request to find the minimum date in all live indices sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) searchSourceBuilder = SearchSourceBuilder() .sort(sort) @@ -283,8 +213,8 @@ class ResponseInterceptor( val minLiveDateRequest = SearchRequest() .source(searchSourceBuilder) /* - If the shard index is a rollup index I want to find the minimum value of all the live indices to compute the overlap - If the shard index is a live index I only care about the minimum value of the current shard index + If the response shard index is a rollup index, need to find the minimum value of all the live indices to compute the overlap + If the response shard index is a live index, need to only compute minimum value of the current shard index */ if (isShardIndexRollup) { liveIndices.forEach { minLiveDateRequest.indices(it) } @@ -303,7 +233,7 @@ class ResponseInterceptor( } override fun onFailure(e: Exception) { - logger.error("ronsax minLiveDate request failed ", e) + logger.error("minLiveDate request failed in response interceptor", e) latch.countDown() } } @@ -318,7 +248,6 @@ class ResponseInterceptor( val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) if (liveDataStartPoint < rollupDataEndPoint) { // Find intersection timestamp -// val intersectionTime = getIntersectionTime(liveDataStartPoint, rollupIndices, dateTargetField) if (isShardIndexRollup) { // Start at 0, end at live data val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) @@ -334,9 +263,7 @@ class ResponseInterceptor( // No overlap so start and end include everything return Pair(0L, Long.MAX_VALUE) } - fun zonedDateTimeToMillis(zonedDateTime: ZonedDateTime): Long { - return zonedDateTime.toInstant().toEpochMilli() - } + // Depending on which metric the aggregation is computer data differently fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any): Pair { when (agg) { @@ -413,23 +340,8 @@ class ResponseInterceptor( } // Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) - /** - * @param intervalAggregations - * @return Int - */ fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { -// /* -// PSUEDOCODE -// 1. Look at first bucket to see which aggs where in initial request -// 2. Store in a map of {aggregationName: [aggType, runningSum/Min/Max/Avg to change] -// 3. Iterate through all buckets -// 4. if bucket in range start <= timeStamp < end -// 5. update all computation values in map -// 6. init new InternalAggregations object -// 7. iterate throguh all key, vals in map and construct an internalAggregation object for each of them, add to InternalAggregations object -// 8. return InternalAggregations object -// */ - // Create an empty map to hold the agg values + // Store the running values of the aggregations being computed // {aggName: String: Pair} val aggValues = mutableMapOf>() // @@ -494,23 +406,6 @@ class ResponseInterceptor( } else { originalHandler?.handleResponse(response) } - } - // when just 1 rollup index is in request, keep for testing - is QueryFetchSearchResult -> { - val queryResult = response.queryResult() - if (queryResult.hasAggs() && isRewrittenInterceptorRequest(queryResult)) { - // Check for overlap - val (startTime, endTime) = findOverlap(queryResult) - // Modify agg to be original result without overlap computed in - queryResult.aggregations(computeAggregationsWithoutOverlap(queryResult.aggregations().expand(), startTime, endTime)) - // TODO change response object -// val r1 = QueryFetchSearchResult(response.queryResult(), response.fetchResult()) -// r1.shardIndex = response.shardIndex -// val r2: T = r1 as T - originalHandler?.handleResponse(response) - } else { - originalHandler?.handleResponse(response) - } } else -> { // Delegate to original handler originalHandler?.handleResponse(response) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 5b6d44f32..94b80eb10 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -120,32 +120,6 @@ class RollupInterceptor( } return false } - fun copyAggregations(oldAggs: AggregatorFactories.Builder): AggregatorFactories.Builder { - try { - val aggsStr = BytesStreamOutput().use { out -> - out.writeVersion(Version.CURRENT) - oldAggs.writeTo(out) - val bytes = BytesReference.toBytes(out.bytes()) - Base64.getUrlEncoder().withoutPadding().encodeToString(bytes) - } - val bytesReference = BytesReference.fromByteBuffer( - ByteBuffer.wrap( - Base64.getUrlDecoder().decode( - aggsStr - ) - ) - ) - val wrapperStreamInput = NamedWriteableAwareStreamInput( - bytesReference.streamInput(), - NamedWriteableRegistry(SearchModule(Settings.EMPTY, emptyList()).namedWriteables) - ) - wrapperStreamInput.setVersion(wrapperStreamInput.readVersion()) - return AggregatorFactories.Builder(wrapperStreamInput) - } catch (e: Exception) { - logger.error(e) - return oldAggs - } - } // Need to modify aggs for rollup docs with avg and value count aggs fun modifyRollupAggs(aggFacts: MutableCollection): AggregatorFactories.Builder { val build = AggregatorFactories.builder() @@ -205,7 +179,7 @@ class RollupInterceptor( .calendarInterval(DateHistogramInterval(rollupInterval)) .format("epoch_millis") .subAggregations(oldAggs) - // Changes aggreagtion in source to new agg + // Changes aggregation in source to new agg request.source(request.source().changeAggregations(listOf(intervalAgg))) return } diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index 01a950f8c..5f8875ad6 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -62,6 +62,10 @@ import org.opensearch.search.aggregations.metrics.ScriptedMetricAggregationBuild import org.opensearch.search.aggregations.metrics.SumAggregationBuilder import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder import org.opensearch.search.builder.SearchSourceBuilder +import java.time.LocalDateTime +import java.time.ZoneOffset +import java.time.ZonedDateTime +import java.time.format.DateTimeFormatter import java.util.* const val DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT = "strict_date_optional_time" @@ -499,6 +503,36 @@ fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collect return ssb } -// fun InternalAvg.getSumAndCount(): Pair { -// return Pair(this.getSum(), this. -// } +fun convertDateStringToEpochMillis(dateString: String): Long { + val pattern = "yyyy-MM-dd HH:mm:ss" + val formatter = DateTimeFormatter.ofPattern(pattern) + val localDateTime = LocalDateTime.parse(dateString, formatter) + val epochMillis = localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli() + return epochMillis +} + +fun convertFixedIntervalStringToMs(fixedInterval: String): Long { + // Possible types are ms, s, m, h, d + val regex = """(\d+)([a-zA-Z]+)""".toRegex() + val matchResult = regex.find(fixedInterval) + ?: throw IllegalArgumentException("Invalid interval format: $fixedInterval") + + val numericValue = matchResult.groupValues[1].toLong() + val intervalType = matchResult.groupValues[2] + + val milliseconds = when (intervalType) { + "ms" -> numericValue + "s" -> numericValue * 1000L + "m" -> numericValue * 60 * 1000L + "h" -> numericValue * 60 * 60 * 1000L + "d" -> numericValue * 24 * 60 * 60 * 1000L + "w" -> numericValue * 7 * 24 * 60 * 60 * 1000L + else -> throw IllegalArgumentException("Unsupported interval type: $intervalType") + } + + return milliseconds +} + +fun zonedDateTimeToMillis(zonedDateTime: ZonedDateTime): Long { + return zonedDateTime.toInstant().toEpochMilli() +} From ef7682f87be37b98fbd226dbbb5b6eac0d03f6b5 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 30 Aug 2023 14:49:20 -0700 Subject: [PATCH 19/44] fixed detekt errors Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 39 +++++++++++-------- .../rollup/interceptor/RollupInterceptor.kt | 18 ++++----- .../rollup/util/RollupUtils.kt | 4 +- 3 files changed, 32 insertions(+), 29 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index b83cf403b..5ebb7b1c2 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -75,6 +75,7 @@ class ResponseInterceptor( } } + @Suppress("TooManyFunctions") private inner class CustomResponseHandler( private val originalHandler: TransportResponseHandler? ) : TransportResponseHandler { @@ -91,6 +92,7 @@ class ResponseInterceptor( } return false } + @Suppress("SpreadOperator") fun getRollupJob(response: QuerySearchResult): Rollup? { val originalRequest = response.shardSearchRequest!! val indices = originalRequest.indices().map { it.toString() }.toTypedArray() @@ -103,6 +105,7 @@ class ResponseInterceptor( } return null } + @Suppress("SpreadOperator") fun getRollupAndLiveIndices(request: ShardSearchRequest): Pair, Array> { val liveIndices = mutableListOf() val rollupIndices = mutableListOf() @@ -160,6 +163,7 @@ class ResponseInterceptor( // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds + @Suppress("LongMethod") fun findOverlap(response: QuerySearchResult): Pair { val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later var dateSourceField: String = "" @@ -239,32 +243,29 @@ class ResponseInterceptor( } ) latch.await() + // if they overlap find part to exclude - if (minLiveDateResponse != null && maxRolledDateResponse != null && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { + @Suppress("ComplexCondition") + if (minLiveDateResponse != null && maxRolledDateResponse != null && minLiveDateResponse!!.hits.hits.isNotEmpty() + && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { // Rollup data ends at maxRolledDate + fixedInterval val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) - if (liveDataStartPoint < rollupDataEndPoint) { - // Find intersection timestamp - if (isShardIndexRollup) { - // Start at 0, end at live data - val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) - return Pair(0L, endTime) - } else { - // Include all live data - return Pair(0L, Long.MAX_VALUE) - } + // If intersection found on rollup index, remove overlap + if ((liveDataStartPoint < rollupDataEndPoint) && isShardIndexRollup) { + // Start at 0, end at live data + val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) + return Pair(0L, endTime) } - } else { - logger.error("Not able to make client calls in response interceptor") } - // No overlap so start and end include everything + // No overlap or is live data index so start and end include everything return Pair(0L, Long.MAX_VALUE) } // Depending on which metric the aggregation is computer data differently + @Suppress("ReturnCount") fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any): Pair { when (agg) { is InternalSum -> { @@ -287,6 +288,7 @@ class ResponseInterceptor( } } // Depending on which metric the aggregation is return a different start value + @Suppress("ReturnCount") fun getAggComputationStartValue(agg: org.opensearch.search.aggregations.Aggregation): Pair { when (agg) { is InternalSum -> return Pair(agg.value, agg.type) @@ -297,6 +299,7 @@ class ResponseInterceptor( else -> throw IllegalArgumentException("This aggregation is not currently supported in rollups searches: ${agg.name}") } } + @Suppress("ReturnCount") fun createNewMetricAgg(aggName: String, aggValue: Any, aggType: String): InternalAggregation { when (aggType) { "sum" -> return InternalSum(aggName, (aggValue as Double), DocValueFormat.RAW, null) @@ -307,7 +310,8 @@ class ResponseInterceptor( } } // Create original avg aggregation - fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, addedAggregations: MutableSet): InternalAvg { + fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, + addedAggregations: MutableSet): InternalAvg { // Sum calc if (modifiedName.contains(".rollup.avg.sum")) { // Won't double count @@ -336,10 +340,11 @@ class ResponseInterceptor( } } } - throw Exception("Can't calculate avg agg for rollup index") + throw NullPointerException("Can't calculate avg agg for rollup index") } -// Returns a new InternalAggregations that contains a merged aggregation(s) with the overlapping data removed, computation varies based on metric used (edge case avg?) +// Returns a new InternalAggregations that contains merged aggregation(s) with the overlapping data removed + @Suppress("NestedBlockDepth") fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { // Store the running values of the aggregations being computed // {aggName: String: Pair} diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 94b80eb10..a9457b0f2 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -6,15 +6,10 @@ package org.opensearch.indexmanagement.rollup.interceptor import org.apache.logging.log4j.LogManager -import org.opensearch.Version import org.opensearch.action.support.IndicesOptions import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.bytes.BytesReference -import org.opensearch.common.io.stream.BytesStreamOutput -import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput -import org.opensearch.common.io.stream.NamedWriteableRegistry import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.index.query.QueryBuilder @@ -43,7 +38,6 @@ import org.opensearch.indexmanagement.rollup.util.populateFieldMappings import org.opensearch.indexmanagement.rollup.util.getDateHistogram import org.opensearch.indexmanagement.rollup.util.rewriteSearchSourceBuilder import org.opensearch.indexmanagement.util.IndexUtils -import org.opensearch.search.SearchModule import org.opensearch.search.aggregations.AggregatorFactories import org.opensearch.search.aggregations.AggregationBuilder import org.opensearch.search.aggregations.AggregationBuilders @@ -62,8 +56,7 @@ import org.opensearch.transport.TransportChannel import org.opensearch.transport.TransportInterceptor import org.opensearch.transport.TransportRequest import org.opensearch.transport.TransportRequestHandler -import java.nio.ByteBuffer -import java.util.Base64 +@Suppress("TooManyFunctions") class RollupInterceptor( val clusterService: ClusterService, val settings: Settings, @@ -85,6 +78,7 @@ class RollupInterceptor( } } // Returns Pair + @Suppress("SpreadOperator") private fun originalSearchContainsRollup(request: ShardSearchRequest): Pair { val indices = request.indices().map { it.toString() }.toTypedArray() val allIndices = indexNameExpressionResolver @@ -201,7 +195,9 @@ class RollupInterceptor( if (containsRollup || isRollupIndex) { val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) // Avoid infinite interceptor loop - if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { + @Suppress("ComplexCondition") + if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() + && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts breakRequestIntoBuckets(request, rollupJob!!) } @@ -209,7 +205,8 @@ class RollupInterceptor( if (isRollupIndex && !isReqeustRollupFormat(request)) { // TODO fix logic to allow response interceptor client calls to have a size of 1 // if (!requestCalledInInterceptor && request.source().size() != 0) { -// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") +// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, " + +// "but found ${request.source().size()}") // } rewriteRollupRequest(request, rollupJob!!, concreteRollupIndicesArray) } @@ -220,6 +217,7 @@ class RollupInterceptor( } } // Returns Pair (concreteRollupIndices: Array, concreteLiveIndicesArray: Array) + @Suppress("SpreadOperator") fun getConcreteIndices(request: ShardSearchRequest): Pair, Array> { val indices = request.indices().map { it.toString() }.toTypedArray() val concreteIndices = indexNameExpressionResolver diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index 5f8875ad6..edfa02288 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -66,7 +66,6 @@ import java.time.LocalDateTime import java.time.ZoneOffset import java.time.ZonedDateTime import java.time.format.DateTimeFormatter -import java.util.* const val DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT = "strict_date_optional_time" const val DATE_FIELD_EPOCH_MILLIS_FORMAT = "epoch_millis" @@ -470,6 +469,7 @@ fun parseRollup(response: GetResponse, xContentRegistry: NamedXContentRegistry = return xcp.parseWithType(response.id, response.seqNo, response.primaryTerm, Rollup.Companion::parse) } // Changes aggregations in search source builder to new original aggregation (Change query too?) +@Suppress("ComplexMethod") fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collection): SearchSourceBuilder { val ssb = SearchSourceBuilder() aggregationBuilderCollection.forEach { ssb.aggregation(it) } @@ -510,7 +510,7 @@ fun convertDateStringToEpochMillis(dateString: String): Long { val epochMillis = localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli() return epochMillis } - +@Suppress("MagicNumber") fun convertFixedIntervalStringToMs(fixedInterval: String): Long { // Possible types are ms, s, m, h, d val regex = """(\d+)([a-zA-Z]+)""".toRegex() From e9fa46cb2ab4748e5ac8e5f3caceb6d04bcd0dd0 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 30 Aug 2023 15:07:21 -0700 Subject: [PATCH 20/44] fixed ktlint error :/' Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 14 ++++++++------ .../rollup/interceptor/RollupInterceptor.kt | 5 ++--- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 5ebb7b1c2..389bc6628 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -243,11 +243,9 @@ class ResponseInterceptor( } ) latch.await() - + val foundMinAndMax = (minLiveDateResponse != null && maxRolledDateResponse != null) // if they overlap find part to exclude - @Suppress("ComplexCondition") - if (minLiveDateResponse != null && maxRolledDateResponse != null && minLiveDateResponse!!.hits.hits.isNotEmpty() - && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { + if (foundMinAndMax && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { // Rollup data ends at maxRolledDate + fixedInterval val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) @@ -310,8 +308,12 @@ class ResponseInterceptor( } } // Create original avg aggregation - fun initRollupAvgAgg(modifiedName: String, value: Any, aggValues: MutableMap>, - addedAggregations: MutableSet): InternalAvg { + fun initRollupAvgAgg( + modifiedName: String, + value: Any, + aggValues: MutableMap>, + addedAggregations: MutableSet + ): InternalAvg { // Sum calc if (modifiedName.contains(".rollup.avg.sum")) { // Won't double count diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index a9457b0f2..e7438a4b1 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -195,9 +195,8 @@ class RollupInterceptor( if (containsRollup || isRollupIndex) { val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) // Avoid infinite interceptor loop - @Suppress("ComplexCondition") - if (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty() - && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { + val isMultiSearch = (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty()) + if (isMultiSearch && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts breakRequestIntoBuckets(request, rollupJob!!) } From c8d8790c27f6e3ab530b273b7f01c59ba0129c28 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 30 Aug 2023 20:56:07 -0700 Subject: [PATCH 21/44] Can run all integ tests at once now Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index bc802adb9..c6ea7a0ce 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -30,7 +30,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { fun `test search a live index and rollup index with no overlap`() { generateNYCTaxiData("source_rollup_search") val rollup = Rollup( - id = "basic_term_query_rollup_search", + id = "base_case1_rollup_search", enabled = true, schemaVersion = 1L, jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), @@ -189,17 +189,17 @@ class ResponseInterceptorIT : RollupRestTestCase() { } // Edge Case fun `test search a live index with no data and rollup index with data`() { - generateNYCTaxiData("source_rollup_search") + generateNYCTaxiData("source_rollup_search_no_data_case") val rollup = Rollup( - id = "basic_term_query_rollup_search", + id = "base_case_2_rollup_search", enabled = true, schemaVersion = 1L, jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), jobLastUpdatedTime = Instant.now(), jobEnabledTime = Instant.now(), description = "basic search test", - sourceIndex = "source_rollup_search", - targetIndex = "target_rollup_search", + sourceIndex = "source_rollup_search_no_data_case", + targetIndex = "target_rollup_search_no_data_case", metadataID = null, roles = emptyList(), pageSize = 10, @@ -266,7 +266,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { } } """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var searchResponse = client().makeRequest("POST", "/source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) var expectedAggs = searchResponse.asMap()["aggregations"] as Map> val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] @@ -278,12 +278,12 @@ class ResponseInterceptorIT : RollupRestTestCase() { // Delete values from live index var deleteResponse = client().makeRequest( "POST", - "source_rollup_search/_delete_by_query", + "source_rollup_search_no_data_case/_delete_by_query", mapOf("refresh" to "true"), StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) ) assertTrue(deleteResponse.restStatus() == RestStatus.OK) - var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_no_data_case,source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue(searchBothResponse.restStatus() == RestStatus.OK) var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> assertEquals( @@ -313,17 +313,17 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) } fun `test search a live index and rollup index with data overlap`() { - generateNYCTaxiData("source_rollup_search") + generateNYCTaxiData("source_rollup_search_data_overlap_case") val rollup = Rollup( - id = "basic_term_query_rollup_search", + id = "case2_rollup_search", enabled = true, schemaVersion = 1L, jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), jobLastUpdatedTime = Instant.now(), jobEnabledTime = Instant.now(), description = "basic search test", - sourceIndex = "source_rollup_search", - targetIndex = "target_rollup_search", + sourceIndex = "source_rollup_search_data_overlap_case", + targetIndex = "target_rollup_search_data_overlap_case", metadataID = null, roles = emptyList(), pageSize = 10, @@ -388,7 +388,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { } } """.trimIndent() - var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] @@ -415,7 +415,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { """.trimIndent() var deleteLiveResponse = client().makeRequest( "POST", - "source_rollup_search/_delete_by_query", + "source_rollup_search_data_overlap_case/_delete_by_query", mapOf("refresh" to "true"), StringEntity(r, ContentType.APPLICATION_JSON) ) @@ -423,7 +423,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) // Rollup index is complete overlap of live data // Search both and check if time series data is the same - var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_data_overlap_case,source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue(searchBothResponse.restStatus() == RestStatus.OK) var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> assertEquals( @@ -453,17 +453,17 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) } fun `test search multiple live data indices and a rollup data index with overlap`() { - generateNYCTaxiData("source_rollup_search") + generateNYCTaxiData("source_rollup_search_multi_index_case") val rollup = Rollup( - id = "basic_term_query_rollup_search", + id = "case3_rollup_search", enabled = true, schemaVersion = 1L, jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), jobLastUpdatedTime = Instant.now(), jobEnabledTime = Instant.now(), description = "basic search test", - sourceIndex = "source_rollup_search", - targetIndex = "target_rollup_search", + sourceIndex = "source_rollup_search_multi_index_case", + targetIndex = "target_rollup_search_multi_index_case", metadataID = null, roles = emptyList(), pageSize = 10, @@ -511,7 +511,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { """.trimIndent() var deleteLiveResponse = client().makeRequest( "POST", - "source_rollup_search/_delete_by_query", + "source_rollup_search_multi_index_case/_delete_by_query", mapOf("refresh" to "true"), StringEntity(r, ContentType.APPLICATION_JSON) ) @@ -519,7 +519,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) // Insert more live data - generateNYCTaxiData("source_rollup_search2") + generateNYCTaxiData("source_rollup_search_multi_index_case2") // Expected values would discard the overlapping rollup index completely var aggReq = """ { @@ -556,7 +556,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { } } """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search,source_rollup_search2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) var expectedAggs = searchResponse.asMap()["aggregations"] as Map> val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] @@ -567,7 +567,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { refreshAllIndices() // Search all 3 indices to check if overlap was removed - var searchAllResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search,source_rollup_search2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue(searchAllResponse.restStatus() == RestStatus.OK) var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> assertEquals( @@ -636,7 +636,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) // Rollup alias into rollup-nyc-taxi-data val rollup = Rollup( - id = "basic_term_query_rollup_search", + id = "alias_rollup_search", enabled = true, schemaVersion = 1L, jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), From 1e3294f29aefde5e0dbc6d989e7e9b00132f4d99 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 31 Aug 2023 10:01:25 -0700 Subject: [PATCH 22/44] removed DateTimeFromatter Signed-off-by: Ronnak Saxena --- .../rollup/util/RollupUtils.kt | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index edfa02288..615af1a74 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -65,7 +65,6 @@ import org.opensearch.search.builder.SearchSourceBuilder import java.time.LocalDateTime import java.time.ZoneOffset import java.time.ZonedDateTime -import java.time.format.DateTimeFormatter const val DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT = "strict_date_optional_time" const val DATE_FIELD_EPOCH_MILLIS_FORMAT = "epoch_millis" @@ -504,11 +503,21 @@ fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collect } fun convertDateStringToEpochMillis(dateString: String): Long { - val pattern = "yyyy-MM-dd HH:mm:ss" - val formatter = DateTimeFormatter.ofPattern(pattern) - val localDateTime = LocalDateTime.parse(dateString, formatter) - val epochMillis = localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli() - return epochMillis + val parts = dateString.split(" ") + val dateParts = parts[0].split("-") + val timeParts = parts[1].split(":") + + val year = dateParts[0].toInt() + val month = dateParts[1].toInt() + val day = dateParts[2].toInt() + + val hour = timeParts[0].toInt() + val minute = timeParts[1].toInt() + val second = timeParts[2].toInt() + + val localDateTime = LocalDateTime.of(year, month, day, hour, minute, second) + val instant = localDateTime.toInstant(ZoneOffset.UTC) + return instant.toEpochMilli() } @Suppress("MagicNumber") fun convertFixedIntervalStringToMs(fixedInterval: String): Long { From 1a7da76c644cce8225b491bc27c766cf750a941a Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 5 Sep 2023 11:56:42 -0700 Subject: [PATCH 23/44] fixed inf interceptor loop, need to pass RollupInterceptorIT.test rollup search multiple target indices failed test Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/RollupInterceptor.kt | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index e7438a4b1..454e58bc4 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -114,6 +114,10 @@ class RollupInterceptor( } return false } + // If the request has a sort on it, size can be > 0 on a rollup search + fun canHaveSize(request: ShardSearchRequest): Boolean { + return request.source().sorts() != null + } // Need to modify aggs for rollup docs with avg and value count aggs fun modifyRollupAggs(aggFacts: MutableCollection): AggregatorFactories.Builder { val build = AggregatorFactories.builder() @@ -203,10 +207,12 @@ class RollupInterceptor( // Rewrite the request to fit rollup format if not already done previously if (isRollupIndex && !isReqeustRollupFormat(request)) { // TODO fix logic to allow response interceptor client calls to have a size of 1 -// if (!requestCalledInInterceptor && request.source().size() != 0) { -// throw IllegalArgumentException("Rollup search must have size explicitly set to 0, " + -// "but found ${request.source().size()}") -// } + if (!canHaveSize(request) && request.source().size() != 0) { + throw IllegalArgumentException( + "Rollup search must have size explicitly set to 0, " + + "but found ${request.source().size()}" + ) + } rewriteRollupRequest(request, rollupJob!!, concreteRollupIndicesArray) } } From b145ecff7fd80723a04948a59ff387a17671a9e7 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 5 Sep 2023 13:18:25 -0700 Subject: [PATCH 24/44] passes all integ tests Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 6 +++--- .../rollup/interceptor/RollupInterceptorIT.kt | 15 --------------- 2 files changed, 3 insertions(+), 18 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 389bc6628..2ce656c57 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -135,7 +135,7 @@ class ResponseInterceptor( // Need to avoid infinite interceptor loop val req = SearchRequest() .source(searchSourceBuilder) - rollupIndices.forEach { req.indices(it) } + .indices(*rollupIndices) var res: SearchResponse? = null val latch = CountDownLatch(1) client.search( @@ -191,7 +191,7 @@ class ResponseInterceptor( // Need to avoid infinite interceptor loop val maxRolledDateRequest = SearchRequest() .source(searchSourceBuilder) - rollupIndices.forEach { maxRolledDateRequest.indices(it) } // add all rollup indices to this request + .indices(*rollupIndices) // add all rollup indices to this request var maxRolledDateResponse: SearchResponse? = null var latch = CountDownLatch(1) client.search( @@ -221,7 +221,7 @@ class ResponseInterceptor( If the response shard index is a live index, need to only compute minimum value of the current shard index */ if (isShardIndexRollup) { - liveIndices.forEach { minLiveDateRequest.indices(it) } + minLiveDateRequest.indices(*liveIndices) } else { // shard index is live index minLiveDateRequest.indices(shardRequestIndex) } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt index e42295418..105cc20c6 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt @@ -1045,21 +1045,6 @@ class RollupInterceptorIT : RollupRestTestCase() { } } """.trimIndent() - // Search 1 non-rollup index and 1 rollup - val searchResult1 = client().makeRequest("POST", "/$sourceIndex2,$targetIndex2/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) - assertTrue(searchResult1.restStatus() == RestStatus.OK) - val failures = extractFailuresFromSearchResponse(searchResult1) - assertNotNull(failures) - assertEquals(1, failures?.size) - assertEquals( - "Searching multiple indices where one is rollup and other is not, didn't return failure", - "illegal_argument_exception", failures?.get(0)?.get("type") ?: "Didn't find failure type in search response" - - ) - assertEquals( - "Searching multiple indices where one is rollup and other is not, didn't return failure", - "Not all indices have rollup job", failures?.get(0)?.get("reason") ?: "Didn't find failure reason in search response" - ) // Search 2 rollups with different mappings try { From a9c6369b15b714a327f6e2b275d8a2711f025160 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 5 Sep 2023 13:26:08 -0700 Subject: [PATCH 25/44] fixed detekt errors Signed-off-by: Ronnak Saxena --- .../indexmanagement/rollup/interceptor/ResponseInterceptor.kt | 3 ++- .../indexmanagement/rollup/interceptor/RollupInterceptor.kt | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 2ce656c57..0e279c22c 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -123,6 +123,7 @@ class ResponseInterceptor( } // Calculated the end time for the current shard index if it is a rollup index with data overlapp + @Suppress("SpreadOperator") fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { // Build search request to find the maximum rollup timestamp <= liveDataStartPoint val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) @@ -163,7 +164,7 @@ class ResponseInterceptor( // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds - @Suppress("LongMethod") + @Suppress("LongMethod", "SpreadOperator") fun findOverlap(response: QuerySearchResult): Pair { val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later var dateSourceField: String = "" diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 454e58bc4..3bb16118a 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -182,7 +182,7 @@ class RollupInterceptor( return } - @Suppress("SpreadOperator") + @Suppress("SpreadOperator", "NestedBlockDepth") override fun interceptHandler( action: String, executor: String, From 495ee4d0c2f05590b116535bd76c7d52dd116d49 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 6 Sep 2023 17:57:52 +0000 Subject: [PATCH 26/44] deleted rest test Signed-off-by: Ronnak Saxena --- .../rollup/resthandler/RestDeleteRollupActionIT.kt | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt index 43fc6dca5..3f5dc42d8 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt @@ -6,7 +6,6 @@ package org.opensearch.indexmanagement.rollup.resthandler import org.opensearch.client.ResponseException -import org.opensearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX import org.opensearch.indexmanagement.IndexManagementPlugin.Companion.ROLLUP_JOBS_BASE_URI import org.opensearch.indexmanagement.makeRequest import org.opensearch.indexmanagement.rollup.RollupRestTestCase @@ -37,15 +36,4 @@ class RestDeleteRollupActionIT : RollupRestTestCase() { assertEquals(RestStatus.NOT_FOUND, e.response.restStatus()) } } - - @Throws(Exception::class) - fun `test deleting a rollup that doesn't exist and config index doesnt exist`() { - try { - deleteIndex(INDEX_MANAGEMENT_INDEX) - client().makeRequest("DELETE", "$ROLLUP_JOBS_BASE_URI/foobarbaz") - fail("expected 404 ResponseException") - } catch (e: ResponseException) { - assertEquals(RestStatus.NOT_FOUND, e.response.restStatus()) - } - } } From 4dd409621dc79466edb27f41802c209a4d12134d Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 6 Sep 2023 21:56:29 +0000 Subject: [PATCH 27/44] added test back Signed-off-by: Ronnak Saxena --- .../rollup/resthandler/RestDeleteRollupActionIT.kt | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt index 3f5dc42d8..cfef1f7bc 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/resthandler/RestDeleteRollupActionIT.kt @@ -6,6 +6,7 @@ package org.opensearch.indexmanagement.rollup.resthandler import org.opensearch.client.ResponseException +import org.opensearch.indexmanagement.IndexManagementPlugin.Companion.INDEX_MANAGEMENT_INDEX import org.opensearch.indexmanagement.IndexManagementPlugin.Companion.ROLLUP_JOBS_BASE_URI import org.opensearch.indexmanagement.makeRequest import org.opensearch.indexmanagement.rollup.RollupRestTestCase @@ -36,4 +37,14 @@ class RestDeleteRollupActionIT : RollupRestTestCase() { assertEquals(RestStatus.NOT_FOUND, e.response.restStatus()) } } + @Throws(Exception::class) + fun `test deleting a rollup that doesn't exist and config index doesnt exist`() { + try { + deleteIndex(INDEX_MANAGEMENT_INDEX) + client().makeRequest("DELETE", "$ROLLUP_JOBS_BASE_URI/foobarbaz") + fail("expected 404 ResponseException") + } catch (e: ResponseException) { + assertEquals(RestStatus.NOT_FOUND, e.response.restStatus()) + } + } } From d94fbbdf7e0154697f2f1e5fe254b4d509c0d2d0 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 6 Sep 2023 15:43:22 -0700 Subject: [PATCH 28/44] trying a new workflow build Signed-off-by: Ronnak Saxena --- .github/workflows/multi-node-test-workflow.yml | 4 ++-- .../indexmanagement/rollup/util/RollupUtilsTests.kt | 3 +++ 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index aaa37dc98..baa186175 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -3,7 +3,7 @@ name: Multi node test workflow on: pull_request: branches: - - "*" + - "**" push: branches: - "*" @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest -PnumNodes=3 + run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt index c063a3487..face563b2 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt @@ -230,4 +230,7 @@ class RollupUtilsTests : OpenSearchTestCase() { assertEquals("Rewritten aggregation builder is not the correct type", aggBuilder.type, rewrittenAgg.type) } } + fun `test changeAggregations`() { + /* add later */ + } } From 4ef546bff57553609e6d89e9da573016fc379e64 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 6 Sep 2023 15:53:20 -0700 Subject: [PATCH 29/44] added stars to worklow files Signed-off-by: Ronnak Saxena --- .github/workflows/docker-security-test-workflow.yml | 2 +- .github/workflows/security-test-workflow.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docker-security-test-workflow.yml b/.github/workflows/docker-security-test-workflow.yml index 6ff563f5b..9ca19f548 100644 --- a/.github/workflows/docker-security-test-workflow.yml +++ b/.github/workflows/docker-security-test-workflow.yml @@ -2,7 +2,7 @@ name: Docker Security Test Workflow on: pull_request: branches: - - "*" + - "**" push: branches: - "*" diff --git a/.github/workflows/security-test-workflow.yml b/.github/workflows/security-test-workflow.yml index 71bfea68e..e8d34fcc7 100644 --- a/.github/workflows/security-test-workflow.yml +++ b/.github/workflows/security-test-workflow.yml @@ -3,7 +3,7 @@ name: Security test workflow on: pull_request: branches: - - "*" + - "**" push: branches: - "*" From bf04e3290eb57035b99ca8a5ba46ebdd9c3c1a19 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 12:18:30 -0700 Subject: [PATCH 30/44] added unit test Signed-off-by: Ronnak Saxena --- .../rollup/util/RollupUtils.kt | 6 +- .../rollup/util/RollupUtilsTests.kt | 60 ++++++++++++++++++- 2 files changed, 63 insertions(+), 3 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt index 615af1a74..86e7342be 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -467,7 +467,7 @@ fun parseRollup(response: GetResponse, xContentRegistry: NamedXContentRegistry = return xcp.parseWithType(response.id, response.seqNo, response.primaryTerm, Rollup.Companion::parse) } -// Changes aggregations in search source builder to new original aggregation (Change query too?) +// Returns a SearchSourceBuilder with different aggregations but the rest of the properties are the same @Suppress("ComplexMethod") fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collection): SearchSourceBuilder { val ssb = SearchSourceBuilder() @@ -501,12 +501,14 @@ fun SearchSourceBuilder.changeAggregations(aggregationBuilderCollection: Collect if (this.collapse() != null) ssb.collapse(this.collapse()) return ssb } - +@Suppress("MagicNumber") fun convertDateStringToEpochMillis(dateString: String): Long { val parts = dateString.split(" ") + require(parts.size == 2) { "Date in was not correct format" } val dateParts = parts[0].split("-") val timeParts = parts[1].split(":") + require((dateParts.size == 3 && timeParts.size == 3)) { "Date in was not correct format" } val year = dateParts[0].toInt() val month = dateParts[1].toInt() val day = dateParts[2].toInt() diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt index face563b2..fdb5cece6 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt @@ -31,8 +31,13 @@ import org.opensearch.indexmanagement.rollup.randomValueCount import org.opensearch.indexmanagement.transform.randomAggregationBuilder import org.opensearch.search.aggregations.metrics.AvgAggregationBuilder import org.opensearch.search.aggregations.metrics.ValueCountAggregationBuilder +import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.test.OpenSearchTestCase import org.opensearch.test.rest.OpenSearchRestTestCase +import java.time.ZoneId +import java.time.ZonedDateTime +import kotlin.test.assertFails +import kotlin.test.assertFailsWith class RollupUtilsTests : OpenSearchTestCase() { @@ -231,6 +236,59 @@ class RollupUtilsTests : OpenSearchTestCase() { } } fun `test changeAggregations`() { - /* add later */ + val ssb = SearchSourceBuilder() + val oldAggBuilder = randomAggregationBuilder() + ssb.aggregation(oldAggBuilder) + var newAgg = randomAggregationBuilder() + while (newAgg == oldAggBuilder) newAgg = randomAggregationBuilder() + val newSsb = ssb.changeAggregations(listOf(newAgg)) + assertNotEquals("Did not change search source builders aggregations :(", newSsb, ssb) + } + fun `test convertDateStringToEpochMillis`() { + // Check correct time format + val dateString = "2023-07-18 12:30:00" + val expectedMillis = 1689683400000L + val actualMillis = convertDateStringToEpochMillis(dateString) + assertEquals(expectedMillis, actualMillis) + + // Testing an invalid date format throws error + assertFails { convertDateStringToEpochMillis("invalid format") } + } + fun `test convertFixedIntervalStringToMs`() { + // Test ms + val msString = "5ms" + val expectedMs = 5L + assertEquals("ms conversion is wrong", convertFixedIntervalStringToMs(msString), expectedMs) + // Test s + val sString = "5s" + val expectedS = 5000L + assertEquals("ms conversion is wrong", convertFixedIntervalStringToMs(sString), expectedS) + // Test m + val mString = "3m" + val expectedM = 180000L + assertEquals("m conversion is wrong", convertFixedIntervalStringToMs(mString), expectedM) + // Test h + val hString = "2h" + val expectedH = 7200000L + assertEquals("h conversion is wrong", convertFixedIntervalStringToMs(hString), expectedH) + // Test d + val dString = "1d" + val expectedD = 86400000L + assertEquals("d conversion is wrong", convertFixedIntervalStringToMs(dString), expectedD) + // Test w + val wString = "1w" + val expectedW = 604800000L + assertEquals("w conversion is wrong", convertFixedIntervalStringToMs(wString), expectedW) + // Test error + val invalid = ";)" + assertFailsWith { + convertFixedIntervalStringToMs(invalid) + } + } + fun `test zonedDateTimeToMillis`() { + val zonedDateTime = ZonedDateTime.of(2023, 7, 18, 12, 30, 0, 0, ZoneId.of("UTC")) + val expectedMillis = 1689683400000L // ms since epoch of the zonedDateTime + val actualMillis = zonedDateTimeToMillis(zonedDateTime) + assertEquals(expectedMillis, actualMillis) } } From d618e9026bbead18ba0ac778b6e140900d094b09 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 12:31:16 -0700 Subject: [PATCH 31/44] resolved some PR comments from Bowen Signed-off-by: Ronnak Saxena --- .../org/opensearch/indexmanagement/IndexManagementPlugin.kt | 2 +- .../indexmanagement/rollup/interceptor/RollupInterceptor.kt | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt index 444bafc09..7db5aba11 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt @@ -392,7 +392,7 @@ class IndexManagementPlugin : JobSchedulerExtension, NetworkPlugin, ActionPlugin namedWriteableRegistry, environment ) - rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver, xContentRegistry) + rollupInterceptor = RollupInterceptor(clusterService, settings, indexNameExpressionResolver) responseInterceptor = ResponseInterceptor(clusterService, settings, indexNameExpressionResolver, client) val jvmService = JvmService(environment.settings()) val transformRunner = TransformRunner.initialize( diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 3bb16118a..d7c15e460 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -11,7 +11,6 @@ import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.settings.Settings -import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.index.query.QueryBuilder import org.opensearch.index.query.TermQueryBuilder import org.opensearch.index.query.TermsQueryBuilder @@ -61,7 +60,6 @@ class RollupInterceptor( val clusterService: ClusterService, val settings: Settings, val indexNameExpressionResolver: IndexNameExpressionResolver, - val xContentRegistry: NamedXContentRegistry ) : TransportInterceptor { private val logger = LogManager.getLogger(javaClass) @@ -206,7 +204,9 @@ class RollupInterceptor( } // Rewrite the request to fit rollup format if not already done previously if (isRollupIndex && !isReqeustRollupFormat(request)) { - // TODO fix logic to allow response interceptor client calls to have a size of 1 + /* Client calls from the response interceptor require request bodies of 1, + otherwise do not allow size > 0 for rollup indices + */ if (!canHaveSize(request) && request.source().size() != 0) { throw IllegalArgumentException( "Rollup search must have size explicitly set to 0, " + From 9c567ee727031dc9f5564e17758c4b3ee8485b6c Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 13:14:42 -0700 Subject: [PATCH 32/44] resolved more comments on my PR Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 1 + .../indexmanagement/rollup/interceptor/RollupInterceptor.kt | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 0e279c22c..27fabd310 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -219,6 +219,7 @@ class ResponseInterceptor( .source(searchSourceBuilder) /* If the response shard index is a rollup index, need to find the minimum value of all the live indices to compute the overlap + This is because I am comparing this index to all the live data to compute the interval I want to keep If the response shard index is a live index, need to only compute minimum value of the current shard index */ if (isShardIndexRollup) { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index d7c15e460..85f28a9a9 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -196,7 +196,10 @@ class RollupInterceptor( // Only modifies rollup searches and avoids internal client calls if (containsRollup || isRollupIndex) { val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) - // Avoid infinite interceptor loop + /* Avoid infinite interceptor loop: + if there is an internal client call made in the reponse interceptor there is only 1 index. + Therefore, conditions are not met for api to combine rollup and live data + */ val isMultiSearch = (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty()) if (isMultiSearch && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts From 79f68e51c1a3f99dfcde031704c145700c0db2f5 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 15:57:48 -0700 Subject: [PATCH 33/44] removed stars from workflows Signed-off-by: Ronnak Saxena --- .github/workflows/docker-security-test-workflow.yml | 2 +- .github/workflows/security-test-workflow.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docker-security-test-workflow.yml b/.github/workflows/docker-security-test-workflow.yml index 9ca19f548..6ff563f5b 100644 --- a/.github/workflows/docker-security-test-workflow.yml +++ b/.github/workflows/docker-security-test-workflow.yml @@ -2,7 +2,7 @@ name: Docker Security Test Workflow on: pull_request: branches: - - "**" + - "*" push: branches: - "*" diff --git a/.github/workflows/security-test-workflow.yml b/.github/workflows/security-test-workflow.yml index e8d34fcc7..71bfea68e 100644 --- a/.github/workflows/security-test-workflow.yml +++ b/.github/workflows/security-test-workflow.yml @@ -3,7 +3,7 @@ name: Security test workflow on: pull_request: branches: - - "**" + - "*" push: branches: - "*" From 768a2bce05d76fa56f92f318331ef8ba349c45d2 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 16:45:28 -0700 Subject: [PATCH 34/44] testing time of alias test case Signed-off-by: Ronnak Saxena --- .github/workflows/multi-node-test-workflow.yml | 2 +- .../rollup/interceptor/ResponseInterceptorIT.kt | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index baa186175..c6206d4ca 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" + run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" -Dtests.method="test search multiple live data indices and a rollup data index with overlap" -PcustomDistributionUrl="https://artifacts.opensearch.org/snapshots/core/opensearch/2.9.0-SNAPSHOT/opensearch-min-2.9.0-SNAPSHOT-darwin-x64-latest.tar.gz" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index c6ea7a0ce..f87780443 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -720,6 +720,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] refreshAllIndices() // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data + val start = System.currentTimeMillis() var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) assertTrue(searchAllResponse.restStatus() == RestStatus.OK) var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> @@ -748,5 +749,7 @@ class ResponseInterceptorIT : RollupRestTestCase() { expectedAvg, responseAggs.getValue("avg_passenger_count")["value"] ) + val elapsedTimeMs = System.currentTimeMillis() - start + assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) } } From fb867583fe8126b2764ef4d126cb6f38d7de356d Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 7 Sep 2023 17:03:36 -0700 Subject: [PATCH 35/44] boring, took too long Signed-off-by: Ronnak Saxena --- .github/workflows/multi-node-test-workflow.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index c6206d4ca..2c9738a98 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" -Dtests.method="test search multiple live data indices and a rollup data index with overlap" -PcustomDistributionUrl="https://artifacts.opensearch.org/snapshots/core/opensearch/2.9.0-SNAPSHOT/opensearch-min-2.9.0-SNAPSHOT-darwin-x64-latest.tar.gz" + run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" -Dtests.method="test search multiple live data indices and a rollup data index with overlap" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() From 17df15622b05c75a8d3b2683e96fca2cd66bb6c5 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Fri, 8 Sep 2023 11:24:25 -0700 Subject: [PATCH 36/44] commented out last 2 tests Signed-off-by: Ronnak Saxena --- .../workflows/multi-node-test-workflow.yml | 2 +- .../interceptor/ResponseInterceptorIT.kt | 600 +++++++++--------- 2 files changed, 301 insertions(+), 301 deletions(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index 2c9738a98..baa186175 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" -Dtests.method="test search multiple live data indices and a rollup data index with overlap" + run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index f87780443..25671ee9f 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -452,304 +452,304 @@ class ResponseInterceptorIT : RollupRestTestCase() { responseAggs.getValue("avg_passenger_count")["value"] ) } - fun `test search multiple live data indices and a rollup data index with overlap`() { - generateNYCTaxiData("source_rollup_search_multi_index_case") - val rollup = Rollup( - id = "case3_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search_multi_index_case", - targetIndex = "target_rollup_search_multi_index_case", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - - refreshAllIndices() - // Split data at 1546304400000 or Jan 01 2019 01:00:00 - // Delete half the values from live data simulating an ism job deleting old data - var r = """ - { - "query": { - "range": { - "tpep_pickup_datetime": { - "lt": 1546304400000, - "format": "epoch_millis", - "time_zone": "+00:00" - } - } - } - } - """.trimIndent() - var deleteLiveResponse = client().makeRequest( - "POST", - "source_rollup_search_multi_index_case/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity(r, ContentType.APPLICATION_JSON) - ) - - assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) - - // Insert more live data - generateNYCTaxiData("source_rollup_search_multi_index_case2") - // Expected values would discard the overlapping rollup index completely - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - - // Search all 3 indices to check if overlap was removed - var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchAllResponse.restStatus() == RestStatus.OK) - var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - } - fun `test search aliased live indices data and rollup data`() { - /* add later */ - // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 - generateNYCTaxiData("nyc-taxi-data-1") - generateNYCTaxiData("nyc-taxi-data-2") - generateNYCTaxiData("nyc-taxi-data-3") - // Add them to alias nyc-taxi-data - val createAliasReq = """ - { - "actions": [ - { - "add": { - "index": "nyc-taxi-data-1", - "alias": "nyc-taxi-data" - } - }, - { - "add": { - "index": "nyc-taxi-data-2", - "alias": "nyc-taxi-data" - } - }, - { - "add": { - "index": "nyc-taxi-data-3", - "alias": "nyc-taxi-data" - } - } - ] - } - """.trimIndent() - val createAliasRes = client().makeRequest( - "POST", - "_aliases", - mapOf(), - StringEntity(createAliasReq, ContentType.APPLICATION_JSON) - ) - assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) - // Rollup alias into rollup-nyc-taxi-data - val rollup = Rollup( - id = "alias_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "nyc-taxi-data", - targetIndex = "rollup-nyc-taxi-data", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - refreshAllIndices() - // Find expected values by searching nyc-taxi-data - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data - val start = System.currentTimeMillis() - var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchAllResponse.restStatus() == RestStatus.OK) - var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - val elapsedTimeMs = System.currentTimeMillis() - start - assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) - } +// fun `test search multiple live data indices and a rollup data index with overlap`() { +// generateNYCTaxiData("source_rollup_search_multi_index_case") +// val rollup = Rollup( +// id = "case3_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "source_rollup_search_multi_index_case", +// targetIndex = "target_rollup_search_multi_index_case", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// +// refreshAllIndices() +// // Split data at 1546304400000 or Jan 01 2019 01:00:00 +// // Delete half the values from live data simulating an ism job deleting old data +// var r = """ +// { +// "query": { +// "range": { +// "tpep_pickup_datetime": { +// "lt": 1546304400000, +// "format": "epoch_millis", +// "time_zone": "+00:00" +// } +// } +// } +// } +// """.trimIndent() +// var deleteLiveResponse = client().makeRequest( +// "POST", +// "source_rollup_search_multi_index_case/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity(r, ContentType.APPLICATION_JSON) +// ) +// +// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) +// +// // Insert more live data +// generateNYCTaxiData("source_rollup_search_multi_index_case2") +// // Expected values would discard the overlapping rollup index completely +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// +// // Search all 3 indices to check if overlap was removed +// var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// } +// fun `test search aliased live indices data and rollup data`() { +// /* add later */ +// // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 +// generateNYCTaxiData("nyc-taxi-data-1") +// generateNYCTaxiData("nyc-taxi-data-2") +// generateNYCTaxiData("nyc-taxi-data-3") +// // Add them to alias nyc-taxi-data +// val createAliasReq = """ +// { +// "actions": [ +// { +// "add": { +// "index": "nyc-taxi-data-1", +// "alias": "nyc-taxi-data" +// } +// }, +// { +// "add": { +// "index": "nyc-taxi-data-2", +// "alias": "nyc-taxi-data" +// } +// }, +// { +// "add": { +// "index": "nyc-taxi-data-3", +// "alias": "nyc-taxi-data" +// } +// } +// ] +// } +// """.trimIndent() +// val createAliasRes = client().makeRequest( +// "POST", +// "_aliases", +// mapOf(), +// StringEntity(createAliasReq, ContentType.APPLICATION_JSON) +// ) +// assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) +// // Rollup alias into rollup-nyc-taxi-data +// val rollup = Rollup( +// id = "alias_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "nyc-taxi-data", +// targetIndex = "rollup-nyc-taxi-data", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// refreshAllIndices() +// // Find expected values by searching nyc-taxi-data +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data +// val start = System.currentTimeMillis() +// var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// val elapsedTimeMs = System.currentTimeMillis() - start +// assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) +// } } From e496c5713529c4ef98272449d71e31632e2f9675 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Fri, 8 Sep 2023 14:04:16 -0700 Subject: [PATCH 37/44] removed all response interceptor tests Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 886 +++++++++--------- 1 file changed, 443 insertions(+), 443 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 25671ee9f..e8e50db58 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -5,453 +5,453 @@ package org.opensearch.indexmanagement.rollup.interceptor -import org.apache.http.entity.ContentType -import org.apache.http.entity.StringEntity -import org.opensearch.indexmanagement.common.model.dimension.DateHistogram -import org.opensearch.indexmanagement.common.model.dimension.Terms -import org.opensearch.indexmanagement.makeRequest +// import org.apache.http.entity.ContentType +// import org.apache.http.entity.StringEntity +// import org.opensearch.indexmanagement.common.model.dimension.DateHistogram +// import org.opensearch.indexmanagement.common.model.dimension.Terms +// import org.opensearch.indexmanagement.makeRequest import org.opensearch.indexmanagement.rollup.RollupRestTestCase -import org.opensearch.indexmanagement.rollup.model.Rollup -import org.opensearch.indexmanagement.rollup.model.RollupMetadata -import org.opensearch.indexmanagement.rollup.model.RollupMetrics -import org.opensearch.indexmanagement.rollup.model.metric.Average -import org.opensearch.indexmanagement.rollup.model.metric.Max -import org.opensearch.indexmanagement.rollup.model.metric.Min -import org.opensearch.indexmanagement.rollup.model.metric.Sum -import org.opensearch.indexmanagement.rollup.model.metric.ValueCount -import org.opensearch.indexmanagement.waitFor -import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule -import org.opensearch.rest.RestStatus -import java.time.Instant -import java.time.temporal.ChronoUnit +// import org.opensearch.indexmanagement.rollup.model.Rollup +// import org.opensearch.indexmanagement.rollup.model.RollupMetadata +// import org.opensearch.indexmanagement.rollup.model.RollupMetrics +// import org.opensearch.indexmanagement.rollup.model.metric.Average +// import org.opensearch.indexmanagement.rollup.model.metric.Max +// import org.opensearch.indexmanagement.rollup.model.metric.Min +// import org.opensearch.indexmanagement.rollup.model.metric.Sum +// import org.opensearch.indexmanagement.rollup.model.metric.ValueCount +// import org.opensearch.indexmanagement.waitFor +// import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule +// import org.opensearch.rest.RestStatus +// import java.time.Instant +// import java.time.temporal.ChronoUnit @Suppress("UNCHECKED_CAST") class ResponseInterceptorIT : RollupRestTestCase() { - fun `test search a live index and rollup index with no overlap`() { - generateNYCTaxiData("source_rollup_search") - val rollup = Rollup( - id = "base_case1_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search", - targetIndex = "target_rollup_search", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - // Get expected aggregation values by searching live data before deletion - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - // Split data at 1546304400000 or Jan 01 2019 01:00:00 - // Delete half the values from live data simulating an ism job deleting old data - var r = """ - { - "query": { - "range": { - "tpep_pickup_datetime": { - "lt": 1546304400000, - "format": "epoch_millis", - "time_zone": "+00:00" - } - } - } - } - """.trimIndent() - var deleteLiveResponse = client().makeRequest( - "POST", - "source_rollup_search/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity(r, ContentType.APPLICATION_JSON) - ) - - assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) - - // Delete half the values from rollup data - r = """ - { - "query": { - "range": { - "tpep_pickup_datetime": { - "gte": 1546304400000, - "format": "epoch_millis", - "time_zone": "+00:00" - } - } - } - } - """.trimIndent() - var deleteRollupResponse = client().makeRequest( - "POST", - "target_rollup_search/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity(r, ContentType.APPLICATION_JSON) - ) - - assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) - var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchBothResponse.restStatus() == RestStatus.OK) - var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - } - // Edge Case - fun `test search a live index with no data and rollup index with data`() { - generateNYCTaxiData("source_rollup_search_no_data_case") - val rollup = Rollup( - id = "base_case_2_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search_no_data_case", - targetIndex = "target_rollup_search_no_data_case", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - - refreshAllIndices() - // Get expected aggregation values by searching live data before deletion - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - // Delete values from live index - var deleteResponse = client().makeRequest( - "POST", - "source_rollup_search_no_data_case/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) - ) - assertTrue(deleteResponse.restStatus() == RestStatus.OK) - var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_no_data_case,source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchBothResponse.restStatus() == RestStatus.OK) - var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - } - fun `test search a live index and rollup index with data overlap`() { - generateNYCTaxiData("source_rollup_search_data_overlap_case") - val rollup = Rollup( - id = "case2_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search_data_overlap_case", - targetIndex = "target_rollup_search_data_overlap_case", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - // Get expected aggregation values by searching live data before deletion - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) - var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - - refreshAllIndices() - // Split data at 1546304400000 or Jan 01 2019 01:00:00 - // Delete half the values from live data simulating an ism job deleting old data - var r = """ - { - "query": { - "range": { - "tpep_pickup_datetime": { - "lt": 1546304400000, - "format": "epoch_millis", - "time_zone": "+00:00" - } - } - } - } - """.trimIndent() - var deleteLiveResponse = client().makeRequest( - "POST", - "source_rollup_search_data_overlap_case/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity(r, ContentType.APPLICATION_JSON) - ) - - assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) - // Rollup index is complete overlap of live data - // Search both and check if time series data is the same - var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_data_overlap_case,source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchBothResponse.restStatus() == RestStatus.OK) - var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - } +// fun `test search a live index and rollup index with no overlap`() { +// generateNYCTaxiData("source_rollup_search") +// val rollup = Rollup( +// id = "base_case1_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "source_rollup_search", +// targetIndex = "target_rollup_search", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// // Get expected aggregation values by searching live data before deletion +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// // Split data at 1546304400000 or Jan 01 2019 01:00:00 +// // Delete half the values from live data simulating an ism job deleting old data +// var r = """ +// { +// "query": { +// "range": { +// "tpep_pickup_datetime": { +// "lt": 1546304400000, +// "format": "epoch_millis", +// "time_zone": "+00:00" +// } +// } +// } +// } +// """.trimIndent() +// var deleteLiveResponse = client().makeRequest( +// "POST", +// "source_rollup_search/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity(r, ContentType.APPLICATION_JSON) +// ) +// +// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) +// +// // Delete half the values from rollup data +// r = """ +// { +// "query": { +// "range": { +// "tpep_pickup_datetime": { +// "gte": 1546304400000, +// "format": "epoch_millis", +// "time_zone": "+00:00" +// } +// } +// } +// } +// """.trimIndent() +// var deleteRollupResponse = client().makeRequest( +// "POST", +// "target_rollup_search/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity(r, ContentType.APPLICATION_JSON) +// ) +// +// assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) +// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// } +// // Edge Case +// fun `test search a live index with no data and rollup index with data`() { +// generateNYCTaxiData("source_rollup_search_no_data_case") +// val rollup = Rollup( +// id = "base_case_2_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "source_rollup_search_no_data_case", +// targetIndex = "target_rollup_search_no_data_case", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// +// refreshAllIndices() +// // Get expected aggregation values by searching live data before deletion +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// // Delete values from live index +// var deleteResponse = client().makeRequest( +// "POST", +// "source_rollup_search_no_data_case/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) +// ) +// assertTrue(deleteResponse.restStatus() == RestStatus.OK) +// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_no_data_case,source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// } +// fun `test search a live index and rollup index with data overlap`() { +// generateNYCTaxiData("source_rollup_search_data_overlap_case") +// val rollup = Rollup( +// id = "case2_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "source_rollup_search_data_overlap_case", +// targetIndex = "target_rollup_search_data_overlap_case", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// // Get expected aggregation values by searching live data before deletion +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// +// refreshAllIndices() +// // Split data at 1546304400000 or Jan 01 2019 01:00:00 +// // Delete half the values from live data simulating an ism job deleting old data +// var r = """ +// { +// "query": { +// "range": { +// "tpep_pickup_datetime": { +// "lt": 1546304400000, +// "format": "epoch_millis", +// "time_zone": "+00:00" +// } +// } +// } +// } +// """.trimIndent() +// var deleteLiveResponse = client().makeRequest( +// "POST", +// "source_rollup_search_data_overlap_case/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity(r, ContentType.APPLICATION_JSON) +// ) +// +// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) +// // Rollup index is complete overlap of live data +// // Search both and check if time series data is the same +// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_data_overlap_case,source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// } // fun `test search multiple live data indices and a rollup data index with overlap`() { // generateNYCTaxiData("source_rollup_search_multi_index_case") // val rollup = Rollup( From 8aa3f81344804bbdd356a064f41f38f1ffbb1740 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Fri, 8 Sep 2023 14:28:22 -0700 Subject: [PATCH 38/44] added one test back Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 356 +++++++++--------- 1 file changed, 178 insertions(+), 178 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index e8e50db58..77ec0753d 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -5,188 +5,188 @@ package org.opensearch.indexmanagement.rollup.interceptor -// import org.apache.http.entity.ContentType -// import org.apache.http.entity.StringEntity -// import org.opensearch.indexmanagement.common.model.dimension.DateHistogram -// import org.opensearch.indexmanagement.common.model.dimension.Terms -// import org.opensearch.indexmanagement.makeRequest +import org.apache.http.entity.ContentType +import org.apache.http.entity.StringEntity +import org.opensearch.indexmanagement.common.model.dimension.DateHistogram +import org.opensearch.indexmanagement.common.model.dimension.Terms +import org.opensearch.indexmanagement.makeRequest import org.opensearch.indexmanagement.rollup.RollupRestTestCase -// import org.opensearch.indexmanagement.rollup.model.Rollup -// import org.opensearch.indexmanagement.rollup.model.RollupMetadata -// import org.opensearch.indexmanagement.rollup.model.RollupMetrics -// import org.opensearch.indexmanagement.rollup.model.metric.Average -// import org.opensearch.indexmanagement.rollup.model.metric.Max -// import org.opensearch.indexmanagement.rollup.model.metric.Min -// import org.opensearch.indexmanagement.rollup.model.metric.Sum -// import org.opensearch.indexmanagement.rollup.model.metric.ValueCount -// import org.opensearch.indexmanagement.waitFor -// import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule -// import org.opensearch.rest.RestStatus -// import java.time.Instant -// import java.time.temporal.ChronoUnit +import org.opensearch.indexmanagement.rollup.model.Rollup +import org.opensearch.indexmanagement.rollup.model.RollupMetadata +import org.opensearch.indexmanagement.rollup.model.RollupMetrics +import org.opensearch.indexmanagement.rollup.model.metric.Average +import org.opensearch.indexmanagement.rollup.model.metric.Max +import org.opensearch.indexmanagement.rollup.model.metric.Min +import org.opensearch.indexmanagement.rollup.model.metric.Sum +import org.opensearch.indexmanagement.rollup.model.metric.ValueCount +import org.opensearch.indexmanagement.waitFor +import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule +import org.opensearch.rest.RestStatus +import java.time.Instant +import java.time.temporal.ChronoUnit @Suppress("UNCHECKED_CAST") class ResponseInterceptorIT : RollupRestTestCase() { -// fun `test search a live index and rollup index with no overlap`() { -// generateNYCTaxiData("source_rollup_search") -// val rollup = Rollup( -// id = "base_case1_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "source_rollup_search", -// targetIndex = "target_rollup_search", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// // Get expected aggregation values by searching live data before deletion -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// // Split data at 1546304400000 or Jan 01 2019 01:00:00 -// // Delete half the values from live data simulating an ism job deleting old data -// var r = """ -// { -// "query": { -// "range": { -// "tpep_pickup_datetime": { -// "lt": 1546304400000, -// "format": "epoch_millis", -// "time_zone": "+00:00" -// } -// } -// } -// } -// """.trimIndent() -// var deleteLiveResponse = client().makeRequest( -// "POST", -// "source_rollup_search/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity(r, ContentType.APPLICATION_JSON) -// ) -// -// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) -// -// // Delete half the values from rollup data -// r = """ -// { -// "query": { -// "range": { -// "tpep_pickup_datetime": { -// "gte": 1546304400000, -// "format": "epoch_millis", -// "time_zone": "+00:00" -// } -// } -// } -// } -// """.trimIndent() -// var deleteRollupResponse = client().makeRequest( -// "POST", -// "target_rollup_search/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity(r, ContentType.APPLICATION_JSON) -// ) -// -// assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) -// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// } + fun `test search a live index and rollup index with no overlap`() { + generateNYCTaxiData("source_rollup_search") + val rollup = Rollup( + id = "base_case1_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search", + targetIndex = "target_rollup_search", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + // Get expected aggregation values by searching live data before deletion + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + + // Delete half the values from rollup data + r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "gte": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteRollupResponse = client().makeRequest( + "POST", + "target_rollup_search/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete rollup data", deleteRollupResponse.restStatus() == RestStatus.OK) + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search,source_rollup_search/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } // // Edge Case // fun `test search a live index with no data and rollup index with data`() { // generateNYCTaxiData("source_rollup_search_no_data_case") From 016d1618a15deffb4f6a8186c07ca93ef2c30b91 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Mon, 11 Sep 2023 15:07:48 -0700 Subject: [PATCH 39/44] fixed data stream integ tests Signed-off-by: Ronnak Saxena --- .../workflows/multi-node-test-workflow.yml | 2 +- .../rollup/interceptor/ResponseInterceptor.kt | 4 +- .../rollup/interceptor/RollupInterceptor.kt | 12 +- .../interceptor/ResponseInterceptorIT.kt | 1128 ++++++++--------- 4 files changed, 575 insertions(+), 571 deletions(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index baa186175..9b548ee14 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest -Dtests.class="org.opensearch.indexmanagement.rollup.interceptor.ResponseInterceptorIT" + run: ./gradlew integTest - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 27fabd310..eb0a4e9db 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -56,7 +56,6 @@ class ResponseInterceptor( val client: Client ) : TransportInterceptor { private val logger = LogManager.getLogger(javaClass) - override fun interceptSender(sender: TransportInterceptor.AsyncSender): TransportInterceptor.AsyncSender { return CustomAsyncSender(sender) } @@ -195,6 +194,7 @@ class ResponseInterceptor( .indices(*rollupIndices) // add all rollup indices to this request var maxRolledDateResponse: SearchResponse? = null var latch = CountDownLatch(1) + logger.info("ronsax sending request to find max rollup time for index: $shardRequestIndex") client.search( maxRolledDateRequest, object : ActionListener { @@ -230,6 +230,7 @@ class ResponseInterceptor( var minLiveDateResponse: SearchResponse? = null latch = CountDownLatch(1) + logger.info("ronsax sending request to find minLiveData for index: $shardRequestIndex") client.search( minLiveDateRequest, object : ActionListener { @@ -256,6 +257,7 @@ class ResponseInterceptor( // If intersection found on rollup index, remove overlap if ((liveDataStartPoint < rollupDataEndPoint) && isShardIndexRollup) { // Start at 0, end at live data + logger.info("ronsax sending request to find rollup endtime for index: $shardRequestIndex") val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) return Pair(0L, endTime) } diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 85f28a9a9..4f591cd3a 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -77,7 +77,7 @@ class RollupInterceptor( } // Returns Pair @Suppress("SpreadOperator") - private fun originalSearchContainsRollup(request: ShardSearchRequest): Pair { + private fun originalSearchContainsRollup(request: ShardSearchRequest): Pair { // Throwing an error on data streams val indices = request.indices().map { it.toString() }.toTypedArray() val allIndices = indexNameExpressionResolver .concreteIndexNames(clusterService.state(), request.indicesOptions(), *indices) @@ -169,7 +169,7 @@ class RollupInterceptor( } // Wraps all existing aggs in bucket aggregation // Notifies the response interceptor that was rewritten since agg name is interceptor_interval_data - // Edge case if User selected this as the aggregation name :/ + // Edge case if User selected interceptor_interval_data as the aggregation name :/ val intervalAgg = AggregationBuilders.dateHistogram("interceptor_interval_data") .field(dateSourceField) .calendarInterval(DateHistogramInterval(rollupInterval)) @@ -190,14 +190,16 @@ class RollupInterceptor( return object : TransportRequestHandler { override fun messageReceived(request: T, channel: TransportChannel, task: Task) { if (searchEnabled && request is ShardSearchRequest) { - val (containsRollup, rollupJob) = originalSearchContainsRollup(request) + val isDataStream = (request.indices().any { IndexUtils.isDataStream(it, clusterService.state()) }) val shardRequestIndex = request.shardId().indexName - val isRollupIndex = isRollupIndex(shardRequestIndex, clusterService.state()) + // isRollupIndex throws an exception if the request is on a data stream + val isRollupIndex = if (isDataStream) false else isRollupIndex(shardRequestIndex, clusterService.state()) + val (containsRollup, rollupJob) = if (isDataStream) Pair(false, null) else originalSearchContainsRollup(request) // Only modifies rollup searches and avoids internal client calls if (containsRollup || isRollupIndex) { val (concreteRollupIndicesArray, concreteLiveIndicesArray) = getConcreteIndices(request) /* Avoid infinite interceptor loop: - if there is an internal client call made in the reponse interceptor there is only 1 index. + if there is an internal client call made in the response interceptor there is only 1 index. Therefore, conditions are not met for api to combine rollup and live data */ val isMultiSearch = (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty()) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 77ec0753d..ad98c58a2 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -188,568 +188,568 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) } // // Edge Case -// fun `test search a live index with no data and rollup index with data`() { -// generateNYCTaxiData("source_rollup_search_no_data_case") -// val rollup = Rollup( -// id = "base_case_2_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "source_rollup_search_no_data_case", -// targetIndex = "target_rollup_search_no_data_case", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// -// refreshAllIndices() -// // Get expected aggregation values by searching live data before deletion -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// // Delete values from live index -// var deleteResponse = client().makeRequest( -// "POST", -// "source_rollup_search_no_data_case/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) -// ) -// assertTrue(deleteResponse.restStatus() == RestStatus.OK) -// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_no_data_case,source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// } -// fun `test search a live index and rollup index with data overlap`() { -// generateNYCTaxiData("source_rollup_search_data_overlap_case") -// val rollup = Rollup( -// id = "case2_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "source_rollup_search_data_overlap_case", -// targetIndex = "target_rollup_search_data_overlap_case", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// // Get expected aggregation values by searching live data before deletion -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// -// refreshAllIndices() -// // Split data at 1546304400000 or Jan 01 2019 01:00:00 -// // Delete half the values from live data simulating an ism job deleting old data -// var r = """ -// { -// "query": { -// "range": { -// "tpep_pickup_datetime": { -// "lt": 1546304400000, -// "format": "epoch_millis", -// "time_zone": "+00:00" -// } -// } -// } -// } -// """.trimIndent() -// var deleteLiveResponse = client().makeRequest( -// "POST", -// "source_rollup_search_data_overlap_case/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity(r, ContentType.APPLICATION_JSON) -// ) -// -// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) -// // Rollup index is complete overlap of live data -// // Search both and check if time series data is the same -// var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_data_overlap_case,source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchBothResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// } -// fun `test search multiple live data indices and a rollup data index with overlap`() { -// generateNYCTaxiData("source_rollup_search_multi_index_case") -// val rollup = Rollup( -// id = "case3_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "source_rollup_search_multi_index_case", -// targetIndex = "target_rollup_search_multi_index_case", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// -// refreshAllIndices() -// // Split data at 1546304400000 or Jan 01 2019 01:00:00 -// // Delete half the values from live data simulating an ism job deleting old data -// var r = """ -// { -// "query": { -// "range": { -// "tpep_pickup_datetime": { -// "lt": 1546304400000, -// "format": "epoch_millis", -// "time_zone": "+00:00" -// } -// } -// } -// } -// """.trimIndent() -// var deleteLiveResponse = client().makeRequest( -// "POST", -// "source_rollup_search_multi_index_case/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity(r, ContentType.APPLICATION_JSON) -// ) -// -// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) -// -// // Insert more live data -// generateNYCTaxiData("source_rollup_search_multi_index_case2") -// // Expected values would discard the overlapping rollup index completely -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// -// // Search all 3 indices to check if overlap was removed -// var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// } -// fun `test search aliased live indices data and rollup data`() { -// /* add later */ -// // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 -// generateNYCTaxiData("nyc-taxi-data-1") -// generateNYCTaxiData("nyc-taxi-data-2") -// generateNYCTaxiData("nyc-taxi-data-3") -// // Add them to alias nyc-taxi-data -// val createAliasReq = """ -// { -// "actions": [ -// { -// "add": { -// "index": "nyc-taxi-data-1", -// "alias": "nyc-taxi-data" -// } -// }, -// { -// "add": { -// "index": "nyc-taxi-data-2", -// "alias": "nyc-taxi-data" -// } -// }, -// { -// "add": { -// "index": "nyc-taxi-data-3", -// "alias": "nyc-taxi-data" -// } -// } -// ] -// } -// """.trimIndent() -// val createAliasRes = client().makeRequest( -// "POST", -// "_aliases", -// mapOf(), -// StringEntity(createAliasReq, ContentType.APPLICATION_JSON) -// ) -// assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) -// // Rollup alias into rollup-nyc-taxi-data -// val rollup = Rollup( -// id = "alias_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "nyc-taxi-data", -// targetIndex = "rollup-nyc-taxi-data", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// refreshAllIndices() -// // Find expected values by searching nyc-taxi-data -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data -// val start = System.currentTimeMillis() -// var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// val elapsedTimeMs = System.currentTimeMillis() - start -// assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) -// } + fun `test search a live index with no data and rollup index with data`() { + generateNYCTaxiData("source_rollup_search_no_data_case") + val rollup = Rollup( + id = "base_case_2_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search_no_data_case", + targetIndex = "target_rollup_search_no_data_case", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + // Get expected aggregation values by searching live data before deletion + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Delete values from live index + var deleteResponse = client().makeRequest( + "POST", + "source_rollup_search_no_data_case/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity("""{"query": {"match_all": {}}}""", ContentType.APPLICATION_JSON) + ) + assertTrue(deleteResponse.restStatus() == RestStatus.OK) + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_no_data_case,source_rollup_search_no_data_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } + fun `test search a live index and rollup index with data overlap`() { + generateNYCTaxiData("source_rollup_search_data_overlap_case") + val rollup = Rollup( + id = "case2_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search_data_overlap_case", + targetIndex = "target_rollup_search_data_overlap_case", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + // Get expected aggregation values by searching live data before deletion + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var expectedSearchResponse = client().makeRequest("POST", "/source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search inital data for expected values", expectedSearchResponse.restStatus() == RestStatus.OK) + var expectedAggs = expectedSearchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search_data_overlap_case/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + // Rollup index is complete overlap of live data + // Search both and check if time series data is the same + var searchBothResponse = client().makeRequest("POST", "/target_rollup_search_data_overlap_case,source_rollup_search_data_overlap_case/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchBothResponse.restStatus() == RestStatus.OK) + var responseAggs = searchBothResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } + fun `test search multiple live data indices and a rollup data index with overlap`() { + generateNYCTaxiData("source_rollup_search_multi_index_case") + val rollup = Rollup( + id = "case3_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search_multi_index_case", + targetIndex = "target_rollup_search_multi_index_case", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search_multi_index_case/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + + // Insert more live data + generateNYCTaxiData("source_rollup_search_multi_index_case2") + // Expected values would discard the overlapping rollup index completely + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + + // Search all 3 indices to check if overlap was removed + var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } + fun `test search aliased live indices data and rollup data`() { + /* add later */ + // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 + generateNYCTaxiData("nyc-taxi-data-1") + generateNYCTaxiData("nyc-taxi-data-2") + generateNYCTaxiData("nyc-taxi-data-3") + // Add them to alias nyc-taxi-data + val createAliasReq = """ + { + "actions": [ + { + "add": { + "index": "nyc-taxi-data-1", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-2", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-3", + "alias": "nyc-taxi-data" + } + } + ] + } + """.trimIndent() + val createAliasRes = client().makeRequest( + "POST", + "_aliases", + mapOf(), + StringEntity(createAliasReq, ContentType.APPLICATION_JSON) + ) + assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) + // Rollup alias into rollup-nyc-taxi-data + val rollup = Rollup( + id = "alias_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "nyc-taxi-data", + targetIndex = "rollup-nyc-taxi-data", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + refreshAllIndices() + // Find expected values by searching nyc-taxi-data + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data + val start = System.currentTimeMillis() + var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + val elapsedTimeMs = System.currentTimeMillis() - start + assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) + } } From 96603ebc639ef93e52b6de35005d1a1f798a6374 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Mon, 11 Sep 2023 16:13:46 -0700 Subject: [PATCH 40/44] commented out breaking tests Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 601 +++++++++--------- 1 file changed, 301 insertions(+), 300 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index ad98c58a2..884d33bdd 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -452,304 +452,305 @@ class ResponseInterceptorIT : RollupRestTestCase() { responseAggs.getValue("avg_passenger_count")["value"] ) } - fun `test search multiple live data indices and a rollup data index with overlap`() { - generateNYCTaxiData("source_rollup_search_multi_index_case") - val rollup = Rollup( - id = "case3_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "source_rollup_search_multi_index_case", - targetIndex = "target_rollup_search_multi_index_case", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - - refreshAllIndices() - // Split data at 1546304400000 or Jan 01 2019 01:00:00 - // Delete half the values from live data simulating an ism job deleting old data - var r = """ - { - "query": { - "range": { - "tpep_pickup_datetime": { - "lt": 1546304400000, - "format": "epoch_millis", - "time_zone": "+00:00" - } - } - } - } - """.trimIndent() - var deleteLiveResponse = client().makeRequest( - "POST", - "source_rollup_search_multi_index_case/_delete_by_query", - mapOf("refresh" to "true"), - StringEntity(r, ContentType.APPLICATION_JSON) - ) - - assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) - - // Insert more live data - generateNYCTaxiData("source_rollup_search_multi_index_case2") - // Expected values would discard the overlapping rollup index completely - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - - // Search all 3 indices to check if overlap was removed - var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchAllResponse.restStatus() == RestStatus.OK) - var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - } - fun `test search aliased live indices data and rollup data`() { - /* add later */ - // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 - generateNYCTaxiData("nyc-taxi-data-1") - generateNYCTaxiData("nyc-taxi-data-2") - generateNYCTaxiData("nyc-taxi-data-3") - // Add them to alias nyc-taxi-data - val createAliasReq = """ - { - "actions": [ - { - "add": { - "index": "nyc-taxi-data-1", - "alias": "nyc-taxi-data" - } - }, - { - "add": { - "index": "nyc-taxi-data-2", - "alias": "nyc-taxi-data" - } - }, - { - "add": { - "index": "nyc-taxi-data-3", - "alias": "nyc-taxi-data" - } - } - ] - } - """.trimIndent() - val createAliasRes = client().makeRequest( - "POST", - "_aliases", - mapOf(), - StringEntity(createAliasReq, ContentType.APPLICATION_JSON) - ) - assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) - // Rollup alias into rollup-nyc-taxi-data - val rollup = Rollup( - id = "alias_rollup_search", - enabled = true, - schemaVersion = 1L, - jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), - jobLastUpdatedTime = Instant.now(), - jobEnabledTime = Instant.now(), - description = "basic search test", - sourceIndex = "nyc-taxi-data", - targetIndex = "rollup-nyc-taxi-data", - metadataID = null, - roles = emptyList(), - pageSize = 10, - delay = 0, - continuous = false, - dimensions = listOf( - DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), - Terms("passenger_count", "passenger_count") - ), - metrics = listOf( - RollupMetrics( - sourceField = "passenger_count", targetField = "passenger_count", - metrics = listOf( - Sum(), Min(), Max(), - ValueCount(), Average() - ) - ) - ) - ).let { createRollup(it, it.id) } - - updateRollupStartTime(rollup) - - waitFor { - val rollupJob = getRollup(rollupId = rollup.id) - assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) - val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) - assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) - } - refreshAllIndices() - // Find expected values by searching nyc-taxi-data - var aggReq = """ - { - "size": 0, - "query": { - "match_all": {} - }, - "aggs": { - "sum_passenger_count": { - "sum": { - "field": "passenger_count" - } - }, - "max_passenger_count": { - "max": { - "field": "passenger_count" - } - }, - "min_passenger_count": { - "min": { - "field": "passenger_count" - } - }, - "avg_passenger_count": { - "avg": { - "field": "passenger_count" - } - }, - "count_passenger_count": { - "value_count": { - "field": "passenger_count" - } - } - } - } - """.trimIndent() - var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) - var expectedAggs = searchResponse.asMap()["aggregations"] as Map> - val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] - val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] - val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] - val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] - val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] - refreshAllIndices() - // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data - val start = System.currentTimeMillis() - var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) - assertTrue(searchAllResponse.restStatus() == RestStatus.OK) - var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> - assertEquals( - "sum agg is wrong", - expectedSum, - responseAggs.getValue("sum_passenger_count")["value"] - ) - assertEquals( - "max agg is wrong", - expectedMax, - responseAggs.getValue("max_passenger_count")["value"] - ) - assertEquals( - "min agg is wrong", - expectedMin, - responseAggs.getValue("min_passenger_count")["value"] - ) - assertEquals( - "value_count is wrong", - expectedCount, - responseAggs.getValue("count_passenger_count")["value"] - ) - assertEquals( - "avg is wrong", - expectedAvg, - responseAggs.getValue("avg_passenger_count")["value"] - ) - val elapsedTimeMs = System.currentTimeMillis() - start - assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) - } + // Breaking subsequent tests but works when executed one its own +// fun `test search multiple live data indices and a rollup data index with overlap`() { +// generateNYCTaxiData("source_rollup_search_multi_index_case") +// val rollup = Rollup( +// id = "case3_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "source_rollup_search_multi_index_case", +// targetIndex = "target_rollup_search_multi_index_case", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// +// refreshAllIndices() +// // Split data at 1546304400000 or Jan 01 2019 01:00:00 +// // Delete half the values from live data simulating an ism job deleting old data +// var r = """ +// { +// "query": { +// "range": { +// "tpep_pickup_datetime": { +// "lt": 1546304400000, +// "format": "epoch_millis", +// "time_zone": "+00:00" +// } +// } +// } +// } +// """.trimIndent() +// var deleteLiveResponse = client().makeRequest( +// "POST", +// "source_rollup_search_multi_index_case/_delete_by_query", +// mapOf("refresh" to "true"), +// StringEntity(r, ContentType.APPLICATION_JSON) +// ) +// +// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) +// +// // Insert more live data +// generateNYCTaxiData("source_rollup_search_multi_index_case2") +// // Expected values would discard the overlapping rollup index completely +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// +// // Search all 3 indices to check if overlap was removed +// var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// } +// fun `test search aliased live indices data and rollup data`() { +// /* add later */ +// // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 +// generateNYCTaxiData("nyc-taxi-data-1") +// generateNYCTaxiData("nyc-taxi-data-2") +// generateNYCTaxiData("nyc-taxi-data-3") +// // Add them to alias nyc-taxi-data +// val createAliasReq = """ +// { +// "actions": [ +// { +// "add": { +// "index": "nyc-taxi-data-1", +// "alias": "nyc-taxi-data" +// } +// }, +// { +// "add": { +// "index": "nyc-taxi-data-2", +// "alias": "nyc-taxi-data" +// } +// }, +// { +// "add": { +// "index": "nyc-taxi-data-3", +// "alias": "nyc-taxi-data" +// } +// } +// ] +// } +// """.trimIndent() +// val createAliasRes = client().makeRequest( +// "POST", +// "_aliases", +// mapOf(), +// StringEntity(createAliasReq, ContentType.APPLICATION_JSON) +// ) +// assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) +// // Rollup alias into rollup-nyc-taxi-data +// val rollup = Rollup( +// id = "alias_rollup_search", +// enabled = true, +// schemaVersion = 1L, +// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), +// jobLastUpdatedTime = Instant.now(), +// jobEnabledTime = Instant.now(), +// description = "basic search test", +// sourceIndex = "nyc-taxi-data", +// targetIndex = "rollup-nyc-taxi-data", +// metadataID = null, +// roles = emptyList(), +// pageSize = 10, +// delay = 0, +// continuous = false, +// dimensions = listOf( +// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), +// Terms("passenger_count", "passenger_count") +// ), +// metrics = listOf( +// RollupMetrics( +// sourceField = "passenger_count", targetField = "passenger_count", +// metrics = listOf( +// Sum(), Min(), Max(), +// ValueCount(), Average() +// ) +// ) +// ) +// ).let { createRollup(it, it.id) } +// +// updateRollupStartTime(rollup) +// +// waitFor { +// val rollupJob = getRollup(rollupId = rollup.id) +// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) +// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) +// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) +// } +// refreshAllIndices() +// // Find expected values by searching nyc-taxi-data +// var aggReq = """ +// { +// "size": 0, +// "query": { +// "match_all": {} +// }, +// "aggs": { +// "sum_passenger_count": { +// "sum": { +// "field": "passenger_count" +// } +// }, +// "max_passenger_count": { +// "max": { +// "field": "passenger_count" +// } +// }, +// "min_passenger_count": { +// "min": { +// "field": "passenger_count" +// } +// }, +// "avg_passenger_count": { +// "avg": { +// "field": "passenger_count" +// } +// }, +// "count_passenger_count": { +// "value_count": { +// "field": "passenger_count" +// } +// } +// } +// } +// """.trimIndent() +// var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) +// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> +// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] +// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] +// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] +// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] +// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] +// refreshAllIndices() +// // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data +// val start = System.currentTimeMillis() +// var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) +// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) +// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> +// assertEquals( +// "sum agg is wrong", +// expectedSum, +// responseAggs.getValue("sum_passenger_count")["value"] +// ) +// assertEquals( +// "max agg is wrong", +// expectedMax, +// responseAggs.getValue("max_passenger_count")["value"] +// ) +// assertEquals( +// "min agg is wrong", +// expectedMin, +// responseAggs.getValue("min_passenger_count")["value"] +// ) +// assertEquals( +// "value_count is wrong", +// expectedCount, +// responseAggs.getValue("count_passenger_count")["value"] +// ) +// assertEquals( +// "avg is wrong", +// expectedAvg, +// responseAggs.getValue("avg_passenger_count")["value"] +// ) +// val elapsedTimeMs = System.currentTimeMillis() - start +// assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) +// } } From 35ee9e6a7826db26d78e489b411cb40ab60ac230 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Mon, 11 Sep 2023 16:27:50 -0700 Subject: [PATCH 41/44] added tests back Signed-off-by: Ronnak Saxena --- .../interceptor/ResponseInterceptorIT.kt | 597 +++++++++--------- 1 file changed, 297 insertions(+), 300 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt index 884d33bdd..cda9f8dfb 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptorIT.kt @@ -453,304 +453,301 @@ class ResponseInterceptorIT : RollupRestTestCase() { ) } // Breaking subsequent tests but works when executed one its own -// fun `test search multiple live data indices and a rollup data index with overlap`() { -// generateNYCTaxiData("source_rollup_search_multi_index_case") -// val rollup = Rollup( -// id = "case3_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "source_rollup_search_multi_index_case", -// targetIndex = "target_rollup_search_multi_index_case", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// -// refreshAllIndices() -// // Split data at 1546304400000 or Jan 01 2019 01:00:00 -// // Delete half the values from live data simulating an ism job deleting old data -// var r = """ -// { -// "query": { -// "range": { -// "tpep_pickup_datetime": { -// "lt": 1546304400000, -// "format": "epoch_millis", -// "time_zone": "+00:00" -// } -// } -// } -// } -// """.trimIndent() -// var deleteLiveResponse = client().makeRequest( -// "POST", -// "source_rollup_search_multi_index_case/_delete_by_query", -// mapOf("refresh" to "true"), -// StringEntity(r, ContentType.APPLICATION_JSON) -// ) -// -// assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) -// -// // Insert more live data -// generateNYCTaxiData("source_rollup_search_multi_index_case2") -// // Expected values would discard the overlapping rollup index completely -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// -// // Search all 3 indices to check if overlap was removed -// var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// } -// fun `test search aliased live indices data and rollup data`() { -// /* add later */ -// // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 -// generateNYCTaxiData("nyc-taxi-data-1") -// generateNYCTaxiData("nyc-taxi-data-2") -// generateNYCTaxiData("nyc-taxi-data-3") -// // Add them to alias nyc-taxi-data -// val createAliasReq = """ -// { -// "actions": [ -// { -// "add": { -// "index": "nyc-taxi-data-1", -// "alias": "nyc-taxi-data" -// } -// }, -// { -// "add": { -// "index": "nyc-taxi-data-2", -// "alias": "nyc-taxi-data" -// } -// }, -// { -// "add": { -// "index": "nyc-taxi-data-3", -// "alias": "nyc-taxi-data" -// } -// } -// ] -// } -// """.trimIndent() -// val createAliasRes = client().makeRequest( -// "POST", -// "_aliases", -// mapOf(), -// StringEntity(createAliasReq, ContentType.APPLICATION_JSON) -// ) -// assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) -// // Rollup alias into rollup-nyc-taxi-data -// val rollup = Rollup( -// id = "alias_rollup_search", -// enabled = true, -// schemaVersion = 1L, -// jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), -// jobLastUpdatedTime = Instant.now(), -// jobEnabledTime = Instant.now(), -// description = "basic search test", -// sourceIndex = "nyc-taxi-data", -// targetIndex = "rollup-nyc-taxi-data", -// metadataID = null, -// roles = emptyList(), -// pageSize = 10, -// delay = 0, -// continuous = false, -// dimensions = listOf( -// DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), -// Terms("passenger_count", "passenger_count") -// ), -// metrics = listOf( -// RollupMetrics( -// sourceField = "passenger_count", targetField = "passenger_count", -// metrics = listOf( -// Sum(), Min(), Max(), -// ValueCount(), Average() -// ) -// ) -// ) -// ).let { createRollup(it, it.id) } -// -// updateRollupStartTime(rollup) -// -// waitFor { -// val rollupJob = getRollup(rollupId = rollup.id) -// assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) -// val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) -// assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) -// } -// refreshAllIndices() -// // Find expected values by searching nyc-taxi-data -// var aggReq = """ -// { -// "size": 0, -// "query": { -// "match_all": {} -// }, -// "aggs": { -// "sum_passenger_count": { -// "sum": { -// "field": "passenger_count" -// } -// }, -// "max_passenger_count": { -// "max": { -// "field": "passenger_count" -// } -// }, -// "min_passenger_count": { -// "min": { -// "field": "passenger_count" -// } -// }, -// "avg_passenger_count": { -// "avg": { -// "field": "passenger_count" -// } -// }, -// "count_passenger_count": { -// "value_count": { -// "field": "passenger_count" -// } -// } -// } -// } -// """.trimIndent() -// var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) -// var expectedAggs = searchResponse.asMap()["aggregations"] as Map> -// val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] -// val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] -// val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] -// val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] -// val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] -// refreshAllIndices() -// // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data -// val start = System.currentTimeMillis() -// var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) -// assertTrue(searchAllResponse.restStatus() == RestStatus.OK) -// var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> -// assertEquals( -// "sum agg is wrong", -// expectedSum, -// responseAggs.getValue("sum_passenger_count")["value"] -// ) -// assertEquals( -// "max agg is wrong", -// expectedMax, -// responseAggs.getValue("max_passenger_count")["value"] -// ) -// assertEquals( -// "min agg is wrong", -// expectedMin, -// responseAggs.getValue("min_passenger_count")["value"] -// ) -// assertEquals( -// "value_count is wrong", -// expectedCount, -// responseAggs.getValue("count_passenger_count")["value"] -// ) -// assertEquals( -// "avg is wrong", -// expectedAvg, -// responseAggs.getValue("avg_passenger_count")["value"] -// ) -// val elapsedTimeMs = System.currentTimeMillis() - start -// assertEquals("ronsax search reqeust took $elapsedTimeMs ms", true, false) -// } + fun `test search multiple live data indices and a rollup data index with overlap`() { + generateNYCTaxiData("source_rollup_search_multi_index_case") + val rollup = Rollup( + id = "case3_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "source_rollup_search_multi_index_case", + targetIndex = "target_rollup_search_multi_index_case", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + // Split data at 1546304400000 or Jan 01 2019 01:00:00 + // Delete half the values from live data simulating an ism job deleting old data + var r = """ + { + "query": { + "range": { + "tpep_pickup_datetime": { + "lt": 1546304400000, + "format": "epoch_millis", + "time_zone": "+00:00" + } + } + } + } + """.trimIndent() + var deleteLiveResponse = client().makeRequest( + "POST", + "source_rollup_search_multi_index_case/_delete_by_query", + mapOf("refresh" to "true"), + StringEntity(r, ContentType.APPLICATION_JSON) + ) + + assertTrue("Could not delete live data", deleteLiveResponse.restStatus() == RestStatus.OK) + + // Insert more live data + generateNYCTaxiData("source_rollup_search_multi_index_case2") + // Expected values would discard the overlapping rollup index completely + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + + // Search all 3 indices to check if overlap was removed + var searchAllResponse = client().makeRequest("POST", "/target_rollup_search_multi_index_case,source_rollup_search_multi_index_case,source_rollup_search_multi_index_case2/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } + fun `test search aliased live indices data and rollup data`() { + /* add later */ + // Create 3 indices with 5,000 docs each nyc-taxi-data-1, nyc-taxi-data-2, nyc-taxi-data-3 + generateNYCTaxiData("nyc-taxi-data-1") + generateNYCTaxiData("nyc-taxi-data-2") + generateNYCTaxiData("nyc-taxi-data-3") + // Add them to alias nyc-taxi-data + val createAliasReq = """ + { + "actions": [ + { + "add": { + "index": "nyc-taxi-data-1", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-2", + "alias": "nyc-taxi-data" + } + }, + { + "add": { + "index": "nyc-taxi-data-3", + "alias": "nyc-taxi-data" + } + } + ] + } + """.trimIndent() + val createAliasRes = client().makeRequest( + "POST", + "_aliases", + mapOf(), + StringEntity(createAliasReq, ContentType.APPLICATION_JSON) + ) + assertTrue("Could not create alias", createAliasRes.restStatus() == RestStatus.OK) + // Rollup alias into rollup-nyc-taxi-data + val rollup = Rollup( + id = "alias_rollup_search", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "basic search test", + sourceIndex = "nyc-taxi-data", + targetIndex = "rollup-nyc-taxi-data", + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("passenger_count", "passenger_count") + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf( + Sum(), Min(), Max(), + ValueCount(), Average() + ) + ) + ) + ).let { createRollup(it, it.id) } + + updateRollupStartTime(rollup) + + waitFor { + val rollupJob = getRollup(rollupId = rollup.id) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + refreshAllIndices() + // Find expected values by searching nyc-taxi-data + var aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { + "field": "passenger_count" + } + }, + "max_passenger_count": { + "max": { + "field": "passenger_count" + } + }, + "min_passenger_count": { + "min": { + "field": "passenger_count" + } + }, + "avg_passenger_count": { + "avg": { + "field": "passenger_count" + } + }, + "count_passenger_count": { + "value_count": { + "field": "passenger_count" + } + } + } + } + """.trimIndent() + var searchResponse = client().makeRequest("POST", "/nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Could not search initial data for expected values", searchResponse.restStatus() == RestStatus.OK) + var expectedAggs = searchResponse.asMap()["aggregations"] as Map> + val expectedSum = expectedAggs.getValue("sum_passenger_count")["value"] + val expectedMax = expectedAggs.getValue("max_passenger_count")["value"] + val expectedMin = expectedAggs.getValue("min_passenger_count")["value"] + val expectedCount = expectedAggs.getValue("count_passenger_count")["value"] + val expectedAvg = expectedAggs.getValue("avg_passenger_count")["value"] + refreshAllIndices() + // Validate result from searching rollup-nyc-taxi-data, searching nyc-taxi-data + var searchAllResponse = client().makeRequest("POST", "/rollup-nyc-taxi-data,nyc-taxi-data/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue(searchAllResponse.restStatus() == RestStatus.OK) + var responseAggs = searchAllResponse.asMap()["aggregations"] as Map> + assertEquals( + "sum agg is wrong", + expectedSum, + responseAggs.getValue("sum_passenger_count")["value"] + ) + assertEquals( + "max agg is wrong", + expectedMax, + responseAggs.getValue("max_passenger_count")["value"] + ) + assertEquals( + "min agg is wrong", + expectedMin, + responseAggs.getValue("min_passenger_count")["value"] + ) + assertEquals( + "value_count is wrong", + expectedCount, + responseAggs.getValue("count_passenger_count")["value"] + ) + assertEquals( + "avg is wrong", + expectedAvg, + responseAggs.getValue("avg_passenger_count")["value"] + ) + } } From 06b7b028491d4037b6f558c66a0c7f55abb0dbe7 Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Tue, 12 Sep 2023 13:29:33 -0700 Subject: [PATCH 42/44] removed countdown latch and added coroutines Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 100 +++++------------- 1 file changed, 29 insertions(+), 71 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index eb0a4e9db..19b6bccdc 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -5,8 +5,12 @@ package org.opensearch.indexmanagement.rollup.interceptor +import kotlinx.coroutines.CoroutineName +import kotlinx.coroutines.CoroutineScope +import kotlinx.coroutines.Dispatchers +import kotlinx.coroutines.SupervisorJob +import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener import org.opensearch.action.search.SearchRequest import org.opensearch.action.search.SearchResponse import org.opensearch.client.Client @@ -24,6 +28,7 @@ import org.opensearch.search.builder.SearchSourceBuilder import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.search.query.QuerySearchResult import org.opensearch.index.query.QueryBuilders +import org.opensearch.indexmanagement.opensearchapi.suspendUntil import org.opensearch.indexmanagement.rollup.util.convertDateStringToEpochMillis import org.opensearch.indexmanagement.rollup.util.convertFixedIntervalStringToMs import org.opensearch.indexmanagement.rollup.util.getRollupJobs @@ -45,7 +50,6 @@ import org.opensearch.transport.TransportRequestOptions import org.opensearch.transport.TransportResponse import org.opensearch.transport.TransportResponseHandler import java.time.ZonedDateTime -import java.util.concurrent.CountDownLatch import kotlin.math.max import kotlin.math.min @@ -54,7 +58,8 @@ class ResponseInterceptor( val settings: Settings, val indexNameExpressionResolver: IndexNameExpressionResolver, val client: Client -) : TransportInterceptor { +) : TransportInterceptor, + CoroutineScope by CoroutineScope(SupervisorJob() + Dispatchers.Default + CoroutineName("Rollup Response Interceptor")) { private val logger = LogManager.getLogger(javaClass) override fun interceptSender(sender: TransportInterceptor.AsyncSender): TransportInterceptor.AsyncSender { return CustomAsyncSender(sender) @@ -123,7 +128,7 @@ class ResponseInterceptor( // Calculated the end time for the current shard index if it is a rollup index with data overlapp @Suppress("SpreadOperator") - fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { + suspend fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { // Build search request to find the maximum rollup timestamp <= liveDataStartPoint val sort = SortBuilders.fieldSort("$dateTargetField.date_histogram").order(SortOrder.DESC) val query = QueryBuilders.boolQuery() @@ -136,23 +141,7 @@ class ResponseInterceptor( val req = SearchRequest() .source(searchSourceBuilder) .indices(*rollupIndices) - var res: SearchResponse? = null - val latch = CountDownLatch(1) - client.search( - req, - object : ActionListener { - override fun onResponse(searchResponse: SearchResponse) { - res = searchResponse - latch.countDown() - } - - override fun onFailure(e: Exception) { - logger.error("request to find intersection time failed :(", e) - latch.countDown() - } - } - ) - latch.await() + val res = client.suspendUntil { search(req, it) } try { return res!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long } catch (e: Exception) { @@ -164,7 +153,7 @@ class ResponseInterceptor( // Returns Pair(startRange: Long, endRange: Long) // Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds @Suppress("LongMethod", "SpreadOperator") - fun findOverlap(response: QuerySearchResult): Pair { + suspend fun findOverlap(response: QuerySearchResult): Pair { val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later var dateSourceField: String = "" var dateTargetField: String = "" @@ -189,27 +178,11 @@ class ResponseInterceptor( .query(oldQuery) .size(1) // Need to avoid infinite interceptor loop - val maxRolledDateRequest = SearchRequest() + val maxRollupDateRequest = SearchRequest() .source(searchSourceBuilder) .indices(*rollupIndices) // add all rollup indices to this request - var maxRolledDateResponse: SearchResponse? = null - var latch = CountDownLatch(1) - logger.info("ronsax sending request to find max rollup time for index: $shardRequestIndex") - client.search( - maxRolledDateRequest, - object : ActionListener { - override fun onResponse(searchResponse: SearchResponse) { - maxRolledDateResponse = searchResponse - latch.countDown() - } - - override fun onFailure(e: Exception) { - logger.error("maxLiveDate request failed in response interceptor", e) - latch.countDown() - } - } - ) - latch.await() + logger.info("Sending maxRollupDate request for $shardRequestIndex") + val maxRollupDateResponse: SearchResponse? = client.suspendUntil { search(maxRollupDateRequest, it) } // Build search request to find the minimum date in all live indices sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) searchSourceBuilder = SearchSourceBuilder() @@ -227,32 +200,15 @@ class ResponseInterceptor( } else { // shard index is live index minLiveDateRequest.indices(shardRequestIndex) } - - var minLiveDateResponse: SearchResponse? = null - latch = CountDownLatch(1) - logger.info("ronsax sending request to find minLiveData for index: $shardRequestIndex") - client.search( - minLiveDateRequest, - object : ActionListener { - override fun onResponse(searchResponse: SearchResponse) { - minLiveDateResponse = searchResponse - latch.countDown() - } - - override fun onFailure(e: Exception) { - logger.error("minLiveDate request failed in response interceptor", e) - latch.countDown() - } - } - ) - latch.await() - val foundMinAndMax = (minLiveDateResponse != null && maxRolledDateResponse != null) + logger.info("Sending minLiveData request for $shardRequestIndex") + var minLiveDateResponse: SearchResponse? = client.suspendUntil { search(minLiveDateRequest, it) } + val foundMinAndMax = (minLiveDateResponse != null && maxRollupDateResponse != null) // if they overlap find part to exclude - if (foundMinAndMax && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRolledDateResponse!!.hits.hits.isNotEmpty()) { + if (foundMinAndMax && minLiveDateResponse!!.hits.hits.isNotEmpty() && maxRollupDateResponse!!.hits.hits.isNotEmpty()) { // Rollup data ends at maxRolledDate + fixedInterval - val maxRolledDate: Long = maxRolledDateResponse!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long + val maxRolledDate: Long = maxRollupDateResponse.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long val rollupDataEndPoint = maxRolledDate + convertFixedIntervalStringToMs(fixedInterval = rollupInterval!!) - val minLiveDate = minLiveDateResponse!!.hits.hits[0].sourceAsMap.get("$dateSourceField") as String + val minLiveDate = minLiveDateResponse.hits.hits[0].sourceAsMap.get("$dateSourceField") as String val liveDataStartPoint = convertDateStringToEpochMillis(minLiveDate) // If intersection found on rollup index, remove overlap if ((liveDataStartPoint < rollupDataEndPoint) && isShardIndexRollup) { @@ -355,8 +311,8 @@ class ResponseInterceptor( // Store the running values of the aggregations being computed // {aggName: String: Pair} val aggValues = mutableMapOf>() -// -// // Iterate through each aggregation and bucket + + // Iterate through each aggregation and bucket val interceptorAgg = intervalAggregations.asMap().get("interceptor_interval_data") as InternalDateHistogram for (bucket in interceptorAgg.buckets) { val zdt = bucket.key as ZonedDateTime @@ -409,11 +365,13 @@ class ResponseInterceptor( // live index is QuerySearchResult -> { if (response.hasAggs() && isRewrittenInterceptorRequest(response)) { - // Check for overlap - val (startTime, endTime) = findOverlap(response) - // Modify agg to be original result without overlap computed in - response.aggregations(computeAggregationsWithoutOverlap(response.aggregations().expand(), startTime, endTime)) - originalHandler?.handleResponse(response) + launch { + // Check for overlap + val (startTime, endTime) = findOverlap(response) + // Modify agg to be original result without overlap computed in + response.aggregations(computeAggregationsWithoutOverlap(response.aggregations().expand(), startTime, endTime)) + originalHandler?.handleResponse(response) + } } else { originalHandler?.handleResponse(response) } From acf1c4b769a8d367cfa850e88d61bd9ce593081e Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Wed, 13 Sep 2023 14:36:28 -0700 Subject: [PATCH 43/44] resolved comments on the PR Signed-off-by: Ronnak Saxena --- .../workflows/multi-node-test-workflow.yml | 2 +- .../rollup/interceptor/ResponseInterceptor.kt | 40 +++++++++++++------ 2 files changed, 28 insertions(+), 14 deletions(-) diff --git a/.github/workflows/multi-node-test-workflow.yml b/.github/workflows/multi-node-test-workflow.yml index 9b548ee14..0bb068e78 100644 --- a/.github/workflows/multi-node-test-workflow.yml +++ b/.github/workflows/multi-node-test-workflow.yml @@ -22,7 +22,7 @@ jobs: - name: Checkout Branch uses: actions/checkout@v2 - name: Run integration tests with multi node config - run: ./gradlew integTest + run: ./gradlew integTest -PnumNodes=3 - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 19b6bccdc..881210c95 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -53,6 +53,13 @@ import java.time.ZonedDateTime import kotlin.math.max import kotlin.math.min +/** +* The Response Interceptor class modifies resopnses if the search API is triggered on rollup and live data +* 1. The class check if the request was rewritten in the RollupInterceptor into more granular buckets +* 2. in findOverlap it checks for overlap between live data and rollup data and returns the interval to include +* 3. computeAggregationsWithoutOverlap() iterates through the buckets and recomputes the aggregations in the expected format +* 4. Returns a new response for each shard to be combined later + **/ class ResponseInterceptor( val clusterService: ClusterService, val settings: Settings, @@ -126,7 +133,9 @@ class ResponseInterceptor( return Pair(rollupIndices.toTypedArray(), liveIndices.toTypedArray()) } - // Calculated the end time for the current shard index if it is a rollup index with data overlapp + /** + * Calculates the end time for the current shard index if it is a rollup index with data overlapp + **/ @Suppress("SpreadOperator") suspend fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { // Build search request to find the maximum rollup timestamp <= liveDataStartPoint @@ -145,16 +154,19 @@ class ResponseInterceptor( try { return res!!.hits.hits[0].sourceAsMap.get("$dateTargetField.date_histogram") as Long } catch (e: Exception) { - logger.error("Not able to retrieve intersection time from response: $e") + logger.error("Not able to retrieve intersection time from response: ", e) } return 0L // dummy :P } - -// Returns Pair(startRange: Long, endRange: Long) -// Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds + /** + * Returns Pair(startRange: Long, endRange: Long) + * Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds + **/ + // TODO intercept at the index level instead of the shard level to avoid redundant client calls for every index @Suppress("LongMethod", "SpreadOperator") suspend fun findOverlap(response: QuerySearchResult): Pair { - val job: Rollup = getRollupJob(response)!! // maybe throw a try catch later + // TODO add more error logs and try catch statements for client calls + val job: Rollup = getRollupJob(response)!! var dateSourceField: String = "" var dateTargetField: String = "" var rollupInterval: String? = "" @@ -181,7 +193,7 @@ class ResponseInterceptor( val maxRollupDateRequest = SearchRequest() .source(searchSourceBuilder) .indices(*rollupIndices) // add all rollup indices to this request - logger.info("Sending maxRollupDate request for $shardRequestIndex") + logger.debug("Sending maxRollupDate request for $shardRequestIndex") val maxRollupDateResponse: SearchResponse? = client.suspendUntil { search(maxRollupDateRequest, it) } // Build search request to find the minimum date in all live indices sort = SortBuilders.fieldSort(dateSourceField).order(SortOrder.ASC) @@ -200,7 +212,7 @@ class ResponseInterceptor( } else { // shard index is live index minLiveDateRequest.indices(shardRequestIndex) } - logger.info("Sending minLiveData request for $shardRequestIndex") + logger.debug("Sending minLiveData request for $shardRequestIndex") var minLiveDateResponse: SearchResponse? = client.suspendUntil { search(minLiveDateRequest, it) } val foundMinAndMax = (minLiveDateResponse != null && maxRollupDateResponse != null) // if they overlap find part to exclude @@ -213,7 +225,7 @@ class ResponseInterceptor( // If intersection found on rollup index, remove overlap if ((liveDataStartPoint < rollupDataEndPoint) && isShardIndexRollup) { // Start at 0, end at live data - logger.info("ronsax sending request to find rollup endtime for index: $shardRequestIndex") + logger.debug("Sending request to find rollup endtime for index: $shardRequestIndex") val endTime = getRollupEndTime(liveDataStartPoint, rollupIndices, dateTargetField) return Pair(0L, endTime) } @@ -222,7 +234,7 @@ class ResponseInterceptor( return Pair(0L, Long.MAX_VALUE) } - // Depending on which metric the aggregation is computer data differently + // Depending on which metric the aggregation is, computation is different @Suppress("ReturnCount") fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any): Pair { when (agg) { @@ -289,7 +301,7 @@ class ResponseInterceptor( } } } else { // Value count calc - // Won't double count + // Put in set to avoid adding the aggregation twice addedAggregations += modifiedName val originalName = modifiedName.removeSuffix(".rollup.avg.value_count") val avgCount = value as Long @@ -305,7 +317,9 @@ class ResponseInterceptor( throw NullPointerException("Can't calculate avg agg for rollup index") } -// Returns a new InternalAggregations that contains merged aggregation(s) with the overlapping data removed + /** + * Returns a new InternalAggregations that contains merged aggregation(s) with the overlapping data removed + */ @Suppress("NestedBlockDepth") fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { // Store the running values of the aggregations being computed @@ -333,7 +347,7 @@ class ResponseInterceptor( } } - // Create a new InternalAggregations with sum values + // Create a new InternalAggregations with recomputed values discarding the overlap val allAggregations = mutableListOf() val addedAggregations = mutableSetOf() // avoid repeating the same aggregations for ((aggName, data) in aggValues) { From 9c9d242d50ab99e73954a65a0fe3f314b5e68f2a Mon Sep 17 00:00:00 2001 From: Ronnak Saxena Date: Thu, 14 Sep 2023 11:03:56 -0700 Subject: [PATCH 44/44] resolved PR comments added kotlin docs for methods Signed-off-by: Ronnak Saxena --- .../rollup/interceptor/ResponseInterceptor.kt | 41 +++++++++--- .../rollup/interceptor/RollupInterceptor.kt | 67 +++++++++++++++---- 2 files changed, 85 insertions(+), 23 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt index 881210c95..608fc3959 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/ResponseInterceptor.kt @@ -55,7 +55,7 @@ import kotlin.math.min /** * The Response Interceptor class modifies resopnses if the search API is triggered on rollup and live data -* 1. The class check if the request was rewritten in the RollupInterceptor into more granular buckets +* 1. The class checks if the request was rewritten in the RollupInterceptor into more granular buckets * 2. in findOverlap it checks for overlap between live data and rollup data and returns the interval to include * 3. computeAggregationsWithoutOverlap() iterates through the buckets and recomputes the aggregations in the expected format * 4. Returns a new response for each shard to be combined later @@ -73,12 +73,12 @@ class ResponseInterceptor( } private inner class CustomAsyncSender(private val originalSender: TransportInterceptor.AsyncSender) : TransportInterceptor.AsyncSender { - override fun sendRequest( - connection: Transport.Connection?, - action: String?, - request: TransportRequest?, - options: TransportRequestOptions?, - handler: TransportResponseHandler? + override fun sendRequest( + connection: Transport.Connection, + action: String, + request: TransportRequest, + options: TransportRequestOptions, + handler: TransportResponseHandler ) { val interceptedHandler = CustomResponseHandler(handler) @@ -94,6 +94,13 @@ class ResponseInterceptor( val response = originalHandler?.read(inStream) return response!! } + + /** + * Check if this response was modified in the request interceptor + * and should be put back together + * @param QuerySearchResult + * @return Boolean + */ fun isRewrittenInterceptorRequest(response: QuerySearchResult): Boolean { val currentAggregations = response.aggregations().expand() for (agg in currentAggregations) { @@ -135,6 +142,8 @@ class ResponseInterceptor( /** * Calculates the end time for the current shard index if it is a rollup index with data overlapp + * @params liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String + * @return Long **/ @Suppress("SpreadOperator") suspend fun getRollupEndTime(liveDataStartPoint: Long, rollupIndices: Array, dateTargetField: String): Long { @@ -159,8 +168,10 @@ class ResponseInterceptor( return 0L // dummy :P } /** - * Returns Pair(startRange: Long, endRange: Long) + * Checks for overlap in timeseries data and returns the non overlapping interval to include * Note startRange is inclusive and endRange is exclusive, they are Longs because the type is epoch milliseconds + * @param QuerySearchResult + * @return Pair(startRange: Long, endRange: Long) **/ // TODO intercept at the index level instead of the shard level to avoid redundant client calls for every index @Suppress("LongMethod", "SpreadOperator") @@ -234,7 +245,11 @@ class ResponseInterceptor( return Pair(0L, Long.MAX_VALUE) } - // Depending on which metric the aggregation is, computation is different + /** + * Depending on which metric the aggregation is, computation is different + * @params agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any + * @return Pair + */ @Suppress("ReturnCount") fun computeRunningValue(agg: org.opensearch.search.aggregations.Aggregation, currentValue: Any): Pair { when (agg) { @@ -279,7 +294,11 @@ class ResponseInterceptor( else -> throw IllegalArgumentException("Could not recreate an aggregation for type $aggType") } } - // Create original avg aggregation + /** + * Create original avg aggregation + * @return InternalAvg + */ + fun initRollupAvgAgg( modifiedName: String, value: Any, @@ -319,6 +338,8 @@ class ResponseInterceptor( /** * Returns a new InternalAggregations that contains merged aggregation(s) with the overlapping data removed + * @params intervalAggregations: InternalAggregations, start: Long, end: Long + * @return InternalAggregations */ @Suppress("NestedBlockDepth") fun computeAggregationsWithoutOverlap(intervalAggregations: InternalAggregations, start: Long, end: Long): InternalAggregations { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt index 4f591cd3a..5cb9a8b46 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -75,7 +75,12 @@ class RollupInterceptor( searchAllJobs = it } } - // Returns Pair + + /** + * Checks if any of the indices in the original request contain a rollup index + * @param ShardSearchRequest + * @return Pair + */ @Suppress("SpreadOperator") private fun originalSearchContainsRollup(request: ShardSearchRequest): Pair { // Throwing an error on data streams val indices = request.indices().map { it.toString() }.toTypedArray() @@ -89,7 +94,11 @@ class RollupInterceptor( } return Pair(false, null) } - // Returns true if request was already modified into "interceptor_interval_data" bucket aggregation + /** + * Returns true if request was already modified into "interceptor_interval_data" bucket aggregation + * @param ShardSearchRequest + * @return Boolean + */ fun isRequestRewrittenIntoBuckets(request: ShardSearchRequest): Boolean { val currentAggs = request.source().aggregations().aggregatorFactories if (currentAggs != null) { @@ -101,8 +110,12 @@ class RollupInterceptor( } return false } - // Helper fn to avoid rewritting a rollup request an extra time - fun isReqeustRollupFormat(request: ShardSearchRequest): Boolean { + /** + * Helper fn to avoid rewritting a rollup request an extra time + * @param ShardSearchRequest + * @return Boolean + */ + fun isRequestRollupFormat(request: ShardSearchRequest): Boolean { if (request.source().query() != null) { val jsonRequest: String = request.source().query().toString() // Detected dummy field from internal search request @@ -112,11 +125,28 @@ class RollupInterceptor( } return false } - // If the request has a sort on it, size can be > 0 on a rollup search - fun canHaveSize(request: ShardSearchRequest): Boolean { + + /** + * If the request has a sort on it, size can be > 0 on a rollup search + * Context: we need to make call to find out the min, max time of rollup and live indexes, + * however, this call would be intercepted here and get re-write. + * + * So the idea is to explicitly allow this call to go through rollup interceptor w/o re-write them, + * so that whether it's from response interceptor or client, both should work. + * @param ShardSearchRequest + * @return Boolean + */ + fun allowRequest(request: ShardSearchRequest): Boolean { return request.source().sorts() != null } - // Need to modify aggs for rollup docs with avg and value count aggs + /** + * Need to modify aggs for rollup docs with avg and value count aggs + * Context in order to recompute the avg metric in the response interceptor + * Need to modify the avg request into a sum and value count request, + * Then ResponseInterceptor puts the aggs back into an avg aggregation + * @param MutableCollection + * @return AggregatorFactories.Builder + */ fun modifyRollupAggs(aggFacts: MutableCollection): AggregatorFactories.Builder { val build = AggregatorFactories.builder() for (agg in aggFacts) { @@ -155,8 +185,12 @@ class RollupInterceptor( return build } - // Wrap original aggregations into buckets based on fixed interval to remove overlap in response interceptor - fun breakRequestIntoBuckets(request: ShardSearchRequest, rollupJob: Rollup) { + /** + * Wrap original aggregations into a bucket aggreagtion based on fixed interval + * to make response more granular and remove overlap in response interceptor + * @params request: ShardSearchRequest, rollupJob: Rollup + */ + fun breakIntoBuckets(request: ShardSearchRequest, rollupJob: Rollup) { val oldAggs = modifyRollupAggs(request.source().aggregations().aggregatorFactories) var dateSourceField: String = "" var rollupInterval: String = "" @@ -205,14 +239,14 @@ class RollupInterceptor( val isMultiSearch = (concreteRollupIndicesArray.isNotEmpty() && concreteLiveIndicesArray.isNotEmpty()) if (isMultiSearch && request.source().aggregations() != null && !isRequestRewrittenIntoBuckets(request)) { // Break apart request to remove overlapping parts - breakRequestIntoBuckets(request, rollupJob!!) + breakIntoBuckets(request, rollupJob!!) } // Rewrite the request to fit rollup format if not already done previously - if (isRollupIndex && !isReqeustRollupFormat(request)) { + if (isRollupIndex && !isRequestRollupFormat(request)) { /* Client calls from the response interceptor require request bodies of 1, otherwise do not allow size > 0 for rollup indices */ - if (!canHaveSize(request) && request.source().size() != 0) { + if (!allowRequest(request) && request.source().size() != 0) { throw IllegalArgumentException( "Rollup search must have size explicitly set to 0, " + "but found ${request.source().size()}" @@ -226,7 +260,10 @@ class RollupInterceptor( } } } - // Returns Pair (concreteRollupIndices: Array, concreteLiveIndicesArray: Array) + + /** + * @return Pair (concreteRollupIndices: Array, concreteLiveIndicesArray: Array) + */ @Suppress("SpreadOperator") fun getConcreteIndices(request: ShardSearchRequest): Pair, Array> { val indices = request.indices().map { it.toString() }.toTypedArray() @@ -245,6 +282,10 @@ class RollupInterceptor( val concreteLiveIndicesArray = concreteLiveIndexNames.toTypedArray() return Pair(concreteRollupIndicesArray, concreteLiveIndicesArray) } + + /** + * Modifies ShardSearchRequest to fit rollup index format + */ fun rewriteRollupRequest(request: ShardSearchRequest, rollupJob: Rollup, concreteRollupIndicesArray: Array) { // To extract fields from QueryStringQueryBuilder we need concrete source index name. val queryFieldMappings = getQueryMetadata(