From 96eb1d605960fa6029fb58625d543bc7ea10d2ec Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Tue, 25 Nov 2025 15:30:28 +0530 Subject: [PATCH 1/6] Adding support for multi-tier rollups in ISM Signed-off-by: Kshitij Tandon --- .../indexmanagement/IndexManagementPlugin.kt | 2 +- .../rollup/RollupMapperService.kt | 7 + .../rollup/RollupMetadataService.kt | 76 ++- .../indexmanagement/rollup/RollupRunner.kt | 93 ++- .../rollup/RollupSearchService.kt | 4 +- .../rollup/interceptor/RollupInterceptor.kt | 58 +- .../rollup/util/RollupUtils.kt | 137 +++- .../action/RollupActionIT.kt | 214 ++++++ .../rollup/RollupMetadataServiceTests.kt | 166 ++++- .../rollup/interceptor/RollupInterceptorIT.kt | 175 +++++ .../interceptor/RollupInterceptorTests.kt | 64 ++ .../rollup/runner/RollupRunnerIT.kt | 646 ++++++++++++++++++ .../rollup/util/RollupUtilsTests.kt | 310 +++++++++ 13 files changed, 1915 insertions(+), 37 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt diff --git a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt index 284f8ab23..5c928f35a 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/IndexManagementPlugin.kt @@ -427,7 +427,7 @@ class IndexManagementPlugin : .registerMapperService(RollupMapperService(client, clusterService, indexNameExpressionResolver)) .registerIndexer(RollupIndexer(settings, clusterService, client)) .registerSearcher(RollupSearchService(settings, clusterService, client)) - .registerMetadataServices(RollupMetadataService(client, xContentRegistry)) + .registerMetadataServices(RollupMetadataService(client, xContentRegistry, clusterService)) .registerConsumers() .registerClusterConfigurationProvider(skipFlag) indexManagementIndices = IndexManagementIndices(settings, client.admin().indices(), clusterService) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMapperService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMapperService.kt index cd58b8856..ee174e67b 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMapperService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMapperService.kt @@ -293,6 +293,13 @@ class RollupMapperService( val indexMappingSource = indexTypeMappings.sourceAsMap + // TODO: Check if this is required or we want to fail in case of rollup on rolled up empty indices +// // If source is a rollup index with no properties (no data rolled up yet), skip field validation +// if (isSourceRollupIndex && !indexMappingSource.containsKey("properties")) { +// logger.info("Source rollup index [$index] has no properties yet, skipping field validation") +// return RollupJobValidationResult.Valid +// } + val issues = mutableSetOf() // Validate source fields in dimensions rollup.dimensions.forEach { dimension -> diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt index 0b68a3548..935202806 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt @@ -36,6 +36,7 @@ import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.model.RollupMetadata import org.opensearch.indexmanagement.rollup.model.RollupStats import org.opensearch.indexmanagement.rollup.util.DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT +import org.opensearch.indexmanagement.rollup.util.isRollupIndex import org.opensearch.indexmanagement.util.NO_ID import org.opensearch.search.aggregations.bucket.composite.InternalComposite import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder @@ -48,7 +49,11 @@ import java.time.Instant // TODO: Wrap client calls in retry for transient failures // Service that handles CRUD operations for rollup metadata @Suppress("TooManyFunctions") -class RollupMetadataService(val client: Client, val xContentRegistry: NamedXContentRegistry) { +class RollupMetadataService( + val client: Client, + val xContentRegistry: NamedXContentRegistry, + val clusterService: org.opensearch.cluster.service.ClusterService, +) { private val logger = LogManager.getLogger(javaClass) // If the job does not have a metadataID then we need to initialize the first metadata @@ -178,6 +183,12 @@ class RollupMetadataService(val client: Client, val xContentRegistry: NamedXCont @Throws(Exception::class) private suspend fun getInitialStartTime(rollup: Rollup): StartingTimeResult { try { + // Check if source is a rollup index and use appropriate method + val isSourceRollupIndex = isRollupIndex(rollup.sourceIndex, clusterService.state()) + if (isSourceRollupIndex) { + // Use min aggregation for rollup indices (RollupInterceptor blocks size > 0) + return getEarliestTimestampFromRollupIndex(rollup) + } // Rollup requires the first dimension to be the date histogram val dateHistogram = rollup.dimensions.first() as DateHistogram val searchSourceBuilder = @@ -219,6 +230,69 @@ class RollupMetadataService(val client: Client, val xContentRegistry: NamedXCont } } + /** + * Get the earliest timestamp from a rollup index by finding the minimum value of the date histogram field. + * This is used to determine the starting point for continuous rollups on rollup indices. + * Uses sort instead of aggregation to avoid rollup interceptor validation. + */ + @Suppress("ReturnCount") + @Throws(Exception::class) + private suspend fun getEarliestTimestampFromRollupIndex(rollup: Rollup): StartingTimeResult { + try { + val dateHistogram = rollup.dimensions.first() as DateHistogram + val dateField = dateHistogram.sourceField + + logger.info("Idhr se ja rha hu mai tumhe kya 2") + + val searchRequest = SearchRequest(rollup.sourceIndex) + .source( + SearchSourceBuilder() + .size(1) + .query(MatchAllQueryBuilder()) + .sort("$dateField.date_histogram", SortOrder.ASC) + .trackTotalHits(false) + .fetchSource(false) + .docValueField("$dateField.date_histogram", DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT), + ) + .allowPartialSearchResults(false) + + // Set BYPASS_SIZE_CHECK to allow size=1 when querying rollup index to get earliest timestamp + // This is needed for continuous rollup initialization on rollup indices (multi-tier rollup) + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.setBypass( + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK, + ) + try { + val response: SearchResponse = client.suspendUntil { search(searchRequest, it) } + + if (response.hits.hits.isEmpty()) { + return StartingTimeResult.NoDocumentsFound + } + + logger.info("Idhr se ja rha hu mai tumhe kya") + + // In rollup indices, date histogram fields are named as "field.date_histogram" + val rollupDateField = "$dateField.date_histogram" + val firstHitTimestampAsString: String = + response.hits.hits.first().field(rollupDateField).getValue() + ?: return StartingTimeResult.NoDocumentsFound + + val formatter = DateFormatter.forPattern(DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT) + val epochMillis = DateFormatters.from(formatter.parse(firstHitTimestampAsString), formatter.locale()).toInstant().toEpochMilli() + return StartingTimeResult.Success(getRoundedTime(epochMillis, dateHistogram)) + } finally { + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.clearBypass() + } + } catch (e: RemoteTransportException) { + val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception + logger.error("Error when getting earliest timestamp from rollup index for rollup [{}]: {}", rollup.id, unwrappedException) + return StartingTimeResult.Failure(unwrappedException) + } catch (e: Exception) { + // TODO: Catching general exceptions for now, can make more granular + logger.error("Error when getting earliest timestamp from rollup index for rollup [{}]: {}", rollup.id, e) + return StartingTimeResult.Failure(e) + } + } + /** * Return time rounded down to the nearest unit of time the interval is based on. * This should map to the equivalent bucket a document with the given timestamp would fall into for the date histogram. diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt index 854355dea..07e01f570 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt @@ -35,6 +35,9 @@ import org.opensearch.indexmanagement.rollup.model.RollupStats import org.opensearch.indexmanagement.rollup.model.incrementStats import org.opensearch.indexmanagement.rollup.model.mergeStats 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.util.acquireLockForScheduledJob import org.opensearch.indexmanagement.util.releaseLockForScheduledJob import org.opensearch.indexmanagement.util.renewLockForScheduledJob @@ -44,6 +47,8 @@ import org.opensearch.jobscheduler.spi.ScheduledJobParameter import org.opensearch.jobscheduler.spi.ScheduledJobRunner import org.opensearch.script.ScriptService import org.opensearch.search.aggregations.bucket.composite.InternalComposite +import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder +import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.client.Client @@ -266,7 +271,16 @@ object RollupRunner : withClosableContext( IndexManagementSecurityContext(job.id, settings, threadPool.threadContext, job.user), ) { - rollupSearchService.executeCompositeSearch(updatableJob, metadata) + // Need to set this bypass as we are already doing the required aggregation re-writing in the composite aggregation + // hence do not need to go through the interceptor for aggregation rewriting + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.setBypass( + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_ROLLUP_SEARCH, + ) + try { + rollupSearchService.executeCompositeSearch(updatableJob, metadata, clusterService) + } finally { + org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.clearBypass() + } } val rollupResult = when (rollupSearchResult) { @@ -412,6 +426,15 @@ object RollupRunner : else -> return@withClosableContext sourceIndexValidationResult } + // Additional validation for rollup-on-rollup scenarios + if (isRollupIndex(job.sourceIndex, clusterService.state())) { + when (val rollupValidationResult = validateRollupOnRollup(job)) { + is RollupJobValidationResult.Valid -> { + } // No action taken when valid + else -> return@withClosableContext rollupValidationResult + } + } + // we validate target index only if there is metadata document in the rollup if (metadata != null) { logger.debug("Attempting to create/validate target index [${job.targetIndex}] for rollup job [${job.id}]") @@ -470,6 +493,74 @@ object RollupRunner : } } } + + @Suppress("ReturnCount") + private fun validateRollupOnRollup(job: Rollup): RollupJobValidationResult { + val sourceRollupJobs = clusterService.state().metadata.index(job.sourceIndex).getRollupJobs() + if (sourceRollupJobs == null) { + return RollupJobValidationResult.Invalid("Source rollup index has no rollup jobs") + } + + val sourceJob = sourceRollupJobs.first() + val targetDateHistogram = job.getDateHistogram() + val sourceDateHistogram = sourceJob.getDateHistogram() + + // Validate interval alignment + val sourceInterval = sourceDateHistogram.fixedInterval ?: sourceDateHistogram.calendarInterval!! + val targetInterval = targetDateHistogram.fixedInterval ?: targetDateHistogram.calendarInterval!! + val intervalValid = validateIntervalAlignment(sourceInterval, targetInterval) + if (!intervalValid) { + return RollupJobValidationResult.Invalid( + "Target interval [$targetInterval] must be an exact multiple of source interval [$sourceInterval]", + ) + } + + // Validate source field compatibility + val sourceDimensionFields = sourceJob.dimensions.map { it.sourceField }.toSet() + val sourceMetricFields = sourceJob.metrics.map { it.sourceField }.toSet() + val targetDimensionFields = job.dimensions.map { it.sourceField }.toSet() + val targetMetricFields = job.metrics.map { it.sourceField }.toSet() + + val invalidDimensionFields = targetDimensionFields - sourceDimensionFields + val invalidMetricFields = targetMetricFields - sourceMetricFields + + return when { + invalidDimensionFields.isNotEmpty() -> RollupJobValidationResult.Invalid( + "Cannot rollup on dimension fields $invalidDimensionFields that don't exist in source rollup", + ) + invalidMetricFields.isNotEmpty() -> RollupJobValidationResult.Invalid( + "Cannot rollup on metric fields $invalidMetricFields that don't exist in source rollup", + ) + else -> { + // Validate metric compatibility + val sourceMetrics = sourceJob.metrics.flatMap { it.metrics }.map { it.type.type }.toSet() + val targetMetrics = job.metrics.flatMap { it.metrics }.map { it.type.type }.toSet() + val unsupportedMetrics = targetMetrics - sourceMetrics + + if (unsupportedMetrics.isNotEmpty()) { + RollupJobValidationResult.Invalid( + "Target rollup requests metrics $unsupportedMetrics that are not available in source rollup", + ) + } else { + RollupJobValidationResult.Valid + } + } + } + } + + private fun validateIntervalAlignment(sourceInterval: String, targetInterval: String): Boolean = try { + val sourceMillis = parseIntervalToMillis(sourceInterval) + val targetMillis = parseIntervalToMillis(targetInterval) + targetMillis % sourceMillis == 0L && targetMillis > sourceMillis + } catch (e: Exception) { + true // Let it through and fail later with better error + } + + private fun parseIntervalToMillis(interval: String): Long = if (DateHistogramAggregationBuilder.DATE_FIELD_UNITS.containsKey(interval)) { + DateHistogramInterval(interval).estimateMillis() + } else { + TimeValue.parseTimeValue(interval, "parseIntervalToMillis").millis + } } sealed class RollupJobResult { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt index 1bb44b1aa..0e68866e5 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt @@ -99,7 +99,7 @@ class RollupSearchService( } @Suppress("CyclomaticComplexMethod") - suspend fun executeCompositeSearch(job: Rollup, metadata: RollupMetadata): RollupSearchResult = try { + suspend fun executeCompositeSearch(job: Rollup, metadata: RollupMetadata, clusterService: ClusterService): RollupSearchResult = try { var retryCount = 0 RollupSearchResult.Success( retrySearchPolicy.retry(logger) { @@ -113,7 +113,7 @@ class RollupSearchService( ) } - val searchRequest = job.copy(pageSize = pageSize).getRollupSearchRequest(metadata) + val searchRequest = job.copy(pageSize = pageSize).getRollupSearchRequest(metadata, clusterService.state()) val cancelTimeoutTimeValue = TimeValue.timeValueMinutes(getCancelAfterTimeInterval(cancelAfterTimeInterval.minutes)) searchRequest.cancelAfterTimeInterval = cancelTimeoutTimeValue 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 72ecd3753..89410c2cc 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -65,6 +65,49 @@ class RollupInterceptor( @Volatile private var searchRawRollupIndices = RollupSettings.ROLLUP_SEARCH_SOURCE_INDICES.get(settings) + companion object { + /** + * Thread-local bypass mechanism for internal operations that need to query rollup indices + * without triggering interceptor validations. + */ + private val bypassInterceptor = ThreadLocal() + + /** + * Bypass level that skips all rollup search validations and rewriting. + * Used when the system needs to query rollup indices directly using composite aggregation + */ + const val BYPASS_ROLLUP_SEARCH = 1 + + /** + * Bypass level that allows non-zero size in rollup searches. + * Used for internal operations like continuous rollup initialization that need to fetch + * actual documents from rollup indices (e.g., getEarliestTimestampFromRollupIndex). + */ + const val BYPASS_SIZE_CHECK = 2 + + /** + * Sets the bypass level for the current thread. + * Must be followed by clearBypass() to avoid leaking the bypass state. + */ + fun setBypass(bypassLevel: Int) { + bypassInterceptor.set(bypassLevel) + } + + /** + * Clears the bypass level for the current thread. + * Should always be called in a finally block after setBypass(). + */ + fun clearBypass() { + bypassInterceptor.remove() + } + + /** + * Gets the current bypass level for the thread. + * Returns 0 if no bypass is set. + */ + fun getBypassLevel(): Int = bypassInterceptor.get() ?: 0 + } + init { clusterService.clusterSettings.addSettingsUpdateConsumer(RollupSettings.ROLLUP_SEARCH_ENABLED) { searchEnabled = it @@ -89,7 +132,20 @@ class RollupInterceptor( val index = request.shardId().indexName val isRollupIndex = isRollupIndex(index, clusterService.state()) if (isRollupIndex) { - if (request.source().size() != 0) { + val bypassLevel = getBypassLevel() + // BYPASS_ROLLUP_SEARCH: Skip all validations and query rewriting + // Used for composite aggregation query which we do for retrieving the buckets + // to write to target index during rollup of rolled up index + if (bypassLevel == BYPASS_ROLLUP_SEARCH) { + actualHandler.messageReceived(request, channel, task) + return + } + + // BYPASS_SIZE_CHECK: Allow non-zero size for internal operations that need to + // fetch documents (e.g., fetching earliest timestamp document for rolled index + // in case of continuous rollup job). Normal rollup searches must have size=0 + // since they should only return aggregations + if (bypassLevel != BYPASS_SIZE_CHECK && request.source().size() != 0) { throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") } 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 dff0456aa..87f8dfe7c 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtils.kt @@ -81,10 +81,19 @@ fun isRollupIndex(index: String, clusterState: ClusterState): Boolean { fun Rollup.isTargetIndexAlias(): Boolean = RollupFieldValueExpressionResolver.indexAliasUtils.isAlias(targetIndex) -fun Rollup.getRollupSearchRequest(metadata: RollupMetadata): SearchRequest { +fun Rollup.getRollupSearchRequest(metadata: RollupMetadata, clusterState: ClusterState): SearchRequest { + // In case of multi-tier rollup the source index will not have raw field stored, so need to handle this for continuous jobs + val isSourceRollupIndex = isRollupIndex(this.sourceIndex, clusterState) + val dateHistogram = this.getDateHistogram() + val dateField = if (isSourceRollupIndex) { + "${dateHistogram.sourceField}.date_histogram" + } else { + dateHistogram.sourceField + } + val query = if (metadata.continuous != null) { - RangeQueryBuilder(this.getDateHistogram().sourceField) + RangeQueryBuilder(dateField) .from(metadata.continuous.nextWindowStartTime.toEpochMilli(), true) .to(metadata.continuous.nextWindowEndTime.toEpochMilli(), false) .format(DATE_FIELD_EPOCH_MILLIS_FORMAT) @@ -95,33 +104,137 @@ fun Rollup.getRollupSearchRequest(metadata: RollupMetadata): SearchRequest { SearchSourceBuilder() .trackTotalHits(false) .size(0) - .aggregation(this.getCompositeAggregationBuilder(metadata.afterKey)) + .aggregation(this.getCompositeAggregationBuilder(metadata.afterKey, clusterState)) .query(query) return SearchRequest(this.sourceIndex) .source(searchSourceBuilder) .allowPartialSearchResults(false) } -@Suppress("CyclomaticComplexMethod", "NestedBlockDepth") -fun Rollup.getCompositeAggregationBuilder(afterKey: Map?): CompositeAggregationBuilder { +/** + * Builds a composite aggregation for rollup jobs, with support for multi-tier rollups + * + * This method handles two scenarios: + * 1. Rolling up raw data from a regular index + * 2. Rolling up pre-aggregated data from a rollup index (multi-tier rollup) + * + * For multi-tier rollups, the source index is itself a rollup index, which means: + * - Dimension fields are stored with type suffixes (e.g., "field.date_histogram", "field.terms") + * - Metric fields are stored as pre-computed aggregations (e.g., "field.sum", "field.min") + * - We need to aggregate the pre-aggregated values to maintain correctness + * + * @param afterKey Optional pagination key for composite aggregation continuation + * @param clusterState Current cluster state used to determine if source is a rollup index + * @return CompositeAggregationBuilder configured for either raw data or rollup-on-rollup aggregation + */ +@Suppress("CyclomaticComplexMethod", "NestedBlockDepth", "LongMethod") +fun Rollup.getCompositeAggregationBuilder(afterKey: Map?, clusterState: ClusterState): CompositeAggregationBuilder { + // Determine if the source index is a rollup index to adjust field references accordingly + val isRollupIndex = isRollupIndex(this.sourceIndex, clusterState) val sources = mutableListOf>() - this.dimensions.forEach { dimension -> sources.add(dimension.toSourceBuilder(appendType = true)) } + + if (isRollupIndex) { + // Multi-tier rollup: Source is a rollup index with pre-aggregated data + // Dimension fields in rollup indices are stored with type suffixes + // Example: "tpep_pickup_datetime" becomes "tpep_pickup_datetime.date_histogram" + this.dimensions.forEach { dimension -> + val sourceBuilder = dimension.toSourceBuilder(appendType = true) + when (dimension) { + is DateHistogram -> { + // Reference the pre-aggregated date histogram field + sourceBuilder.field("${dimension.targetField}.${dimension.type.type}") + } + is Terms -> { + // Reference the pre-aggregated terms field + sourceBuilder.field("${dimension.targetField}.${dimension.type.type}") + } + is Histogram -> { + // Reference the pre-aggregated histogram field + sourceBuilder.field("${dimension.targetField}.${dimension.type.type}") + } + } + sources.add(sourceBuilder) + } + } else { + // Standard rollup: Source is a regular index with raw data + // Use the original source fields directly + this.dimensions.forEach { dimension -> sources.add(dimension.toSourceBuilder(appendType = true)) } + } + return CompositeAggregationBuilder(this.id, sources).size(this.pageSize).also { compositeAgg -> afterKey?.let { compositeAgg.aggregateAfter(it) } + + // Add metric aggregations based on source index type this.metrics.forEach { metric -> val subAggs = metric.metrics.flatMap { agg -> when (agg) { is Average -> { + if (isRollupIndex) { + // Multi-tier rollup: Average is stored as separate sum and value_count fields + // We sum both components to aggregate across multiple rollup documents + // Final average is computed during query time: total_sum / total_value_count + listOf( + SumAggregationBuilder(metric.targetFieldWithType(agg) + ".sum") + .field(metric.targetFieldWithType(agg) + ".sum"), + SumAggregationBuilder(metric.targetFieldWithType(agg) + ".value_count") + .field(metric.targetFieldWithType(agg) + ".value_count"), + ) + } else { + // Standard rollup: Compute sum and count from raw data + listOf( + SumAggregationBuilder(metric.targetFieldWithType(agg) + ".sum") + .field(metric.sourceField), + ValueCountAggregationBuilder(metric.targetFieldWithType(agg) + ".value_count") + .field(metric.sourceField), + ) + } + } + is Sum -> { + // Multi-tier: Sum the pre-computed sum values + // Standard: Sum the raw field values + listOf( + SumAggregationBuilder(metric.targetFieldWithType(agg)) + .field( + if (isRollupIndex) metric.targetFieldWithType(agg) else metric.sourceField, + ), + ) + } + is Max -> { + // Multi-tier: Take max of pre-computed max values (max of maxes) + // Standard: Take max of raw field values + listOf( + MaxAggregationBuilder(metric.targetFieldWithType(agg)) + .field( + if (isRollupIndex) metric.targetFieldWithType(agg) else metric.sourceField, + ), + ) + } + is Min -> { + // Multi-tier: Take min of pre-computed min values (min of mins) + // Standard: Take min of raw field values + listOf( + MinAggregationBuilder(metric.targetFieldWithType(agg)) + .field( + if (isRollupIndex) metric.targetFieldWithType(agg) else metric.sourceField, + ), + ) + } + is ValueCount -> { listOf( - SumAggregationBuilder(metric.targetFieldWithType(agg) + ".sum").field(metric.sourceField), - ValueCountAggregationBuilder(metric.targetFieldWithType(agg) + ".value_count").field(metric.sourceField), + if (isRollupIndex) { + // Multi-tier rollup: Sum the pre-computed value_count fields + // Each rollup document contains a count of raw documents it represents + // Summing these counts gives us the total count of original raw documents + SumAggregationBuilder(metric.targetFieldWithType(agg)) + .field(metric.targetFieldWithType(agg)) + } else { + // Standard rollup: Count the raw documents + ValueCountAggregationBuilder(metric.targetFieldWithType(agg)) + .field(metric.sourceField) + }, ) } - is Sum -> listOf(SumAggregationBuilder(metric.targetFieldWithType(agg)).field(metric.sourceField)) - is Max -> listOf(MaxAggregationBuilder(metric.targetFieldWithType(agg)).field(metric.sourceField)) - is Min -> listOf(MinAggregationBuilder(metric.targetFieldWithType(agg)).field(metric.sourceField)) - is ValueCount -> listOf(ValueCountAggregationBuilder(metric.targetFieldWithType(agg)).field(metric.sourceField)) // This shouldn't be possible as rollup will fail to initialize with an unsupported metric else -> throw IllegalArgumentException("Found unsupported metric aggregation ${agg.type.type}") } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt index b5a101e10..4834f7ddf 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt @@ -31,8 +31,10 @@ import org.opensearch.indexmanagement.rollup.model.metric.Sum import org.opensearch.indexmanagement.rollup.model.metric.ValueCount import org.opensearch.indexmanagement.rollup.toJsonString import org.opensearch.indexmanagement.waitFor +import org.opensearch.rest.RestRequest import java.time.Instant import java.time.temporal.ChronoUnit +import java.util.Collections.emptyMap import java.util.Locale class RollupActionIT : IndexStateManagementRestTestCase() { @@ -545,4 +547,216 @@ class RollupActionIT : IndexStateManagementRestTestCase() { ) } } + + @Suppress("UNCHECKED_CAST") + fun `test multi-tier rollup from raw to 1m to 10m to 1h`() { + val indexName = "${testIndexName}_multi_tier_raw" + val policyID = "${testIndexName}_multi_tier_policy" + val rollup1mTarget = "${testIndexName}_rollup_1m" + val rollup10mTarget = "${testIndexName}_rollup_10m" + val rollup1hTarget = "${testIndexName}_rollup_1h" + + val rollup1m = ISMRollup( + description = "Rollup to 1m interval", + targetIndex = rollup1mTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1m")), + metrics = listOf(RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum(), Min(), Max(), ValueCount(), Average()))), + ) + + val rollup10m = ISMRollup( + description = "Rollup to 10m interval", + targetIndex = rollup10mTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "10m")), + metrics = listOf(RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum(), Min(), Max(), ValueCount(), Average()))), + ) + + val rollup1h = ISMRollup( + description = "Rollup to 1h interval", + targetIndex = rollup1hTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h")), + metrics = listOf(RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum(), Min(), Max(), ValueCount(), Average()))), + ) + + val states = listOf(State("rollup_tier1_1m", listOf(RollupAction(rollup1m, 0)), listOf())) + val policy = Policy( + id = policyID, + description = "Multi-tier rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + + val mapping = "\"properties\": {\"tpep_pickup_datetime\": {\"type\": \"date\"}, \"passenger_count\": {\"type\": \"integer\"}}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = mapping) + + // Index some test data + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"tpep_pickup_datetime":"2021-01-01T00:05:00Z","passenger_count":2}""", + ContentType.APPLICATION_JSON, + ), + ) + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { assertEquals(policyID, getExplainManagedIndexMetaData(indexName).policyID) } + + val rollup1mId = rollup1m.toRollup(indexName).id + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup1mId, indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message"), + ) + } + + updateRollupStartTime(rollup1m.toRollup(indexName)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup1mId) + assertNotNull("Rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup1mId, indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message"), + ) + } + + val policy2ID = "${policyID}_tier2" + val states2 = listOf(State("rollup_tier2_10m", listOf(RollupAction(rollup10m, 0)), listOf())) + val policy2 = Policy( + id = policy2ID, + description = "Tier 2 rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states2[0].name, + states = states2, + ) + createPolicy(policy2, policy2ID) + addPolicyToIndex(rollup1mTarget, policy2ID) + val managedIndexConfig2 = getExistingManagedIndexConfig(rollup1mTarget) + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { assertEquals(policy2ID, getExplainManagedIndexMetaData(rollup1mTarget).policyID) } + + val rollup10mId = rollup10m.toRollup(rollup1mTarget).id + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup10mId, rollup1mTarget), + getExplainManagedIndexMetaData(rollup1mTarget).info?.get("message"), + ) + } + + updateRollupStartTime(rollup10m.toRollup(rollup1mTarget)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup10mId) + assertNotNull("Second tier rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + if (rollupMetadata.status == RollupMetadata.Status.FAILED) { + fail("Second tier rollup failed: ${rollupMetadata.failureReason}") + } + assertEquals("Second tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup10mId, rollup1mTarget), + getExplainManagedIndexMetaData(rollup1mTarget).info?.get("message"), + ) + } + + assertIndexExists(rollup1mTarget) + assertIndexExists(rollup10mTarget) + + val policy3ID = "${policyID}_tier3" + val states3 = listOf(State("rollup_tier3_1h", listOf(RollupAction(rollup1h, 0)), listOf())) + val policy3 = Policy( + id = policy3ID, + description = "Tier 3 rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states3[0].name, + states = states3, + ) + createPolicy(policy3, policy3ID) + addPolicyToIndex(rollup10mTarget, policy3ID) + val managedIndexConfig3 = getExistingManagedIndexConfig(rollup10mTarget) + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { assertEquals(policy3ID, getExplainManagedIndexMetaData(rollup10mTarget).policyID) } + + val rollup1hId = rollup1h.toRollup(rollup10mTarget).id + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup1hId, rollup10mTarget), + getExplainManagedIndexMetaData(rollup10mTarget).info?.get("message"), + ) + } + + updateRollupStartTime(rollup1h.toRollup(rollup10mTarget)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup1hId) + assertNotNull("Third tier rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + if (rollupMetadata.status == RollupMetadata.Status.FAILED) { + fail("Third tier rollup failed: ${rollupMetadata.failureReason}") + } + assertEquals("Third tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup1hId, rollup10mTarget), + getExplainManagedIndexMetaData(rollup10mTarget).info?.get("message"), + ) + } + + assertIndexExists(rollup1hTarget) + + // Validate all the metrics between source and 3rd level rollup index + val aggReq = """ + { + "size": 0, + "query": { "match_all": {} }, + "aggs": { + "sum_passenger": { "sum": { "field": "passenger_count" } }, + "min_passenger": { "min": { "field": "passenger_count" } }, + "max_passenger": { "max": { "field": "passenger_count" } }, + "value_count_passenger": { "value_count": { "field": "passenger_count" } }, + "avg_passenger": { "avg": { "field": "passenger_count" } } + } + } + """.trimIndent() + + val sourceResponse = client().makeRequest(RestRequest.Method.POST.name, "/$indexName/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + val rollupResponse = client().makeRequest(RestRequest.Method.POST.name, "/$rollup1hTarget/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + + val sourceAggs = sourceResponse.asMap()["aggregations"] as Map> + val rollupAggs = rollupResponse.asMap()["aggregations"] as Map> + + assertEquals("Sum should be consistent", sourceAggs["sum_passenger"]!!["value"], rollupAggs["sum_passenger"]!!["value"]) + assertEquals("Min should be consistent", sourceAggs["min_passenger"]!!["value"], rollupAggs["min_passenger"]!!["value"]) + assertEquals("Max should be consistent", sourceAggs["max_passenger"]!!["value"], rollupAggs["max_passenger"]!!["value"]) + assertEquals("Value count should be consistent", sourceAggs["value_count_passenger"]!!["value"], rollupAggs["value_count_passenger"]!!["value"]) + assertEquals("Average should be consistent", sourceAggs["avg_passenger"]!!["value"], rollupAggs["avg_passenger"]!!["value"]) + } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataServiceTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataServiceTests.kt index d52aef893..8d3cdbe12 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataServiceTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataServiceTests.kt @@ -73,7 +73,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-10-02T05:01:00Z") val expectedWindowEndTime = getInstant("2020-10-02T05:02:00Z") @@ -114,7 +114,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-10-02T05:00:00Z") val expectedWindowEndTime = getInstant("2020-10-02T06:00:00Z") @@ -155,7 +155,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = localDateAtTimezone("2020-03-08T01:00:00", ZoneId.of("America/Los_Angeles")) // Should jump to March 3, 2020, at 3AM PST for end time due to DST @@ -197,7 +197,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-10-02T00:00:00Z") val expectedWindowEndTime = getInstant("2020-10-03T00:00:00Z") @@ -238,7 +238,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-02-28T00:00:00Z") val expectedWindowEndTime = getInstant("2020-02-29T00:00:00Z") @@ -279,7 +279,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) // Since Monday is the beginning of the calendar week, the start time will be last Monday // given that the first document timestamp was on Sunday @@ -322,7 +322,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2019-12-01T00:00:00Z") val expectedWindowEndTime = getInstant("2020-01-01T00:00:00Z") @@ -363,7 +363,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-04-01T00:00:00Z") val expectedWindowEndTime = getInstant("2020-07-01T00:00:00Z") @@ -404,7 +404,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-01-01T00:00:00Z") val expectedWindowEndTime = getInstant("2021-01-01T00:00:00Z") @@ -445,7 +445,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = localDateAtTimezone("2020-04-24T08:00:00", ZoneId.of("America/Los_Angeles")) val expectedWindowEndTime = localDateAtTimezone("2020-04-24T09:00:00", ZoneId.of("America/Los_Angeles")) @@ -486,7 +486,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = localDateAtTimezone("2020-04-24T01:00:00", ZoneId.of("America/Los_Angeles")) val expectedWindowEndTime = localDateAtTimezone("2020-04-24T02:00:00", ZoneId.of("America/Los_Angeles")) @@ -527,7 +527,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = getInstant("2020-04-24T21:00:00Z") val expectedWindowEndTime = getInstant("2020-04-25T00:00:00Z") @@ -568,7 +568,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) val expectedWindowStartTime = localDateAtTimezone("2020-03-08T00:00:00", ZoneId.of("America/Los_Angeles")) // Fixed interval does not understand daylight savings time so 3 hours (60 minutes * 3) is added to the start time @@ -611,7 +611,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = getIndexResponse(), indexException = null, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) // 30 days (24 hours * 30) increments since epoch will land us on 2020-01-09 as the nearest bucket val expectedWindowStartTime = getInstant("2020-01-09T00:00:00Z") @@ -644,7 +644,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { listener.onFailure(getException) }.whenever(this.mock).get(any(), any()) } - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) runBlocking { val initMetadataResult = metadataService.init(rollup) @@ -671,7 +671,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { indexResponse = null, indexException = indexException, ) - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) runBlocking { val initMetadataResult = metadataService.init(rollup) @@ -711,7 +711,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { listener.onResponse(getResponse) }.whenever(this.mock).get(any(), any()) } - RollupMetadataService(client, xContentRegistry) + RollupMetadataService(client, xContentRegistry, getClusterService()) // runBlocking { // val getExistingMetadataResult = metadataService.getExistingMetadata(metadata.id) @@ -734,7 +734,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { listener.onResponse(getResponse) }.whenever(this.mock).get(any(), any()) } - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) runBlocking { val getExistingMetadataResult = @@ -760,7 +760,7 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { listener.onFailure(getException) }.whenever(this.mock).get(any(), any()) } - val metadataService = RollupMetadataService(client, xContentRegistry) + val metadataService = RollupMetadataService(client, xContentRegistry, getClusterService()) runBlocking { val getExistingMetadataResult = @@ -808,6 +808,17 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { return indexResponse } + private fun getClusterService(): org.opensearch.cluster.service.ClusterService { + val mockMetadata = mock() + val mockClusterState = + mock { + doReturn(mockMetadata).whenever(this.mock).metadata + } + return mock { + doReturn(mockClusterState).whenever(this.mock).state() + } + } + private fun getClient( searchResponse: SearchResponse?, searchException: Exception?, @@ -841,4 +852,121 @@ class RollupMetadataServiceTests : OpenSearchTestCase() { private fun localDateAtTimezone(localTime: String, timezone: ZoneId) = LocalDateTime.parse(localTime).atZone(timezone).toInstant() + + fun `test metadata for continuous rollup from rollup index source`() { + val dimensions = + listOf( + randomCalendarDateHistogram().copy( + sourceField = "timestamp", + targetField = "timestamp", + calendarInterval = "1h", + timezone = ZoneId.of(DateHistogram.UTC), + ), + ) + val rollup = + randomRollup().copy( + enabled = true, + jobEnabledTime = Instant.now(), + metadataID = null, + continuous = true, + dimensions = dimensions, + sourceIndex = "rollup-index", + ) + + val firstDocTimestamp = "2020-10-02T05:35:15Z" + val searchResponse = getSearchResponseForRollupTimestamp(rollup, firstDocTimestamp) + val client = + getClient( + searchResponse = searchResponse, + searchException = null, + indexResponse = getIndexResponse(), + indexException = null, + ) + val metadataService = RollupMetadataService(client, xContentRegistry, getRollupClusterService(rollup.sourceIndex)) + + val expectedWindowStartTime = getInstant("2020-10-02T05:00:00Z") + val expectedWindowEndTime = getInstant("2020-10-02T06:00:00Z") + + runBlocking { + val initMetadataResult = metadataService.init(rollup) + require(initMetadataResult is MetadataResult.Success) { "Init metadata returned unexpected results" } + + val metadata = initMetadataResult.metadata + assertNotNull(metadata.continuous) + assertEquals(expectedWindowStartTime, metadata.continuous!!.nextWindowStartTime) + assertEquals(expectedWindowEndTime, metadata.continuous!!.nextWindowEndTime) + } + } + + fun `test metadata for continuous rollup from rollup index with no documents`() { + val dimensions = + listOf( + randomCalendarDateHistogram().copy( + sourceField = "timestamp", + targetField = "timestamp", + calendarInterval = "1h", + timezone = ZoneId.of(DateHistogram.UTC), + ), + ) + val rollup = + randomRollup().copy( + enabled = true, + jobEnabledTime = Instant.now(), + metadataID = null, + continuous = true, + dimensions = dimensions, + sourceIndex = "rollup-index", + ) + + val searchResponse: SearchResponse = mock() + val searchHits = SearchHits(emptyArray(), null, 0.0F) + whenever(searchResponse.hits).doReturn(searchHits) + + val client = + getClient( + searchResponse = searchResponse, + searchException = null, + indexResponse = getIndexResponse(), + indexException = null, + ) + val metadataService = RollupMetadataService(client, xContentRegistry, getRollupClusterService(rollup.sourceIndex)) + + runBlocking { + val initMetadataResult = metadataService.init(rollup) + require(initMetadataResult is MetadataResult.NoMetadata) { "Init metadata should return NoMetadata when no documents found" } + } + } + + private fun getRollupClusterService(indexName: String): org.opensearch.cluster.service.ClusterService { + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true) + .build() + val mockIndexMetadata = mock { + doReturn(settings).whenever(this.mock).settings + } + val mockMetadata = mock { + doReturn(mockIndexMetadata).whenever(this.mock).index(indexName) + } + val mockClusterState = mock { + doReturn(mockMetadata).whenever(this.mock).metadata + } + return mock { + doReturn(mockClusterState).whenever(this.mock).state() + } + } + + private fun getSearchResponseForRollupTimestamp(rollup: Rollup, timestamp: String): SearchResponse { + val dateHistogram = rollup.dimensions.first() as DateHistogram + val rollupFieldName = "${dateHistogram.sourceField}.date_histogram" + + val docField = DocumentField(rollupFieldName, listOf(timestamp)) + val searchHit = SearchHit(0) + searchHit.setDocumentField(rollupFieldName, docField) + val searchHits = SearchHits(arrayOf(searchHit), null, 0.0F) + + val searchResponse: SearchResponse = mock() + whenever(searchResponse.hits).doReturn(searchHits) + + return searchResponse + } } 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 e902cdb55..c41512673 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorIT.kt @@ -2005,4 +2005,179 @@ class RollupInterceptorIT : RollupRestTestCase() { Assert.assertTrue(e.message!!.contains("Can't parse query_string query without sourceIndex mappings!")) } } + + // Test to ensure we block size > 0 search queries on rolled up index + fun `test rollup search enforces size equals zero`() { + generateNYCTaxiData("source_size_check_test") + val rollup = Rollup( + id = "size_check_test", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "size check test", + sourceIndex = "source_size_check_test", + targetIndex = "target_size_check_test", + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf(DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h")), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum(), 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() + + // Size > 0 should fail + val reqWithSize = """{"size": 1, "aggs": {"sum": {"sum": {"field": "passenger_count"}}}}""" + try { + client().makeRequest("POST", "/target_size_check_test/_search", emptyMap(), StringEntity(reqWithSize, ContentType.APPLICATION_JSON)) + fail("Expected 400 BAD_REQUEST response") + } catch (e: ResponseException) { + assertEquals("Unexpected status", RestStatus.BAD_REQUEST, e.response.restStatus()) + assertTrue(e.message?.contains("Rollup search must have size explicitly set to 0") ?: false) + } + + // Size = 0 should succeed + val reqWithZeroSize = """{"size": 0, "aggs": {"sum": {"sum": {"field": "passenger_count"}}}}""" + val response = client().makeRequest("POST", "/target_size_check_test/_search", emptyMap(), StringEntity(reqWithZeroSize, ContentType.APPLICATION_JSON)) + assertEquals("Size 0 should be allowed", RestStatus.OK, response.restStatus()) + } + + fun `test multi-tier rollup search`() { + val sourceIdx = "source_multi_tier_search" + val tier1Idx = "tier1_rollup_search" + val tier2Idx = "tier2_rollup_search" + + generateNYCTaxiData(sourceIdx) + + // Tier 1: Rollup from raw data to hourly + val tier1Rollup = Rollup( + id = "tier1_hourly_rollup", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Tier 1 hourly rollup", + sourceIndex = sourceIdx, + targetIndex = tier1Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum(), Min(), Max(), ValueCount(), Average()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(tier1Rollup) + + waitFor { + val rollupJob = getRollup(rollupId = tier1Rollup.id) + assertNotNull("Tier 1 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Tier 1 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Tier 2: Rollup from hourly to daily + val tier2Rollup = Rollup( + id = "tier2_daily_rollup", + enabled = true, + schemaVersion = 1L, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Tier 2 daily rollup", + sourceIndex = tier1Idx, + targetIndex = tier2Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 10, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1d"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum(), Min(), Max(), ValueCount(), Average()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(tier2Rollup) + + waitFor { + val rollupJob = getRollup(rollupId = tier2Rollup.id) + assertNotNull("Tier 2 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Tier 2 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + refreshAllIndices() + + // Search tier 2 rollup index + val req = """ + { + "size": 0, + "query": { + "term": { "RatecodeID": 1 } + }, + "aggs": { + "sum_passenger": { "sum": { "field": "passenger_count" } }, + "min_passenger": { "min": { "field": "passenger_count" } }, + "max_passenger": { "max": { "field": "passenger_count" } }, + "value_count_passenger": { "value_count": { "field": "passenger_count" } }, + "avg_passenger": { "avg": { "field": "passenger_count" } } + } + } + """.trimIndent() + + val sourceRes = client().makeRequest("POST", "/$sourceIdx/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(sourceRes.restStatus() == RestStatus.OK) + val tier2Res = client().makeRequest("POST", "/$tier2Idx/_search", emptyMap(), StringEntity(req, ContentType.APPLICATION_JSON)) + assertTrue(tier2Res.restStatus() == RestStatus.OK) + + val sourceAggs = sourceRes.asMap()["aggregations"] as Map> + val tier2Aggs = tier2Res.asMap()["aggregations"] as Map> + + assertEquals("Sum not consistent", sourceAggs["sum_passenger"]!!["value"], tier2Aggs["sum_passenger"]!!["value"]) + assertEquals("Min not consistent", sourceAggs["min_passenger"]!!["value"], tier2Aggs["min_passenger"]!!["value"]) + assertEquals("Max not consistent", sourceAggs["max_passenger"]!!["value"], tier2Aggs["max_passenger"]!!["value"]) + assertEquals("Value count not consistent", sourceAggs["value_count_passenger"]!!["value"], tier2Aggs["value_count_passenger"]!!["value"]) + assertEquals("Average not consistent", sourceAggs["avg_passenger"]!!["value"], tier2Aggs["avg_passenger"]!!["value"]) + } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt new file mode 100644 index 000000000..8e8af8150 --- /dev/null +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.indexmanagement.rollup.interceptor + +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.BYPASS_ROLLUP_SEARCH +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.BYPASS_SIZE_CHECK +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.clearBypass +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.getBypassLevel +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.setBypass +import org.opensearch.test.OpenSearchTestCase + +class RollupInterceptorTests : OpenSearchTestCase() { + + fun `test setBypass and getBypassLevel for BYPASS_ROLLUP_SEARCH`() { + setBypass(BYPASS_ROLLUP_SEARCH) + assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + clearBypass() + } + + fun `test setBypass and getBypassLevel for BYPASS_SIZE_CHECK`() { + setBypass(BYPASS_SIZE_CHECK) + assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) + clearBypass() + } + + fun `test getBypassLevel returns 0 when not set`() { + clearBypass() + assertEquals(0, getBypassLevel()) + } + + fun `test clearBypass resets bypass level`() { + setBypass(BYPASS_ROLLUP_SEARCH) + assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + clearBypass() + assertEquals(0, getBypassLevel()) + } + + fun `test bypass is thread local`() { + setBypass(BYPASS_ROLLUP_SEARCH) + assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + + val thread = Thread { + assertEquals(0, getBypassLevel()) + setBypass(BYPASS_SIZE_CHECK) + assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) + } + thread.start() + thread.join() + + assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + clearBypass() + } + + fun `test multiple setBypass calls overwrite previous value`() { + setBypass(BYPASS_ROLLUP_SEARCH) + assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + setBypass(BYPASS_SIZE_CHECK) + assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) + clearBypass() + } +} diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/runner/RollupRunnerIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/runner/RollupRunnerIT.kt index d7e215089..dd3a9572b 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/runner/RollupRunnerIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/runner/RollupRunnerIT.kt @@ -1546,6 +1546,652 @@ class RollupRunnerIT : RollupRestTestCase() { // - Source index with pattern with invalid indices // - Source index with pattern mapping to some closed indices + fun `test rollup on rollup with valid subset fields`() { + val sourceIdxName = "source_rollup_validation" + val firstRollupIdxName = "first_rollup_validation" + val secondRollupIdxName = "second_rollup_validation" + + generateNYCTaxiData(sourceIdxName) + + // Create first level rollup + val firstRollup = Rollup( + id = "first_rollup_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "First level rollup", + sourceIndex = sourceIdxName, + targetIndex = firstRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + RollupMetrics(sourceField = "total_amount", targetField = "total_amount", metrics = listOf(Average())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(firstRollup) + waitFor { + val rollupJob = getRollup(rollupId = firstRollup.id) + assertNotNull("First rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Create second level rollup with subset of fields (should succeed) + val secondRollup = Rollup( + id = "second_rollup_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Second level rollup with subset", + sourceIndex = firstRollupIdxName, + targetIndex = secondRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1d"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(secondRollup) + waitFor { + val rollupJob = getRollup(rollupId = secondRollup.id) + assertNotNull("Second rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Second rollup should succeed with subset fields", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + } + + fun `test rollup on rollup with invalid metric fields`() { + val sourceIdxName = "source_rollup_invalid_metrics" + val firstRollupIdxName = "first_rollup_invalid_metrics" + val secondRollupIdxName = "second_rollup_invalid_metrics" + + generateNYCTaxiData(sourceIdxName) + + // Create first level rollup + val firstRollup = Rollup( + id = "first_rollup_invalid_metrics_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "First level rollup for invalid test", + sourceIndex = sourceIdxName, + targetIndex = firstRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(firstRollup) + waitFor { + val rollupJob = getRollup(rollupId = firstRollup.id) + assertNotNull("First rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Create second level rollup with invalid field (should fail) + val secondRollup = Rollup( + id = "second_rollup_invalid_metrics_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Second level rollup with invalid field", + sourceIndex = firstRollupIdxName, + targetIndex = secondRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1d"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Average())), // This field doesn't exist in first rollup + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(secondRollup) + waitFor { + val rollupJob = getRollup(rollupId = secondRollup.id) + assertNotNull("Second rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Second rollup should fail with invalid field", RollupMetadata.Status.FAILED, rollupMetadata.status) + assertTrue( + "Failure reason should mention invalid metric field", + rollupMetadata.failureReason?.contains("Target rollup requests metrics") == true, + ) + } + } + + fun `test rollup on rollup with invalid timestamp interval`() { + val sourceIdxName = "source_rollup_invalid_interval" + val firstRollupIdxName = "first_rollup_invalid_interval" + val secondRollupIdxName = "second_rollup_invalid_interval" + + generateNYCTaxiData(sourceIdxName) + + // Create first level rollup + val firstRollup = Rollup( + id = "first_rollup_invalid_interval_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "First level rollup for invalid test", + sourceIndex = sourceIdxName, + targetIndex = firstRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "10m"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(firstRollup) + waitFor { + val rollupJob = getRollup(rollupId = firstRollup.id) + assertNotNull("First rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Create second level rollup with invalid field (should fail) + val secondRollup = Rollup( + id = "second_rollup_invalid_interval_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Second level rollup with invalid field", + sourceIndex = firstRollupIdxName, + targetIndex = secondRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "15m"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Average())), // This field doesn't exist in first rollup + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(secondRollup) + waitFor { + val rollupJob = getRollup(rollupId = secondRollup.id) + assertNotNull("Second rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Second rollup should fail with invalid field", RollupMetadata.Status.FAILED, rollupMetadata.status) + assertTrue( + "Failure reason should mention invalid metric field", + rollupMetadata.failureReason?.contains("must be an exact multiple of source interval") == true, + ) + } + } + + fun `test rollup on rollup with invalid fields`() { + val sourceIdxName = "source_rollup_invalid_fields" + val firstRollupIdxName = "first_rollup_invalid_fields" + val secondRollupIdxName = "second_rollup_invalid_fields" + + generateNYCTaxiData(sourceIdxName) + + // Create first level rollup + val firstRollup = Rollup( + id = "first_rollup_invalid_fields_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "First level rollup for invalid test", + sourceIndex = sourceIdxName, + targetIndex = firstRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1s"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(firstRollup) + waitFor { + val rollupJob = getRollup(rollupId = firstRollup.id) + assertNotNull("First rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Create second level rollup with invalid field (should fail) + val secondRollup = Rollup( + id = "second_rollup_invalid_fields_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Second level rollup with invalid field", + sourceIndex = firstRollupIdxName, + targetIndex = secondRollupIdxName, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "5s"), + Terms("passenger_count", "passenger_count"), + ), + metrics = listOf( + RollupMetrics(sourceField = "passenger_count", targetField = "passenger_count", metrics = listOf(Sum())), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(secondRollup) + waitFor { + val rollupJob = getRollup(rollupId = secondRollup.id) + assertNotNull("Second rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Second rollup should fail with invalid field", RollupMetadata.Status.FAILED, rollupMetadata.status) + assertTrue( + "Failure reason should mention invalid metric field", + rollupMetadata.failureReason?.contains("that don't exist in source rollup") == true, + ) + } + } + + fun `test multi tier rollup with all metrics 3 levels`() { + val sourceIdx = "ecommerce_raw_data" + val level1Idx = "ecommerce_hourly_rollup" + val level2Idx = "ecommerce_daily_rollup" + val level3Idx = "ecommerce_weekly_rollup" + + // Create source data with NYC taxi data + generateNYCTaxiData(sourceIdx) + + // Level 1: Raw data -> Hourly rollup (all metrics) + val level1Rollup = Rollup( + id = "ecommerce_level1_rollup", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Level 1: Hourly rollup with all metrics", + sourceIndex = sourceIdx, + targetIndex = level1Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf(Sum(), Average(), Min(), Max(), ValueCount()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(level1Rollup) + waitFor { + val rollupJob = getRollup(rollupId = level1Rollup.id) + assertNotNull("Level 1 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Level 1 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Level 2: Hourly -> Daily rollup + val level2Rollup = Rollup( + id = "ecommerce_level2_rollup", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Level 2: Daily rollup from hourly data", + sourceIndex = level1Idx, + targetIndex = level2Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1d"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf(Sum(), Average(), Min(), Max(), ValueCount()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(level2Rollup) + waitFor { + val rollupJob = getRollup(rollupId = level2Rollup.id) + assertNotNull("Level 2 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Level 2 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Level 3: Daily -> Weekly rollup + val level3Rollup = Rollup( + id = "ecommerce_level3_rollup", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Level 3: Weekly rollup from daily data", + sourceIndex = level2Idx, + targetIndex = level3Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "7d"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf(Sum(), Average(), Min(), Max(), ValueCount()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(level3Rollup) + waitFor { + val rollupJob = getRollup(rollupId = level3Rollup.id) + assertNotNull("Level 3 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Level 3 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + + // Verify data consistency across all levels + verifyMultiTierRollupData(sourceIdx, level3Idx) + } + } + + @Suppress("UNCHECKED_CAST") + fun `test continuous rollup on rollup index`() { + val sourceIdx = "continuous_source_data" + val level1Idx = "continuous_level1_rollup" + val level2Idx = "continuous_level2_rollup" + + generateNYCTaxiData(sourceIdx) + + // Level 1: Non-continuous rollup to create rollup index + val level1Rollup = Rollup( + id = "continuous_level1_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Level 1: Create rollup index", + sourceIndex = sourceIdx, + targetIndex = level1Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 0, + continuous = false, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1d"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf(Sum(), Average(), Min(), Max(), ValueCount()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(level1Rollup) + waitFor { + val rollupJob = getRollup(rollupId = level1Rollup.id) + assertNotNull("Level 1 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("Level 1 rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + // Level 2: Continuous rollup on rollup index + val level2Rollup = Rollup( + id = "continuous_level2_job", + schemaVersion = 1L, + enabled = true, + jobSchedule = IntervalSchedule(Instant.now(), 1, ChronoUnit.MINUTES), + jobLastUpdatedTime = Instant.now(), + jobEnabledTime = Instant.now(), + description = "Level 2: Continuous rollup on rollup index", + sourceIndex = level1Idx, + targetIndex = level2Idx, + targetIndexSettings = null, + metadataID = null, + roles = emptyList(), + pageSize = 100, + delay = 100, + continuous = true, + dimensions = listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "7d"), + Terms(sourceField = "PULocationID", targetField = "PULocationID"), + ), + metrics = listOf( + RollupMetrics( + sourceField = "passenger_count", targetField = "passenger_count", + metrics = listOf(Sum(), Average(), Min(), Max(), ValueCount()), + ), + ), + ).let { createRollup(it, it.id) } + + updateRollupStartTime(level2Rollup) + waitFor { + val rollupJob = getRollup(rollupId = level2Rollup.id) + assertNotNull("Level 2 rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertNotNull("Level 2 continuous metadata not found", rollupMetadata.continuous) + assertEquals("Level 2 rollup is not started", RollupMetadata.Status.STARTED, rollupMetadata.status) + assertTrue("Level 2 rollup processed no documents", rollupMetadata.stats.documentsProcessed > 0) + assertTrue("Level 2 rollup indexed no rollups", rollupMetadata.stats.rollupsIndexed > 0) + + // Verify metrics are consistent between source and level 2 rollup + val aggReq = """ + { + "size": 0, + "query": { "match_all": {} }, + "aggs": { + "sum_passenger": { "sum": { "field": "passenger_count" } }, + "min_passenger": { "min": { "field": "passenger_count" } }, + "max_passenger": { "max": { "field": "passenger_count" } }, + "value_count_passenger": { "value_count": { "field": "passenger_count" } }, + "avg_passenger": { "avg": { "field": "passenger_count" } } + } + } + """.trimIndent() + + val sourceRes = client().makeRequest( + RestRequest.Method.POST.name, + "/$sourceIdx/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + val level2Res = client().makeRequest( + RestRequest.Method.POST.name, + "/$level2Idx/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + + val sourceAggs = sourceRes.asMap()["aggregations"] as Map> + val level2Aggs = level2Res.asMap()["aggregations"] as Map> + + assertEquals( + "Sum not consistent", + sourceAggs["sum_passenger"]!!["value"], + level2Aggs["sum_passenger"]!!["value"], + ) + assertEquals( + "Min not consistent", + sourceAggs["min_passenger"]!!["value"], + level2Aggs["min_passenger"]!!["value"], + ) + assertEquals( + "Max not consistent", + sourceAggs["max_passenger"]!!["value"], + level2Aggs["max_passenger"]!!["value"], + ) + assertEquals( + "Value count not consistent", + sourceAggs["value_count_passenger"]!!["value"], + level2Aggs["value_count_passenger"]!!["value"], + ) + assertEquals( + "Average not consistent", + sourceAggs["avg_passenger"]!!["value"], + level2Aggs["avg_passenger"]!!["value"], + ) + } + } + + @Suppress("UNCHECKED_CAST") + private fun verifyMultiTierRollupData(sourceIdx: String, level3Idx: String) { + // Verify all 5 metrics are consistent between source and final rollup + val aggReq = """ + { + "size": 0, + "query": { + "match_all": {} + }, + "aggs": { + "sum_passenger_count": { + "sum": { "field": "passenger_count" } + }, + "avg_passenger_count": { + "avg": { "field": "passenger_count" } + }, + "min_passenger_count": { + "min": { "field": "passenger_count" } + }, + "max_passenger_count": { + "max": { "field": "passenger_count" } + }, + "count_passenger_count": { + "value_count": { "field": "passenger_count" } + } + } + } + """.trimIndent() + + val sourceResponse = client().makeRequest(RestRequest.Method.POST.name, "/$sourceIdx/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Source search should succeed", sourceResponse.restStatus() == RestStatus.OK) + + val level3Response = client().makeRequest(RestRequest.Method.POST.name, "/$level3Idx/_search", emptyMap(), StringEntity(aggReq, ContentType.APPLICATION_JSON)) + assertTrue("Level 3 search should succeed", level3Response.restStatus() == RestStatus.OK) + + val sourceAggs = sourceResponse.asMap()["aggregations"] as Map> + val level3Aggs = level3Response.asMap()["aggregations"] as Map> + + // Verify all 5 metrics + assertEquals( + "Sum should be consistent", + sourceAggs["sum_passenger_count"]!!["value"], + level3Aggs["sum_passenger_count"]!!["value"], + ) + assertEquals( + "Average should be consistent", + sourceAggs["avg_passenger_count"]!!["value"], + level3Aggs["avg_passenger_count"]!!["value"], + ) + assertEquals( + "Min should be consistent", + sourceAggs["min_passenger_count"]!!["value"], + level3Aggs["min_passenger_count"]!!["value"], + ) + assertEquals( + "Max should be consistent", + sourceAggs["max_passenger_count"]!!["value"], + level3Aggs["max_passenger_count"]!!["value"], + ) + + assertEquals( + "Value count should be consistent", + sourceAggs["count_passenger_count"]!!["value"], + level3Aggs["count_passenger_count"]!!["value"], + ) + } + private fun deleteRollupMetadata(metadataId: String) { val response = adminClient().makeRequest("DELETE", "${IndexManagementPlugin.INDEX_MANAGEMENT_INDEX}/_doc/$metadataId") assertEquals("Unable to delete rollup metadata $metadataId", RestStatus.OK, response.restStatus()) 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 4fe289ca5..dfa7c2479 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupUtilsTests.kt @@ -27,11 +27,14 @@ import org.opensearch.indexmanagement.opensearchapi.convertToMap import org.opensearch.indexmanagement.rollup.model.RollupFieldMapping import org.opensearch.indexmanagement.rollup.model.RollupMetrics import org.opensearch.indexmanagement.rollup.randomAverage +import org.opensearch.indexmanagement.rollup.randomDateHistogram +import org.opensearch.indexmanagement.rollup.randomHistogram import org.opensearch.indexmanagement.rollup.randomMax import org.opensearch.indexmanagement.rollup.randomMin import org.opensearch.indexmanagement.rollup.randomRollup import org.opensearch.indexmanagement.rollup.randomSum import org.opensearch.indexmanagement.rollup.randomTermQuery +import org.opensearch.indexmanagement.rollup.randomTerms import org.opensearch.indexmanagement.rollup.randomValueCount import org.opensearch.indexmanagement.transform.randomAggregationBuilder import org.opensearch.search.aggregations.metrics.AvgAggregationBuilder @@ -247,4 +250,311 @@ class RollupUtilsTests : OpenSearchTestCase() { val actual = isRollupIndex(indexName, clusterState) assertFalse(actual) } + + fun `test getCompositeAggregationBuilder with non-rollup source index`() { + val rollup = randomRollup() + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.EMPTY + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + assertEquals(rollup.pageSize, compositeAgg.size()) + assertEquals(rollup.dimensions.size, compositeAgg.sources().size) + } + + fun `test getCompositeAggregationBuilder with rollup source index for dimensions`() { + val dateHistogram = randomDateHistogram() + val terms = randomTerms() + val histogram = randomHistogram() + val rollup = randomRollup().copy(dimensions = listOf(dateHistogram, terms, histogram)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + assertEquals(rollup.dimensions.size, compositeAgg.sources().size) + compositeAgg.sources().forEachIndexed { index, source -> + val dimension = rollup.dimensions[index] + assertTrue(source.field().endsWith(".${dimension.type.type}")) + } + } + + fun `test getCompositeAggregationBuilder with rollup source index for Average metric`() { + val rollupMetrics = RollupMetrics("test_field", "test_field", listOf(randomAverage())) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertEquals(2, subAggs.size) + assertTrue(subAggs.any { it.name.endsWith(".avg.sum") }) + assertTrue(subAggs.any { it.name.endsWith(".avg.value_count") }) + } + + fun `test getCompositeAggregationBuilder with rollup source index for Sum metric`() { + val rollupMetrics = RollupMetrics("test_field", "test_field", listOf(randomSum())) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertTrue(subAggs.any { it.name.endsWith(".sum") && it.type == "sum" }) + } + + fun `test getCompositeAggregationBuilder with rollup source index for Max metric`() { + val rollupMetrics = RollupMetrics("test_field", "test_field", listOf(randomMax())) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertTrue(subAggs.any { it.name.endsWith(".max") && it.type == "max" }) + } + + fun `test getCompositeAggregationBuilder with rollup source index for Min metric`() { + val rollupMetrics = RollupMetrics("test_field", "test_field", listOf(randomMin())) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertTrue(subAggs.any { it.name.endsWith(".min") && it.type == "min" }) + } + + fun `test getCompositeAggregationBuilder with rollup source index for ValueCount metric`() { + val rollupMetrics = RollupMetrics("test_field", "test_field", listOf(randomValueCount())) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertTrue(subAggs.any { it.name.endsWith(".value_count") && it.type == "sum" }) + } + + fun `test getCompositeAggregationBuilder with rollup source index for multiple metrics`() { + val rollupMetrics = RollupMetrics( + "test_field", "test_field", + listOf(randomAverage(), randomSum(), randomMax(), randomMin(), randomValueCount()), + ) + val rollup = randomRollup().copy(metrics = listOf(rollupMetrics)) + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(null, clusterState) + + val subAggs = compositeAgg.subAggregations + assertEquals(6, subAggs.size) + } + + fun `test getCompositeAggregationBuilder with afterKey`() { + val rollup = randomRollup() + val afterKey = mapOf("test_key" to "test_value") + + val clusterState: ClusterState = mock() + val metadata: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.EMPTY + + whenever(clusterState.metadata).doReturn(metadata) + whenever(metadata.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val compositeAgg = rollup.getCompositeAggregationBuilder(afterKey, clusterState) + + assertNotNull(compositeAgg) + } + + fun `test getRollupSearchRequest with non-rollup source index`() { + val rollup = randomRollup() + val metadata = org.opensearch.indexmanagement.rollup.model.RollupMetadata( + rollupID = rollup.id, + lastUpdatedTime = java.time.Instant.now(), + status = org.opensearch.indexmanagement.rollup.model.RollupMetadata.Status.STARTED, + stats = org.opensearch.indexmanagement.rollup.model.RollupStats(0, 0, 0, 0, 0), + ) + + val clusterState: ClusterState = mock() + val metadataObj: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.EMPTY + + whenever(clusterState.metadata).doReturn(metadataObj) + whenever(metadataObj.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val searchRequest = rollup.getRollupSearchRequest(metadata, clusterState) + + assertNotNull(searchRequest) + assertEquals(rollup.sourceIndex, searchRequest.indices()[0]) + val query = searchRequest.source().query() + assertTrue(query is MatchAllQueryBuilder) + } + + fun `test getRollupSearchRequest with rollup source index uses transformed date field`() { + val dateHistogram = randomDateHistogram() + val rollup = randomRollup().copy(dimensions = listOf(dateHistogram)) + val metadata = org.opensearch.indexmanagement.rollup.model.RollupMetadata( + rollupID = rollup.id, + lastUpdatedTime = java.time.Instant.now(), + status = org.opensearch.indexmanagement.rollup.model.RollupMetadata.Status.STARTED, + stats = org.opensearch.indexmanagement.rollup.model.RollupStats(0, 0, 0, 0, 0), + ) + + val clusterState: ClusterState = mock() + val metadataObj: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadataObj) + whenever(metadataObj.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val searchRequest = rollup.getRollupSearchRequest(metadata, clusterState) + + assertNotNull(searchRequest) + val query = searchRequest.source().query() + assertTrue(query is MatchAllQueryBuilder) + } + + fun `test getRollupSearchRequest with continuous metadata and rollup source index`() { + val dateHistogram = randomDateHistogram() + val rollup = randomRollup().copy(dimensions = listOf(dateHistogram)) + val startTime = java.time.Instant.now().minusSeconds(3600) + val endTime = java.time.Instant.now() + val continuousMetadata = org.opensearch.indexmanagement.rollup.model.ContinuousMetadata(startTime, endTime) + val metadata = org.opensearch.indexmanagement.rollup.model.RollupMetadata( + rollupID = rollup.id, + lastUpdatedTime = java.time.Instant.now(), + continuous = continuousMetadata, + status = org.opensearch.indexmanagement.rollup.model.RollupMetadata.Status.STARTED, + stats = org.opensearch.indexmanagement.rollup.model.RollupStats(0, 0, 0, 0, 0), + ) + + val clusterState: ClusterState = mock() + val metadataObj: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.builder() + .put(org.opensearch.indexmanagement.rollup.settings.RollupSettings.ROLLUP_INDEX.key, true).build() + + whenever(clusterState.metadata).doReturn(metadataObj) + whenever(metadataObj.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val searchRequest = rollup.getRollupSearchRequest(metadata, clusterState) + + assertNotNull(searchRequest) + val query = searchRequest.source().query() + assertTrue(query is RangeQueryBuilder) + val rangeQuery = query as RangeQueryBuilder + assertEquals("${dateHistogram.sourceField}.date_histogram", rangeQuery.fieldName()) + assertEquals(startTime.toEpochMilli(), rangeQuery.from()) + assertEquals(endTime.toEpochMilli(), rangeQuery.to()) + } + + fun `test getRollupSearchRequest with continuous metadata and non-rollup source index`() { + val dateHistogram = randomDateHistogram() + val rollup = randomRollup().copy(dimensions = listOf(dateHistogram)) + val startTime = java.time.Instant.now().minusSeconds(3600) + val endTime = java.time.Instant.now() + val continuousMetadata = org.opensearch.indexmanagement.rollup.model.ContinuousMetadata(startTime, endTime) + val metadata = org.opensearch.indexmanagement.rollup.model.RollupMetadata( + rollupID = rollup.id, + lastUpdatedTime = java.time.Instant.now(), + continuous = continuousMetadata, + status = org.opensearch.indexmanagement.rollup.model.RollupMetadata.Status.STARTED, + stats = org.opensearch.indexmanagement.rollup.model.RollupStats(0, 0, 0, 0, 0), + ) + + val clusterState: ClusterState = mock() + val metadataObj: Metadata = mock() + val indexMetadata: org.opensearch.cluster.metadata.IndexMetadata = mock() + val settings = org.opensearch.common.settings.Settings.EMPTY + + whenever(clusterState.metadata).doReturn(metadataObj) + whenever(metadataObj.index(rollup.sourceIndex)).doReturn(indexMetadata) + whenever(indexMetadata.settings).doReturn(settings) + + val searchRequest = rollup.getRollupSearchRequest(metadata, clusterState) + + assertNotNull(searchRequest) + val query = searchRequest.source().query() + assertTrue(query is RangeQueryBuilder) + val rangeQuery = query as RangeQueryBuilder + assertEquals(dateHistogram.sourceField, rangeQuery.fieldName()) + assertEquals(startTime.toEpochMilli(), rangeQuery.from()) + assertEquals(endTime.toEpochMilli(), rangeQuery.to()) + } } From 115d67e729d253f9f6b0d89cb152e95afc30c59c Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Sat, 29 Nov 2025 00:26:56 +0530 Subject: [PATCH 2/6] Using flag in place of thread context for internal search calls Signed-off-by: Kshitij Tandon --- .../rollup/RollupMetadataService.kt | 42 ++++++++----------- .../indexmanagement/rollup/RollupRunner.kt | 11 +---- .../rollup/RollupSearchService.kt | 10 +++++ .../rollup/interceptor/RollupInterceptor.kt | 34 +++++++++++++-- 4 files changed, 60 insertions(+), 37 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt index 935202806..06e07d121 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt @@ -242,7 +242,7 @@ class RollupMetadataService( val dateHistogram = rollup.dimensions.first() as DateHistogram val dateField = dateHistogram.sourceField - logger.info("Idhr se ja rha hu mai tumhe kya 2") + val bypassMarker = "_rollup_internal_bypass_${org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK}" val searchRequest = SearchRequest(rollup.sourceIndex) .source( @@ -251,37 +251,31 @@ class RollupMetadataService( .query(MatchAllQueryBuilder()) .sort("$dateField.date_histogram", SortOrder.ASC) .trackTotalHits(false) - .fetchSource(false) + .fetchSource(org.opensearch.search.fetch.subphase.FetchSourceContext(false, arrayOf(bypassMarker), emptyArray())) .docValueField("$dateField.date_histogram", DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT), ) .allowPartialSearchResults(false) - // Set BYPASS_SIZE_CHECK to allow size=1 when querying rollup index to get earliest timestamp - // This is needed for continuous rollup initialization on rollup indices (multi-tier rollup) - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.setBypass( - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK, - ) - try { - val response: SearchResponse = client.suspendUntil { search(searchRequest, it) } + // Set bypass in ThreadContext for multi-node support - skips all rollup validations + // This is needed because we're querying a rollup index directly without aggregations + // Set bypass via preference parameter for multi-node support - allows size > 0 for fetching earliest timestamp +// searchRequest.preference("_rollup_bypass:${org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK}") - if (response.hits.hits.isEmpty()) { - return StartingTimeResult.NoDocumentsFound - } + val response: SearchResponse = client.suspendUntil { search(searchRequest, it) } - logger.info("Idhr se ja rha hu mai tumhe kya") + if (response.hits.hits.isEmpty()) { + return StartingTimeResult.NoDocumentsFound + } - // In rollup indices, date histogram fields are named as "field.date_histogram" - val rollupDateField = "$dateField.date_histogram" - val firstHitTimestampAsString: String = - response.hits.hits.first().field(rollupDateField).getValue() - ?: return StartingTimeResult.NoDocumentsFound + // In rollup indices, date histogram fields are named as "field.date_histogram" + val rollupDateField = "$dateField.date_histogram" + val firstHitTimestampAsString: String = + response.hits.hits.first().field(rollupDateField).getValue() + ?: return StartingTimeResult.NoDocumentsFound - val formatter = DateFormatter.forPattern(DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT) - val epochMillis = DateFormatters.from(formatter.parse(firstHitTimestampAsString), formatter.locale()).toInstant().toEpochMilli() - return StartingTimeResult.Success(getRoundedTime(epochMillis, dateHistogram)) - } finally { - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.clearBypass() - } + val formatter = DateFormatter.forPattern(DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT) + val epochMillis = DateFormatters.from(formatter.parse(firstHitTimestampAsString), formatter.locale()).toInstant().toEpochMilli() + return StartingTimeResult.Success(getRoundedTime(epochMillis, dateHistogram)) } catch (e: RemoteTransportException) { val unwrappedException = ExceptionsHelper.unwrapCause(e) as Exception logger.error("Error when getting earliest timestamp from rollup index for rollup [{}]: {}", rollup.id, unwrappedException) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt index 07e01f570..eeacad579 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupRunner.kt @@ -271,16 +271,7 @@ object RollupRunner : withClosableContext( IndexManagementSecurityContext(job.id, settings, threadPool.threadContext, job.user), ) { - // Need to set this bypass as we are already doing the required aggregation re-writing in the composite aggregation - // hence do not need to go through the interceptor for aggregation rewriting - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.setBypass( - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_ROLLUP_SEARCH, - ) - try { - rollupSearchService.executeCompositeSearch(updatableJob, metadata, clusterService) - } finally { - org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.clearBypass() - } + rollupSearchService.executeCompositeSearch(updatableJob, metadata, clusterService) } val rollupResult = when (rollupSearchResult) { diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt index 0e68866e5..d49266e8f 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupSearchService.kt @@ -19,6 +19,7 @@ import org.opensearch.core.action.ActionListener import org.opensearch.core.common.breaker.CircuitBreakingException import org.opensearch.indexmanagement.opensearchapi.retry import org.opensearch.indexmanagement.opensearchapi.suspendUntil +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.model.RollupMetadata import org.opensearch.indexmanagement.rollup.settings.RollupSettings.Companion.MINIMUM_CANCEL_AFTER_TIME_INTERVAL_MINUTES @@ -26,6 +27,7 @@ import org.opensearch.indexmanagement.rollup.settings.RollupSettings.Companion.R import org.opensearch.indexmanagement.rollup.settings.RollupSettings.Companion.ROLLUP_SEARCH_BACKOFF_MILLIS import org.opensearch.indexmanagement.rollup.util.getRollupSearchRequest import org.opensearch.search.aggregations.MultiBucketConsumerService +import org.opensearch.search.fetch.subphase.FetchSourceContext import org.opensearch.transport.RemoteTransportException import org.opensearch.transport.client.Client import java.time.Instant @@ -117,6 +119,14 @@ class RollupSearchService( val cancelTimeoutTimeValue = TimeValue.timeValueMinutes(getCancelAfterTimeInterval(cancelAfterTimeInterval.minutes)) searchRequest.cancelAfterTimeInterval = cancelTimeoutTimeValue + // Set bypass flag via FetchSourceContext for multi-node support + // Using fetchSource=false with marker in includes to avoid affecting actual source fetching + val bypassMarker = "_rollup_internal_bypass_${RollupInterceptor.BYPASS_ROLLUP_SEARCH}" + + searchRequest.source().fetchSource( + FetchSourceContext(false, arrayOf(bypassMarker), emptyArray()), + ) + search(searchRequest, listener) } }, 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 89410c2cc..35b1270bd 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -69,6 +69,7 @@ class RollupInterceptor( /** * Thread-local bypass mechanism for internal operations that need to query rollup indices * without triggering interceptor validations. + * Kept for backward compatibility with single-node clusters. */ private val bypassInterceptor = ThreadLocal() @@ -108,6 +109,21 @@ class RollupInterceptor( fun getBypassLevel(): Int = bypassInterceptor.get() ?: 0 } + /** + * Reads the bypass value from the request's FetchSourceContext. + * Returns the bypass level if the marker is present in includes array, null otherwise. + * This enables bypass mechanism to work in multi-node clusters. + */ + private fun getBypassFromFetchSource(request: ShardSearchRequest): Int? { + val includes = request.source()?.fetchSource()?.includes() + + // Look for our bypass marker in the includes array and extract the bypass level + return includes + ?.find { it.startsWith("_rollup_internal_bypass_") } + ?.substringAfter("_rollup_internal_bypass_") + ?.toIntOrNull() + } + init { clusterService.clusterSettings.addSettingsUpdateConsumer(RollupSettings.ROLLUP_SEARCH_ENABLED) { searchEnabled = it @@ -129,14 +145,26 @@ class RollupInterceptor( ): TransportRequestHandler = object : TransportRequestHandler { override fun messageReceived(request: T, channel: TransportChannel, task: Task) { if (searchEnabled && request is ShardSearchRequest) { + logger.info("Idhr aaya hu 3") val index = request.shardId().indexName val isRollupIndex = isRollupIndex(index, clusterService.state()) if (isRollupIndex) { - val bypassLevel = getBypassLevel() + // Check bypass from FetchSourceContext (multi-node) OR ThreadLocal (single-node/backward compat) + val bypassFromFetchSource = getBypassFromFetchSource(request) + val bypassFromThreadLocal = getBypassLevel() + + // Use FetchSourceContext value if present, otherwise fall back to ThreadLocal + val effectiveBypass = bypassFromFetchSource ?: bypassFromThreadLocal + + logger.debug( + "RollupInterceptor bypass check - fetchSource: $bypassFromFetchSource, " + + "threadLocal: $bypassFromThreadLocal, effective: $effectiveBypass", + ) + // BYPASS_ROLLUP_SEARCH: Skip all validations and query rewriting // Used for composite aggregation query which we do for retrieving the buckets // to write to target index during rollup of rolled up index - if (bypassLevel == BYPASS_ROLLUP_SEARCH) { + if (effectiveBypass == BYPASS_ROLLUP_SEARCH) { actualHandler.messageReceived(request, channel, task) return } @@ -145,7 +173,7 @@ class RollupInterceptor( // fetch documents (e.g., fetching earliest timestamp document for rolled index // in case of continuous rollup job). Normal rollup searches must have size=0 // since they should only return aggregations - if (bypassLevel != BYPASS_SIZE_CHECK && request.source().size() != 0) { + if (effectiveBypass != BYPASS_SIZE_CHECK && request.source().size() != 0) { throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") } From cad079c3411aeb29522cc13f377bccecd9039b4b Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Sat, 29 Nov 2025 11:25:54 +0530 Subject: [PATCH 3/6] Removing some unwanted code Signed-off-by: Kshitij Tandon --- .../rollup/RollupMetadataService.kt | 13 +- .../rollup/interceptor/RollupInterceptor.kt | 61 ++----- .../interceptor/RollupInterceptorTests.kt | 153 +++++++++++++----- 3 files changed, 134 insertions(+), 93 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt index 06e07d121..9ab1ea04a 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/RollupMetadataService.kt @@ -31,6 +31,7 @@ import org.opensearch.indexmanagement.IndexManagementPlugin import org.opensearch.indexmanagement.common.model.dimension.DateHistogram import org.opensearch.indexmanagement.opensearchapi.parseWithType import org.opensearch.indexmanagement.opensearchapi.suspendUntil +import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor import org.opensearch.indexmanagement.rollup.model.ContinuousMetadata import org.opensearch.indexmanagement.rollup.model.Rollup import org.opensearch.indexmanagement.rollup.model.RollupMetadata @@ -41,6 +42,7 @@ import org.opensearch.indexmanagement.util.NO_ID import org.opensearch.search.aggregations.bucket.composite.InternalComposite import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.search.fetch.subphase.FetchSourceContext import org.opensearch.search.sort.SortOrder import org.opensearch.transport.RemoteTransportException import org.opensearch.transport.client.Client @@ -242,7 +244,9 @@ class RollupMetadataService( val dateHistogram = rollup.dimensions.first() as DateHistogram val dateField = dateHistogram.sourceField - val bypassMarker = "_rollup_internal_bypass_${org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK}" + // For multi-tier rollup, we would be querying a document on a rollup index + // So we set this bypassMarker in fetchSource as a flag to help bypass the validation in RollupInterceptor + val bypassMarker = "_rollup_internal_bypass_${RollupInterceptor.BYPASS_SIZE_CHECK}" val searchRequest = SearchRequest(rollup.sourceIndex) .source( @@ -251,16 +255,11 @@ class RollupMetadataService( .query(MatchAllQueryBuilder()) .sort("$dateField.date_histogram", SortOrder.ASC) .trackTotalHits(false) - .fetchSource(org.opensearch.search.fetch.subphase.FetchSourceContext(false, arrayOf(bypassMarker), emptyArray())) + .fetchSource(FetchSourceContext(false, arrayOf(bypassMarker), emptyArray())) .docValueField("$dateField.date_histogram", DATE_FIELD_STRICT_DATE_OPTIONAL_TIME_FORMAT), ) .allowPartialSearchResults(false) - // Set bypass in ThreadContext for multi-node support - skips all rollup validations - // This is needed because we're querying a rollup index directly without aggregations - // Set bypass via preference parameter for multi-node support - allows size > 0 for fetching earliest timestamp -// searchRequest.preference("_rollup_bypass:${org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.BYPASS_SIZE_CHECK}") - val response: SearchResponse = client.suspendUntil { search(searchRequest, it) } if (response.hits.hits.isEmpty()) { 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 35b1270bd..2fb9c2e6c 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -66,16 +66,10 @@ class RollupInterceptor( @Volatile private var searchRawRollupIndices = RollupSettings.ROLLUP_SEARCH_SOURCE_INDICES.get(settings) companion object { - /** - * Thread-local bypass mechanism for internal operations that need to query rollup indices - * without triggering interceptor validations. - * Kept for backward compatibility with single-node clusters. - */ - private val bypassInterceptor = ThreadLocal() - /** * Bypass level that skips all rollup search validations and rewriting. - * Used when the system needs to query rollup indices directly using composite aggregation + * Used when the system needs to query rollup indices directly using composite aggregation. + * Passed via FetchSourceContext marker for multi-node support. */ const val BYPASS_ROLLUP_SEARCH = 1 @@ -83,30 +77,9 @@ class RollupInterceptor( * Bypass level that allows non-zero size in rollup searches. * Used for internal operations like continuous rollup initialization that need to fetch * actual documents from rollup indices (e.g., getEarliestTimestampFromRollupIndex). + * Passed via FetchSourceContext marker for multi-node support. */ const val BYPASS_SIZE_CHECK = 2 - - /** - * Sets the bypass level for the current thread. - * Must be followed by clearBypass() to avoid leaking the bypass state. - */ - fun setBypass(bypassLevel: Int) { - bypassInterceptor.set(bypassLevel) - } - - /** - * Clears the bypass level for the current thread. - * Should always be called in a finally block after setBypass(). - */ - fun clearBypass() { - bypassInterceptor.remove() - } - - /** - * Gets the current bypass level for the thread. - * Returns 0 if no bypass is set. - */ - fun getBypassLevel(): Int = bypassInterceptor.get() ?: 0 } /** @@ -114,7 +87,7 @@ class RollupInterceptor( * Returns the bypass level if the marker is present in includes array, null otherwise. * This enables bypass mechanism to work in multi-node clusters. */ - private fun getBypassFromFetchSource(request: ShardSearchRequest): Int? { + internal fun getBypassFromFetchSource(request: ShardSearchRequest): Int? { val includes = request.source()?.fetchSource()?.includes() // Look for our bypass marker in the includes array and extract the bypass level @@ -145,35 +118,25 @@ class RollupInterceptor( ): TransportRequestHandler = object : TransportRequestHandler { override fun messageReceived(request: T, channel: TransportChannel, task: Task) { if (searchEnabled && request is ShardSearchRequest) { - logger.info("Idhr aaya hu 3") val index = request.shardId().indexName val isRollupIndex = isRollupIndex(index, clusterService.state()) if (isRollupIndex) { - // Check bypass from FetchSourceContext (multi-node) OR ThreadLocal (single-node/backward compat) - val bypassFromFetchSource = getBypassFromFetchSource(request) - val bypassFromThreadLocal = getBypassLevel() + // Check bypass from FetchSourceContext for multi-node support + val bypassLevel = getBypassFromFetchSource(request) - // Use FetchSourceContext value if present, otherwise fall back to ThreadLocal - val effectiveBypass = bypassFromFetchSource ?: bypassFromThreadLocal - - logger.debug( - "RollupInterceptor bypass check - fetchSource: $bypassFromFetchSource, " + - "threadLocal: $bypassFromThreadLocal, effective: $effectiveBypass", - ) + logger.debug("RollupInterceptor bypass check - bypassLevel: $bypassLevel") // BYPASS_ROLLUP_SEARCH: Skip all validations and query rewriting - // Used for composite aggregation query which we do for retrieving the buckets - // to write to target index during rollup of rolled up index - if (effectiveBypass == BYPASS_ROLLUP_SEARCH) { + // Used for composite aggregation queries when rolling up rollup indices + if (bypassLevel == BYPASS_ROLLUP_SEARCH) { actualHandler.messageReceived(request, channel, task) return } // BYPASS_SIZE_CHECK: Allow non-zero size for internal operations that need to - // fetch documents (e.g., fetching earliest timestamp document for rolled index - // in case of continuous rollup job). Normal rollup searches must have size=0 - // since they should only return aggregations - if (effectiveBypass != BYPASS_SIZE_CHECK && request.source().size() != 0) { + // fetch documents (e.g., fetching earliest timestamp document for continuous rollup initialization) + // Normal rollup searches must have size=0 since they should only return aggregations + if (bypassLevel != BYPASS_SIZE_CHECK && request.source().size() != 0) { throw IllegalArgumentException("Rollup search must have size explicitly set to 0, but found ${request.source().size()}") } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt index 8e8af8150..1e385727b 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptorTests.kt @@ -5,60 +5,139 @@ package org.opensearch.indexmanagement.rollup.interceptor +import org.junit.Before +import org.mockito.Mockito.mock +import org.mockito.Mockito.`when` +import org.opensearch.cluster.metadata.IndexNameExpressionResolver +import org.opensearch.cluster.service.ClusterService +import org.opensearch.common.settings.ClusterSettings +import org.opensearch.common.settings.Settings import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.BYPASS_ROLLUP_SEARCH import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.BYPASS_SIZE_CHECK -import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.clearBypass -import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.getBypassLevel -import org.opensearch.indexmanagement.rollup.interceptor.RollupInterceptor.Companion.setBypass +import org.opensearch.indexmanagement.rollup.settings.RollupSettings +import org.opensearch.search.builder.SearchSourceBuilder +import org.opensearch.search.fetch.subphase.FetchSourceContext +import org.opensearch.search.internal.ShardSearchRequest import org.opensearch.test.OpenSearchTestCase class RollupInterceptorTests : OpenSearchTestCase() { - fun `test setBypass and getBypassLevel for BYPASS_ROLLUP_SEARCH`() { - setBypass(BYPASS_ROLLUP_SEARCH) - assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) - clearBypass() + private lateinit var interceptor: RollupInterceptor + + @Before + fun setup() { + interceptor = createInterceptor() + } + + fun `test getBypassFromFetchSource returns null when no source`() { + val request = mock(ShardSearchRequest::class.java) + `when`(request.source()).thenReturn(null) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertNull(bypassLevel) } - fun `test setBypass and getBypassLevel for BYPASS_SIZE_CHECK`() { - setBypass(BYPASS_SIZE_CHECK) - assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) - clearBypass() + fun `test getBypassFromFetchSource returns null when no FetchSourceContext`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertNull(bypassLevel) } - fun `test getBypassLevel returns 0 when not set`() { - clearBypass() - assertEquals(0, getBypassLevel()) + fun `test getBypassFromFetchSource returns null when no includes array`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource(FetchSourceContext(true)) + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertNull(bypassLevel) } - fun `test clearBypass resets bypass level`() { - setBypass(BYPASS_ROLLUP_SEARCH) - assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) - clearBypass() - assertEquals(0, getBypassLevel()) + fun `test getBypassFromFetchSource returns null when no bypass marker present`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource(FetchSourceContext(false, arrayOf("field1", "field2"), emptyArray())) + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertNull(bypassLevel) } - fun `test bypass is thread local`() { - setBypass(BYPASS_ROLLUP_SEARCH) - assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) + fun `test getBypassFromFetchSource extracts BYPASS_ROLLUP_SEARCH correctly`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource(FetchSourceContext(false, arrayOf("_rollup_internal_bypass_$BYPASS_ROLLUP_SEARCH"), emptyArray())) - val thread = Thread { - assertEquals(0, getBypassLevel()) - setBypass(BYPASS_SIZE_CHECK) - assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) - } - thread.start() - thread.join() + `when`(request.source()).thenReturn(source) - assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) - clearBypass() + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertEquals(BYPASS_ROLLUP_SEARCH, bypassLevel) } - fun `test multiple setBypass calls overwrite previous value`() { - setBypass(BYPASS_ROLLUP_SEARCH) - assertEquals(BYPASS_ROLLUP_SEARCH, getBypassLevel()) - setBypass(BYPASS_SIZE_CHECK) - assertEquals(BYPASS_SIZE_CHECK, getBypassLevel()) - clearBypass() + fun `test getBypassFromFetchSource extracts BYPASS_SIZE_CHECK correctly`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource(FetchSourceContext(false, arrayOf("_rollup_internal_bypass_$BYPASS_SIZE_CHECK"), emptyArray())) + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertEquals(BYPASS_SIZE_CHECK, bypassLevel) + } + + fun `test getBypassFromFetchSource finds marker among multiple includes`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource( + FetchSourceContext(false, arrayOf("field1", "_rollup_internal_bypass_$BYPASS_ROLLUP_SEARCH", "field2"), emptyArray()), + ) + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertEquals(BYPASS_ROLLUP_SEARCH, bypassLevel) + } + + fun `test getBypassFromFetchSource returns null for invalid bypass marker`() { + val request = mock(ShardSearchRequest::class.java) + val source = SearchSourceBuilder() + source.fetchSource(FetchSourceContext(false, arrayOf("_rollup_internal_bypass_invalid"), emptyArray())) + + `when`(request.source()).thenReturn(source) + + val bypassLevel = interceptor.getBypassFromFetchSource(request) + + assertNull(bypassLevel) + } + + // Helper method to create interceptor instance + private fun createInterceptor(): RollupInterceptor { + val clusterService = mock(ClusterService::class.java) + val clusterSettings = ClusterSettings( + Settings.EMPTY, + setOf( + RollupSettings.ROLLUP_SEARCH_ENABLED, + RollupSettings.ROLLUP_SEARCH_ALL_JOBS, + RollupSettings.ROLLUP_SEARCH_SOURCE_INDICES, + ), + ) + `when`(clusterService.clusterSettings).thenReturn(clusterSettings) + + val settings = Settings.EMPTY + val resolver = mock(IndexNameExpressionResolver::class.java) + return RollupInterceptor(clusterService, settings, resolver) } } From b5e5ad826b79c2a0ffe4987bcd67a019d8bbaa14 Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Tue, 2 Dec 2025 14:15:01 +0530 Subject: [PATCH 4/6] Add support for source index in ISM Policy Rollup Action to help perform multi-tier rollups Signed-off-by: Kshitij Tandon --- .../step/rollup/AttemptCreateRollupJobStep.kt | 96 ++- .../indexmanagement/rollup/model/ISMRollup.kt | 32 +- .../RollupFieldValueExpressionResolver.kt | 51 ++ .../mappings/opendistro-ism-config.json | 11 +- .../action/RollupActionIT.kt | 782 ++++++++++++++++++ .../step/AttemptCreateRollupJobStepTests.kt | 343 ++++++++ .../rollup/model/ISMRollupTests.kt | 214 +++++ ...RollupFieldValueExpressionResolverTests.kt | 100 +++ 8 files changed, 1614 insertions(+), 15 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt b/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt index 9edbbea8a..5bd089774 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt @@ -18,6 +18,7 @@ import org.opensearch.indexmanagement.rollup.action.index.IndexRollupRequest import org.opensearch.indexmanagement.rollup.action.index.IndexRollupResponse import org.opensearch.indexmanagement.rollup.action.start.StartRollupAction import org.opensearch.indexmanagement.rollup.action.start.StartRollupRequest +import org.opensearch.indexmanagement.rollup.util.RollupFieldValueExpressionResolver import org.opensearch.indexmanagement.spi.indexstatemanagement.Step import org.opensearch.indexmanagement.spi.indexstatemanagement.model.ActionProperties import org.opensearch.indexmanagement.spi.indexstatemanagement.model.ManagedIndexMetaData @@ -39,39 +40,110 @@ class AttemptCreateRollupJobStep(private val action: RollupAction) : Step(name) val previousRunRollupId = managedIndexMetadata.actionMetaData?.actionProperties?.rollupId val hasPreviousRollupAttemptFailed = managedIndexMetadata.actionMetaData?.actionProperties?.hasRollupFailed - // Creating a rollup job - val rollup = action.ismRollup.toRollup(indexName, context.user) - rollupId = rollup.id - logger.info("Attempting to create a rollup job $rollupId for index $indexName") - - val indexRollupRequest = IndexRollupRequest(rollup, WriteRequest.RefreshPolicy.IMMEDIATE) - try { + // Create a temporary rollup object for template resolution context. + // This provides the rollup's source_index as {{ctx.source_index}} in templates. + val tempRollup = action.ismRollup.toRollup(indexName, context.user) + + // Resolve source_index template if provided, else use managed index name. + // This enables patterns like: + // - source_index: "{{ctx.index}}" -> resolves to the managed index name + // - source_index: null -> defaults to the managed index name (backward compatible) + val resolvedSourceIndex = if (action.ismRollup.sourceIndex != null) { + RollupFieldValueExpressionResolver.resolve( + tempRollup, + action.ismRollup.sourceIndex, + indexName, + ) + } else { + indexName + } + + // Resolve target_index template. + // Common patterns: + // - "rollup_{{ctx.index}}" -> "rollup_logs-2024-01" + // - "rollup_tier2_{{ctx.index}}" -> "rollup_tier2_rollup_tier1_logs-2024-01" + val resolvedTargetIndex = RollupFieldValueExpressionResolver.resolve( + tempRollup, + action.ismRollup.targetIndex, + indexName, + ) + + // Validate resolved indices to ensure they are valid and different. + // This catches configuration errors early before attempting to create the rollup job. + validateResolvedIndices(resolvedSourceIndex, resolvedTargetIndex) + + logger.info( + "Executing rollup from source [$resolvedSourceIndex] to target [$resolvedTargetIndex] " + + "for managed index [$indexName]", + ) + + // Create the final rollup job with resolved source_index and target_index. + // The copy() ensures that template variables are replaced with actual index names + // before the rollup job is persisted and executed. + val rollup = action.ismRollup.toRollup(indexName, context.user) + .copy(sourceIndex = resolvedSourceIndex, targetIndex = resolvedTargetIndex) + rollupId = rollup.id + logger.info("Attempting to create a rollup job $rollupId for index $indexName") + + val indexRollupRequest = IndexRollupRequest(rollup, WriteRequest.RefreshPolicy.IMMEDIATE) val response: IndexRollupResponse = context.client.suspendUntil { execute(IndexRollupAction.INSTANCE, indexRollupRequest, it) } logger.info("Received status ${response.status.status} on trying to create rollup job $rollupId") stepStatus = StepStatus.COMPLETED info = mapOf("message" to getSuccessMessage(rollup.id, indexName)) + } catch (e: IllegalArgumentException) { + val message = "Failed to validate resolved indices for rollup job" + logger.error(message, e) + stepStatus = StepStatus.FAILED + info = mapOf("message" to message, "cause" to "${e.message}") } catch (e: VersionConflictEngineException) { - val message = getFailedJobExistsMessage(rollup.id, indexName) + val message = getFailedJobExistsMessage(rollupId ?: "unknown", indexName) logger.info(message) if (rollupId == previousRunRollupId && hasPreviousRollupAttemptFailed == true) { - startRollupJob(rollup.id, context) + startRollupJob(rollupId ?: "unknown", context) } else { stepStatus = StepStatus.COMPLETED info = mapOf("info" to message) } } catch (e: RemoteTransportException) { - processFailure(rollup.id, indexName, ExceptionsHelper.unwrapCause(e) as Exception) + processFailure(rollupId ?: "unknown", indexName, ExceptionsHelper.unwrapCause(e) as Exception) } catch (e: OpenSearchException) { - processFailure(rollup.id, indexName, e) + processFailure(rollupId ?: "unknown", indexName, e) } catch (e: Exception) { - processFailure(rollup.id, indexName, e) + val message = "Failed to create rollup job" + logger.error(message, e) + stepStatus = StepStatus.FAILED + info = mapOf("message" to message, "cause" to "${e.message}") } return this } + /** + * Validates that resolved source and target indices are valid and different. + * + * This validation ensures that: + * 1. The resolved source_index is not empty or whitespace-only + * 2. The resolved target_index is not empty or whitespace-only + * 3. The source and target indices are different (prevents self-rollup) + * + * @param sourceIndex The resolved source index name (after template resolution) + * @param targetIndex The resolved target index name (after template resolution) + * @throws IllegalArgumentException if any validation rule fails, with a descriptive error message + */ + private fun validateResolvedIndices(sourceIndex: String, targetIndex: String) { + require(sourceIndex.isNotBlank()) { + "Resolved source_index cannot be empty" + } + require(targetIndex.isNotBlank()) { + "Resolved target_index cannot be empty" + } + require(sourceIndex != targetIndex) { + "Source and target indices must be different: $sourceIndex" + } + } + fun processFailure(rollupId: String, indexName: String, e: Exception) { val message = getFailedMessage(rollupId, indexName) logger.error(message, e) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt index aad38ab1a..d4cc95655 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt @@ -35,6 +35,7 @@ data class ISMRollup( val pageSize: Int, val dimensions: List, val metrics: List, + val sourceIndex: String? = null, ) : ToXContentObject, Writeable { // TODO: This can be moved to a common place, since this is shared between Rollup and ISMRollup @@ -58,6 +59,9 @@ data class ISMRollup( .field(Rollup.PAGE_SIZE_FIELD, pageSize) .field(Rollup.DIMENSIONS_FIELD, dimensions) .field(Rollup.METRICS_FIELD, metrics) + if (sourceIndex != null) { + builder.field(Rollup.SOURCE_INDEX_FIELD, sourceIndex) + } if (targetIndexSettings != null) { builder.startObject(Rollup.TARGET_INDEX_SETTINGS_FIELD) targetIndexSettings.toXContent(builder, params) @@ -67,8 +71,20 @@ data class ISMRollup( return builder } + /** + * Converts ISMRollup configuration to a Rollup job. + * + * Source index resolution priority: + * 1. If ISMRollup.sourceIndex is set, use it (for multi-tier rollups) + * 2. Otherwise, use the sourceIndex parameter (managed index from ISM context) + * + * @param sourceIndex The managed index from ISM context (fallback if ISMRollup.sourceIndex is null) + * @param user Optional user context for the rollup job + * @return Rollup job configuration + */ fun toRollup(sourceIndex: String, user: User? = null): Rollup { - val id = sourceIndex + toString() + val resolvedSourceIndex = this.sourceIndex ?: sourceIndex + val id = resolvedSourceIndex + toString() val currentTime = Instant.now() return Rollup( id = DigestUtils.sha1Hex(id), @@ -80,7 +96,7 @@ data class ISMRollup( jobLastUpdatedTime = currentTime, jobEnabledTime = currentTime, description = this.description, - sourceIndex = sourceIndex, + sourceIndex = resolvedSourceIndex, targetIndex = this.targetIndex, targetIndexSettings = this.targetIndexSettings, metadataID = null, @@ -120,6 +136,11 @@ data class ISMRollup( dimensionsList.toList() }, metrics = sin.readList(::RollupMetrics), + sourceIndex = if (sin.version.onOrAfter(Version.V_3_0_0) && sin.readBoolean()) { + sin.readString() + } else { + null + }, ) override fun toString(): String { @@ -159,6 +180,10 @@ data class ISMRollup( } } out.writeCollection(metrics) + if (out.version.onOrAfter(Version.V_3_0_0)) { + out.writeBoolean(sourceIndex != null) + if (sourceIndex != null) out.writeString(sourceIndex) + } } companion object { @@ -170,6 +195,7 @@ data class ISMRollup( ): ISMRollup { var description = "" var targetIndex = "" + var sourceIndex: String? = null var targetIndexSettings: Settings? = null var pageSize = 0 val dimensions = mutableListOf() @@ -184,6 +210,7 @@ data class ISMRollup( when (fieldName) { Rollup.DESCRIPTION_FIELD -> description = xcp.text() Rollup.TARGET_INDEX_FIELD -> targetIndex = xcp.text() + Rollup.SOURCE_INDEX_FIELD -> sourceIndex = xcp.text() Rollup.TARGET_INDEX_SETTINGS_FIELD -> { XContentParserUtils.ensureExpectedToken( XContentParser.Token.START_OBJECT, @@ -224,6 +251,7 @@ data class ISMRollup( metrics = metrics, targetIndex = targetIndex, targetIndexSettings = targetIndexSettings, + sourceIndex = sourceIndex, ) } } diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt index 7c163b608..4d2de9a5f 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt @@ -24,6 +24,15 @@ object RollupFieldValueExpressionResolver { private lateinit var clusterService: ClusterService lateinit var indexAliasUtils: IndexAliasUtils + /** + * Resolves template variables in a field value using the rollup object as context. + * This method is kept for backward compatibility and only supports {{ctx.source_index}}. + * + * @param rollup The rollup object providing context for template resolution + * @param fieldValue The field value that may contain template variables (e.g., "{{ctx.source_index}}") + * @return The resolved field value with variables replaced, or the original value if resolution + * produces an empty/null result + */ fun resolve(rollup: Rollup, fieldValue: String): String { val script = Script(ScriptType.INLINE, Script.DEFAULT_TEMPLATE_LANG, fieldValue, mapOf()) @@ -44,6 +53,48 @@ object RollupFieldValueExpressionResolver { return if (compiledValue.isNullOrBlank()) fieldValue else compiledValue } + /** + * Resolves template variables in a field value using the rollup object and managed index name as context. + * This is the primary method used by ISM rollup actions to resolve source_index and target_index templates. + * + * This method extends the basic resolve() by adding the managed index name to the template context, + * enabling the use of {{ctx.index}} in addition to {{ctx.source_index}}. + * + * @param rollup The rollup object providing context for template resolution. The rollup's source_index + * field is made available as {{ctx.source_index}} in templates. + * @param fieldValue The field value that may contain Mustache template variables. Common patterns: + * - "{{ctx.index}}" - resolves to the managed index name + * - "{{ctx.source_index}}" - resolves to the rollup's source index + * - "rollup_{{ctx.index}}" - adds a prefix to the managed index name + * - Literal values without templates are returned unchanged + * @param managedIndexName The name of the index being managed by ISM. This is the index to which + * the ISM policy is applied, and it's made available as {{ctx.index}} in templates. + * For multi-tier rollups, this would be the output of the previous tier. + * @return The resolved field value with all template variables replaced. If the resolved value is an alias, + * it's automatically resolved to the write index name. If resolution produces an empty or null result, + * the original fieldValue is returned unchanged. + */ + fun resolve(rollup: Rollup, fieldValue: String, managedIndexName: String): String { + val script = Script(ScriptType.INLINE, Script.DEFAULT_TEMPLATE_LANG, fieldValue, mapOf()) + + val contextMap = + rollup.toXContent(XContentFactory.jsonBuilder(), XCONTENT_WITHOUT_TYPE) + .toMap() + .filterKeys { key -> key in validTopContextFields } + .plus("index" to managedIndexName) + + var compiledValue = + scriptService.compile(script, TemplateScript.CONTEXT) + .newInstance(script.params + mapOf("ctx" to contextMap)) + .execute() + + if (indexAliasUtils.isAlias(compiledValue)) { + compiledValue = indexAliasUtils.getWriteIndexNameForAlias(compiledValue) + } + + return if (compiledValue.isNullOrBlank()) fieldValue else compiledValue + } + fun registerServices(scriptService: ScriptService, clusterService: ClusterService) { this.scriptService = scriptService this.clusterService = clusterService diff --git a/src/main/resources/mappings/opendistro-ism-config.json b/src/main/resources/mappings/opendistro-ism-config.json index 6250ba15d..ea5e7d5e0 100644 --- a/src/main/resources/mappings/opendistro-ism-config.json +++ b/src/main/resources/mappings/opendistro-ism-config.json @@ -1,6 +1,6 @@ { "_meta" : { - "schema_version": 24 + "schema_version": 25 }, "dynamic": "strict", "properties": { @@ -374,6 +374,15 @@ "description": { "type": "text" }, + "source_index": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, "target_index": { "type": "text", "fields": { diff --git a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt index 4834f7ddf..de1cb715b 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/action/RollupActionIT.kt @@ -759,4 +759,786 @@ class RollupActionIT : IndexStateManagementRestTestCase() { assertEquals("Value count should be consistent", sourceAggs["value_count_passenger"]!!["value"], rollupAggs["value_count_passenger"]!!["value"]) assertEquals("Average should be consistent", sourceAggs["avg_passenger"]!!["value"], rollupAggs["avg_passenger"]!!["value"]) } + + fun `test single-tier rollup with explicit source_index`() { + val indexName = "${testIndexName}_single_tier_explicit_source" + val sourceIndexName = "${testIndexName}_explicit_source_data" + val policyID = "${testIndexName}_single_tier_explicit_policy" + val targetIndexName = "${testIndexName}_single_tier_target" + + // Create source index with data + val mapping = "\"properties\": {\"timestamp\": {\"type\": \"date\"}, \"value\": {\"type\": \"long\"}}" + createIndex(sourceIndexName, null, mapping = mapping) + + // Index test data into source index + client().makeRequest( + "POST", + "/$sourceIndexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:00:00Z","value":10}""", + ContentType.APPLICATION_JSON, + ), + ) + client().makeRequest( + "POST", + "/$sourceIndexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:01:00Z","value":20}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Create rollup with explicit source_index + val rollup = ISMRollup( + description = "Rollup with explicit source_index", + sourceIndex = sourceIndexName, + targetIndex = targetIndexName, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "1h")), + metrics = listOf( + RollupMetrics( + sourceField = "value", + targetField = "value", + metrics = listOf(Sum(), Min(), Max(), ValueCount()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = Policy( + id = policyID, + description = "Single-tier rollup with explicit source", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = mapping) + + // Execute rollup + assertIndexRolledUp(indexName, policyID, rollup) + + // Verify the rollup job used the correct source index + val rollupId = rollup.toRollup(indexName).id + val rollupJob = getRollup(rollupId = rollupId) + assertEquals("Rollup should use explicit source_index", sourceIndexName, rollupJob.sourceIndex) + + // Verify target index exists and has data + assertIndexExists(targetIndexName) + } + + fun `test single-tier rollup without source_index for backward compatibility`() { + val indexName = "${testIndexName}_single_tier_backward_compat" + val policyID = "${testIndexName}_single_tier_backward_policy" + val targetIndexName = "${testIndexName}_single_tier_backward_target" + + // Create rollup WITHOUT source_index (backward compatibility) + val rollup = ISMRollup( + description = "Rollup without source_index for backward compatibility", + targetIndex = targetIndexName, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "1h")), + metrics = listOf( + RollupMetrics( + sourceField = "value", + targetField = "value", + metrics = listOf(Sum(), Min(), Max(), ValueCount()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = Policy( + id = policyID, + description = "Single-tier rollup backward compatibility", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + + val mapping = "\"properties\": {\"timestamp\": {\"type\": \"date\"}, \"value\": {\"type\": \"long\"}}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = mapping) + + // Index test data into managed index + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:00:00Z","value":15}""", + ContentType.APPLICATION_JSON, + ), + ) + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:02:00Z","value":25}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Execute rollup + assertIndexRolledUp(indexName, policyID, rollup) + + // Verify the rollup job used the managed index as source (backward compatibility) + val rollupId = rollup.toRollup(indexName).id + val rollupJob = getRollup(rollupId = rollupId) + assertEquals("Rollup should use managed index as source when source_index not specified", indexName, rollupJob.sourceIndex) + + // Verify target index exists and has data + assertIndexExists(targetIndexName) + } + + fun `test single-tier rollup verifies correct source is used`() { + val indexName = "${testIndexName}_verify_source" + val explicitSourceIndex = "${testIndexName}_explicit_source" + val policyID = "${testIndexName}_verify_source_policy" + val targetIndexName = "${testIndexName}_verify_source_target" + + // Create explicit source index with specific data + val mapping = "\"properties\": {\"timestamp\": {\"type\": \"date\"}, \"count\": {\"type\": \"long\"}}" + createIndex(explicitSourceIndex, null, mapping = mapping) + + // Index data with value 100 in explicit source + client().makeRequest( + "POST", + "/$explicitSourceIndex/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:00:00Z","count":100}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Create managed index with different data + createIndex(indexName, null, mapping = mapping) + + // Index data with value 50 in managed index + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:00:00Z","count":50}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Create rollup with explicit source_index pointing to explicitSourceIndex + val rollup = ISMRollup( + description = "Rollup to verify correct source usage", + sourceIndex = explicitSourceIndex, + targetIndex = targetIndexName, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "1h")), + metrics = listOf( + RollupMetrics( + sourceField = "count", + targetField = "count", + metrics = listOf(Sum()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = Policy( + id = policyID, + description = "Verify correct source usage", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + + createPolicy(policy, policyID) + addPolicyToIndex(indexName, policyID) + + // Execute rollup + assertIndexRolledUp(indexName, policyID, rollup) + + // Verify the rollup job used the correct source index + val rollupId = rollup.toRollup(indexName).id + val rollupJob = getRollup(rollupId = rollupId) + assertEquals("Rollup should use explicit source_index", explicitSourceIndex, rollupJob.sourceIndex) + + // Verify target index exists + assertIndexExists(targetIndexName) + } + + @Suppress("UNCHECKED_CAST") + fun `test 3-tier rollup with source_index field raw to 1m to 5m to 10m`() { + val indexName = "${testIndexName}_3tier_raw" + val policyID = "${testIndexName}_3tier_policy" + val rollup1mTarget = "${testIndexName}_3tier_1m" + val rollup5mTarget = "${testIndexName}_3tier_5m" + val rollup10mTarget = "${testIndexName}_3tier_10m" + + // First tier: raw -> 1m + val rollup1m = ISMRollup( + description = "Rollup raw to 1m interval", + targetIndex = rollup1mTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "1m")), + metrics = listOf( + RollupMetrics( + sourceField = "count", + targetField = "count", + metrics = listOf(Sum(), Min(), Max(), ValueCount()), + ), + ), + ) + + val states = listOf(State("rollup_1m", listOf(RollupAction(rollup1m, 0)), listOf())) + val policy = Policy( + id = policyID, + description = "First tier rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + + val mapping = "\"properties\": {\"timestamp\": {\"type\": \"date\"}, \"count\": {\"type\": \"long\"}}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = mapping) + + // Index test data + for (i in 0..9) { + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"timestamp":"2021-01-01T00:0$i:00Z","count":${i + 1}}""", + ContentType.APPLICATION_JSON, + ), + ) + } + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { assertEquals(policyID, getExplainManagedIndexMetaData(indexName).policyID) } + + // Execute first tier rollup (raw -> 1m) + val rollup1mId = rollup1m.toRollup(indexName).id + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup1mId, indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message"), + ) + } + + updateRollupStartTime(rollup1m.toRollup(indexName)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup1mId) + assertNotNull("First tier rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + assertEquals("First tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup1mId, indexName), + getExplainManagedIndexMetaData(indexName).info?.get("message"), + ) + } + + assertIndexExists(rollup1mTarget) + + // Second tier: 1m -> 5m (using source_index field) + val rollup5m = ISMRollup( + description = "Rollup 1m to 5m interval", + sourceIndex = rollup1mTarget, + targetIndex = rollup5mTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "5m")), + metrics = listOf( + RollupMetrics( + sourceField = "count", + targetField = "count", + metrics = listOf(Sum(), Min(), Max()), + ), + ), + ) + + val policy2ID = "${policyID}_tier2" + val states2 = listOf(State("rollup_5m", listOf(RollupAction(rollup5m, 0)), listOf())) + val policy2 = Policy( + id = policy2ID, + description = "Second tier rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states2[0].name, + states = states2, + ) + + createPolicy(policy2, policy2ID) + addPolicyToIndex(rollup1mTarget, policy2ID) + val managedIndexConfig2 = getExistingManagedIndexConfig(rollup1mTarget) + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { assertEquals(policy2ID, getExplainManagedIndexMetaData(rollup1mTarget).policyID) } + + val rollup5mId = rollup5m.toRollup(rollup1mTarget).id + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup5mId, rollup1mTarget), + getExplainManagedIndexMetaData(rollup1mTarget).info?.get("message"), + ) + } + + updateRollupStartTime(rollup5m.toRollup(rollup1mTarget)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup5mId) + assertNotNull("Second tier rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + if (rollupMetadata.status == RollupMetadata.Status.FAILED) { + fail("Second tier rollup failed: ${rollupMetadata.failureReason}") + } + assertEquals("Second tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig2) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup5mId, rollup1mTarget), + getExplainManagedIndexMetaData(rollup1mTarget).info?.get("message"), + ) + } + + assertIndexExists(rollup5mTarget) + + // Third tier: 5m -> 10m (using source_index field) + val rollup10m = ISMRollup( + description = "Rollup 5m to 10m interval", + sourceIndex = rollup5mTarget, + targetIndex = rollup10mTarget, + targetIndexSettings = null, + pageSize = 100, + dimensions = listOf(DateHistogram(sourceField = "timestamp", fixedInterval = "10m")), + metrics = listOf( + RollupMetrics( + sourceField = "count", + targetField = "count", + metrics = listOf(Sum()), + ), + ), + ) + + val policy3ID = "${policyID}_tier3" + val states3 = listOf(State("rollup_10m", listOf(RollupAction(rollup10m, 0)), listOf())) + val policy3 = Policy( + id = policy3ID, + description = "Third tier rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states3[0].name, + states = states3, + ) + + createPolicy(policy3, policy3ID) + addPolicyToIndex(rollup5mTarget, policy3ID) + val managedIndexConfig3 = getExistingManagedIndexConfig(rollup5mTarget) + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { assertEquals(policy3ID, getExplainManagedIndexMetaData(rollup5mTarget).policyID) } + + val rollup10mId = rollup10m.toRollup(rollup5mTarget).id + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { + assertEquals( + AttemptCreateRollupJobStep.getSuccessMessage(rollup10mId, rollup5mTarget), + getExplainManagedIndexMetaData(rollup5mTarget).info?.get("message"), + ) + } + + updateRollupStartTime(rollup10m.toRollup(rollup5mTarget)) + waitFor(timeout = Instant.ofEpochSecond(60)) { + val rollupJob = getRollup(rollupId = rollup10mId) + assertNotNull("Third tier rollup job doesn't have metadata set", rollupJob.metadataID) + val rollupMetadata = getRollupMetadata(rollupJob.metadataID!!) + if (rollupMetadata.status == RollupMetadata.Status.FAILED) { + fail("Third tier rollup failed: ${rollupMetadata.failureReason}") + } + assertEquals("Third tier rollup is not finished", RollupMetadata.Status.FINISHED, rollupMetadata.status) + } + + updateManagedIndexConfigStartTime(managedIndexConfig3) + waitFor { + assertEquals( + WaitForRollupCompletionStep.getJobCompletionMessage(rollup10mId, rollup5mTarget), + getExplainManagedIndexMetaData(rollup5mTarget).info?.get("message"), + ) + } + + assertIndexExists(rollup10mTarget) + + // Verify the rollup jobs used the correct source indices + val rollup5mJob = getRollup(rollupId = rollup5mId) + assertEquals("Second tier rollup should use 1m rollup as source", rollup1mTarget, rollup5mJob.sourceIndex) + + val rollup10mJob = getRollup(rollupId = rollup10mId) + assertEquals("Third tier rollup should use 5m rollup as source", rollup5mTarget, rollup10mJob.sourceIndex) + + // Verify data flows correctly through all tiers + val aggReq = """ + { + "size": 0, + "query": { "match_all": {} }, + "aggs": { + "sum_count": { "sum": { "field": "count" } } + } + } + """.trimIndent() + + val sourceResponse = client().makeRequest( + RestRequest.Method.POST.name, + "/$indexName/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + val tier1Response = client().makeRequest( + RestRequest.Method.POST.name, + "/$rollup1mTarget/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + val tier2Response = client().makeRequest( + RestRequest.Method.POST.name, + "/$rollup5mTarget/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + val tier3Response = client().makeRequest( + RestRequest.Method.POST.name, + "/$rollup10mTarget/_search", + emptyMap(), + StringEntity(aggReq, ContentType.APPLICATION_JSON), + ) + + val sourceAggs = sourceResponse.asMap()["aggregations"] as Map> + val tier1Aggs = tier1Response.asMap()["aggregations"] as Map> + val tier2Aggs = tier2Response.asMap()["aggregations"] as Map> + val tier3Aggs = tier3Response.asMap()["aggregations"] as Map> + + val sourceSum = sourceAggs["sum_count"]!!["value"] + val tier1Sum = tier1Aggs["sum_count"]!!["value"] + val tier2Sum = tier2Aggs["sum_count"]!!["value"] + val tier3Sum = tier3Aggs["sum_count"]!!["value"] + + assertEquals("Sum should be consistent between raw and 1m tier", sourceSum, tier1Sum) + assertEquals("Sum should be consistent between 1m and 5m tier", tier1Sum, tier2Sum) + assertEquals("Sum should be consistent between 5m and 10m tier", tier2Sum, tier3Sum) + } + + fun `test rollup action with source_index template using ctx index`() { + val indexName = "${testIndexName}_template_source_ctx_index" + val policyID = "${testIndexName}_policy_template_source" + val rollup = + ISMRollup( + description = "rollup with source_index template", + sourceIndex = "{{ctx.index}}", + targetIndex = "rollup_{{ctx.index}}", + targetIndexSettings = null, + pageSize = 100, + 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()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = + Policy( + id = policyID, + description = "rollup policy with source_index template", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + val sourceIndexMappingString = + "\"properties\": {\"tpep_pickup_datetime\": { \"type\": \"date\" }, \"RatecodeID\": { \"type\": " + + "\"keyword\" }, \"PULocationID\": { \"type\": \"keyword\" }, \"passenger_count\": { \"type\": \"integer\" }, \"total_amount\": " + + "{ \"type\": \"double\" }}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = sourceIndexMappingString) + + // Index test data + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"tpep_pickup_datetime":"2021-01-01T00:00:00Z","RatecodeID":"1","PULocationID":"100","passenger_count":2,"total_amount":15.5}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Execute the rollup action and wait for completion + assertIndexRolledUp(indexName, policyID, rollup) + + // Verify the rollup index was created with the correct name + assertIndexExists("rollup_$indexName") + } + + fun `test rollup action with both source_index and target_index templates`() { + val indexName = "${testIndexName}_template_both_fields" + val policyID = "${testIndexName}_policy_template_both" + val rollup = + ISMRollup( + description = "rollup with both templates", + sourceIndex = "{{ctx.index}}", + targetIndex = "target_{{ctx.index}}_rollup", + targetIndexSettings = null, + pageSize = 100, + dimensions = + listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = + listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum(), Average()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = + Policy( + id = policyID, + description = "rollup policy with both templates", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + val sourceIndexMappingString = + "\"properties\": {\"tpep_pickup_datetime\": { \"type\": \"date\" }, \"RatecodeID\": { \"type\": " + + "\"keyword\" }, \"passenger_count\": { \"type\": \"integer\" }}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = sourceIndexMappingString) + + // Index test data + client().makeRequest( + "POST", + "/$indexName/_doc?refresh=true", + StringEntity( + """{"tpep_pickup_datetime":"2021-01-01T00:00:00Z","RatecodeID":"1","passenger_count":3}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Execute the rollup action and wait for completion + assertIndexRolledUp(indexName, policyID, rollup) + + // Verify the rollup index was created with the correct name + assertIndexExists("target_${indexName}_rollup") + } + + fun `test multi-tier rollup with templated source_index`() { + val rawIndexName = "${testIndexName}_multi_tier_raw" + val tier1PolicyID = "${testIndexName}_tier1_policy" + val tier2PolicyID = "${testIndexName}_tier2_policy" + + // First tier rollup: raw data -> tier1 rollup + val tier1Rollup = + ISMRollup( + description = "tier 1 rollup", + targetIndex = "rollup_tier1_{{ctx.index}}", + targetIndexSettings = null, + pageSize = 100, + dimensions = + listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = + listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum(), Average()), + ), + ), + ) + + val tier1ActionConfig = RollupAction(tier1Rollup, 0) + val tier1States = listOf(State("rollup", listOf(tier1ActionConfig), listOf())) + val tier1Policy = + Policy( + id = tier1PolicyID, + description = "tier 1 rollup policy", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = tier1States[0].name, + states = tier1States, + ) + val sourceIndexMappingString = + "\"properties\": {\"tpep_pickup_datetime\": { \"type\": \"date\" }, \"RatecodeID\": { \"type\": " + + "\"keyword\" }, \"passenger_count\": { \"type\": \"integer\" }}" + createPolicy(tier1Policy, tier1PolicyID) + createIndex(rawIndexName, tier1PolicyID, mapping = sourceIndexMappingString) + + // Index test data + client().makeRequest( + "POST", + "/$rawIndexName/_doc?refresh=true", + StringEntity( + """{"tpep_pickup_datetime":"2021-01-01T00:00:00Z","RatecodeID":"1","passenger_count":5}""", + ContentType.APPLICATION_JSON, + ), + ) + + // Execute tier 1 rollup and wait for completion + assertIndexRolledUp(rawIndexName, tier1PolicyID, tier1Rollup) + + val tier1TargetIndex = "rollup_tier1_$rawIndexName" + assertIndexExists(tier1TargetIndex) + + // Second tier rollup: tier1 rollup -> tier2 rollup with templated source_index + val tier2Rollup = + ISMRollup( + description = "tier 2 rollup with templated source", + sourceIndex = "{{ctx.index}}", + targetIndex = "rollup_tier2_{{ctx.index}}", + targetIndexSettings = null, + pageSize = 100, + dimensions = + listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "2h"), + Terms("RatecodeID", "RatecodeID"), + ), + metrics = + listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum()), + ), + ), + ) + + val tier2ActionConfig = RollupAction(tier2Rollup, 0) + val tier2States = listOf(State("rollup", listOf(tier2ActionConfig), listOf())) + val tier2Policy = + Policy( + id = tier2PolicyID, + description = "tier 2 rollup policy with templated source", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = tier2States[0].name, + states = tier2States, + ) + createPolicy(tier2Policy, tier2PolicyID) + + // Add policy to tier1 rollup index and execute tier 2 rollup + addPolicyToIndex(tier1TargetIndex, tier2PolicyID) + assertIndexRolledUp(tier1TargetIndex, tier2PolicyID, tier2Rollup) + + // Verify tier 2 rollup index was created + assertIndexExists("rollup_tier2_$tier1TargetIndex") + } + + fun `test rollup action fails when source equals target after resolution`() { + val indexName = "${testIndexName}_source_equals_target" + val policyID = "${testIndexName}_policy_source_equals_target" + val rollup = + ISMRollup( + description = "rollup with source equals target", + sourceIndex = "{{ctx.index}}", + targetIndex = "{{ctx.index}}", + targetIndexSettings = null, + pageSize = 100, + dimensions = + listOf( + DateHistogram(sourceField = "tpep_pickup_datetime", fixedInterval = "1h"), + ), + metrics = + listOf( + RollupMetrics( + sourceField = "passenger_count", + targetField = "passenger_count", + metrics = listOf(Sum()), + ), + ), + ) + + val actionConfig = RollupAction(rollup, 0) + val states = listOf(State("rollup", listOf(actionConfig), listOf())) + val policy = + Policy( + id = policyID, + description = "rollup policy with source equals target", + schemaVersion = 1L, + lastUpdatedTime = Instant.now().truncatedTo(ChronoUnit.MILLIS), + errorNotification = randomErrorNotification(), + defaultState = states[0].name, + states = states, + ) + val sourceIndexMappingString = + "\"properties\": {\"tpep_pickup_datetime\": { \"type\": \"date\" }, \"passenger_count\": { \"type\": \"integer\" }}" + createPolicy(policy, policyID) + createIndex(indexName, policyID, mapping = sourceIndexMappingString) + + val managedIndexConfig = getExistingManagedIndexConfig(indexName) + + // Change the start time so the job will trigger + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { assertEquals(policyID, getExplainManagedIndexMetaData(indexName).policyID) } + + // Speed up to second execution + updateManagedIndexConfigStartTime(managedIndexConfig) + waitFor { + val metadata = getExplainManagedIndexMetaData(indexName) + assertEquals("Step should fail", AttemptCreateRollupJobStep.name, metadata.stepMetaData?.name) + assertEquals("Step should be in failed state", "failed", metadata.stepMetaData?.stepStatus.toString()) + val info = metadata.info as? Map + val cause = info?.get("cause")?.toString() ?: "" + assertTrue( + "Error message should indicate source and target must be different, got: $cause", + cause.contains("Source and target") || cause.contains("same index") || cause.contains("cannot be the same"), + ) + } + } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/AttemptCreateRollupJobStepTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/AttemptCreateRollupJobStepTests.kt index a73dd9ce1..476b85fb1 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/AttemptCreateRollupJobStepTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/AttemptCreateRollupJobStepTests.kt @@ -5,8 +5,10 @@ package org.opensearch.indexmanagement.indexstatemanagement.step +import org.opensearch.indexmanagement.indexstatemanagement.action.RollupAction import org.opensearch.indexmanagement.indexstatemanagement.randomRollupActionConfig import org.opensearch.indexmanagement.indexstatemanagement.step.rollup.AttemptCreateRollupJobStep +import org.opensearch.indexmanagement.rollup.randomISMRollup import org.opensearch.indexmanagement.spi.indexstatemanagement.Step import org.opensearch.indexmanagement.spi.indexstatemanagement.model.ActionMetaData import org.opensearch.indexmanagement.spi.indexstatemanagement.model.ActionProperties @@ -38,4 +40,345 @@ class AttemptCreateRollupJobStepTests : OpenSearchTestCase() { fun `test isIdempotent`() { assertTrue(step.isIdempotent()) } + + fun `test source index resolution with explicit source_index`() { + // Create ISMRollup with explicit source_index + val explicitSourceIndex = "explicit_source_index" + val ismRollup = randomISMRollup().copy(sourceIndex = explicitSourceIndex) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create rollup using the managed index name + val managedIndexName = "managed_index" + val rollup = ismRollup.toRollup(managedIndexName) + + // Verify that the rollup uses the explicit source_index, not the managed index + // The rollup ID is generated from the resolved source index + val expectedRollupId = ismRollup.toRollup(managedIndexName).id + assertEquals("Rollup ID should be generated from explicit source_index", expectedRollupId, rollup.id) + + // Verify the sourceIndex field in the rollup matches the explicit source + assertEquals("Source index should match explicit source_index", explicitSourceIndex, rollup.sourceIndex) + } + + fun `test source index resolution without source_index uses managed index`() { + // Create ISMRollup without source_index (null) + val ismRollup = randomISMRollup().copy(sourceIndex = null) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create rollup using the managed index name + val managedIndexName = "managed_index" + val rollup = ismRollup.toRollup(managedIndexName) + + // Verify that the rollup uses the managed index as source + assertEquals("Source index should match managed index when source_index is null", managedIndexName, rollup.sourceIndex) + } + + fun `test source index resolution with empty source_index uses managed index`() { + // Create ISMRollup with null source_index + val ismRollup = randomISMRollup().copy(sourceIndex = null) + + // Create rollup using the managed index name + val managedIndexName = "my_managed_index" + val rollup = ismRollup.toRollup(managedIndexName) + + // Verify that the rollup uses the managed index as source (backward compatibility) + assertEquals("Source index should default to managed index", managedIndexName, rollup.sourceIndex) + } + + fun `test source index resolution with different source and managed indices`() { + // Create ISMRollup with explicit source_index different from managed index + val explicitSourceIndex = "rollup_1m_index" + val managedIndexName = "rollup_5m_index" + val ismRollup = randomISMRollup().copy(sourceIndex = explicitSourceIndex) + + // Create rollup + val rollup = ismRollup.toRollup(managedIndexName) + + // Verify that the rollup uses the explicit source_index + assertEquals("Source index should be the explicit source_index", explicitSourceIndex, rollup.sourceIndex) + assertNotEquals("Source index should not be the managed index", managedIndexName, rollup.sourceIndex) + } + + fun `test process failure includes source index in error context`() { + // Create ISMRollup with explicit source_index + val explicitSourceIndex = "source_index_that_fails" + val ismRollup = randomISMRollup().copy(sourceIndex = explicitSourceIndex) + val action = RollupAction(ismRollup = ismRollup, index = 0) + val step = AttemptCreateRollupJobStep(action) + + val managedIndexName = "managed_index" + val rollup = ismRollup.toRollup(managedIndexName) + + // Simulate failure + val errorMessage = "Source index not found" + step.processFailure(rollup.id, managedIndexName, Exception(errorMessage)) + + val metadata = ManagedIndexMetaData( + managedIndexName, "indexUuid", "policy_id", null, null, null, null, null, null, null, + ActionMetaData(AttemptCreateRollupJobStep.name, 1, 0, false, 0, null, ActionProperties(rollupId = rollup.id)), + null, null, null, + ) + + val updatedMetadata = step.getUpdatedManagedIndexMetadata(metadata) + + // Verify failure status and error message + assertEquals("Step status should be FAILED", Step.StepStatus.FAILED, updatedMetadata.stepMetaData?.stepStatus) + assertEquals( + "Error message should indicate failure", + AttemptCreateRollupJobStep.getFailedMessage(rollup.id, managedIndexName), + updatedMetadata.info?.get("message"), + ) + assertEquals("Error cause should be included", errorMessage, updatedMetadata.info?.get("cause")) + } + + fun `test rollup ID generation with explicit source_index`() { + // Create two ISMRollup configs with same settings but different source indices + val sourceIndex1 = "source_index_1" + val sourceIndex2 = "source_index_2" + val managedIndexName = "managed_index" + + val baseRollup = randomISMRollup() + val ismRollup1 = baseRollup.copy(sourceIndex = sourceIndex1) + val ismRollup2 = baseRollup.copy(sourceIndex = sourceIndex2) + + val rollup1 = ismRollup1.toRollup(managedIndexName) + val rollup2 = ismRollup2.toRollup(managedIndexName) + + // Verify that rollup IDs are different because source indices are different + assertNotEquals("Rollup IDs should differ when source indices differ", rollup1.id, rollup2.id) + + // Verify that source indices are correctly set + assertEquals("First rollup should use first source index", sourceIndex1, rollup1.sourceIndex) + assertEquals("Second rollup should use second source index", sourceIndex2, rollup2.sourceIndex) + } + + fun `test rollup ID generation without source_index uses managed index`() { + // Create ISMRollup without source_index + val ismRollup = randomISMRollup().copy(sourceIndex = null) + val managedIndexName = "managed_index" + + val rollup = ismRollup.toRollup(managedIndexName) + + // Verify that the rollup uses managed index as source + assertEquals("Rollup should use managed index as source", managedIndexName, rollup.sourceIndex) + + // Verify that rollup ID is generated from managed index + val expectedRollupId = ismRollup.toRollup(managedIndexName).id + assertEquals("Rollup ID should be consistent", expectedRollupId, rollup.id) + } + + fun `test source_index template resolution is called with templated source_index`() { + // Create ISMRollup with templated source_index + val templatedSourceIndex = "{{ctx.index}}" + val managedIndexName = "logs-2024-01" + val ismRollup = randomISMRollup().copy( + sourceIndex = templatedSourceIndex, + targetIndex = "rollup_{{ctx.index}}", + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create a temporary rollup to verify template resolution context + val tempRollup = ismRollup.toRollup(managedIndexName) + + // Verify that when toRollup is called with a managed index name, + // the source_index field in ISMRollup is used (not yet resolved) + assertEquals( + "ISMRollup should contain the template before resolution", + templatedSourceIndex, + ismRollup.sourceIndex, + ) + + // Verify that the rollup object created from ISMRollup uses the explicit source_index + // (In the actual step execution, this would be resolved by RollupFieldValueExpressionResolver) + assertEquals( + "Rollup should use the explicit source_index from ISMRollup", + templatedSourceIndex, + tempRollup.sourceIndex, + ) + + // Verify that the target_index also contains a template + assertEquals( + "Target index should contain template", + "rollup_{{ctx.index}}", + ismRollup.targetIndex, + ) + } + + fun `test source_index template resolution parameters for multi-tier rollup`() { + // Simulate a multi-tier rollup scenario where the managed index is itself a rollup + val firstTierRollupIndex = "rollup_tier1_logs-2024-01" + val templatedSourceIndex = "{{ctx.index}}" + val templatedTargetIndex = "rollup_tier2_{{ctx.index}}" + + val ismRollup = randomISMRollup().copy( + sourceIndex = templatedSourceIndex, + targetIndex = templatedTargetIndex, + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create a temporary rollup with the first-tier rollup index as the managed index + val tempRollup = ismRollup.toRollup(firstTierRollupIndex) + + // Verify that the ISMRollup contains templates + assertEquals( + "Source index should contain template", + templatedSourceIndex, + ismRollup.sourceIndex, + ) + assertEquals( + "Target index should contain template", + templatedTargetIndex, + ismRollup.targetIndex, + ) + + // Verify that the temporary rollup has the templated source_index + // (before resolution by RollupFieldValueExpressionResolver) + assertEquals( + "Temporary rollup should have templated source_index", + templatedSourceIndex, + tempRollup.sourceIndex, + ) + } + + fun `test source_index and target_index template resolution parameters`() { + // Create ISMRollup with templates in both source_index and target_index + val managedIndexName = "my-index-2024-01" + val sourceTemplate = "{{ctx.index}}" + val targetTemplate = "rollup_{{ctx.index}}" + + val ismRollup = randomISMRollup().copy( + sourceIndex = sourceTemplate, + targetIndex = targetTemplate, + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create temporary rollup for template resolution context + val tempRollup = ismRollup.toRollup(managedIndexName) + + // Verify that both fields contain templates before resolution + assertEquals( + "Source index should contain template", + sourceTemplate, + ismRollup.sourceIndex, + ) + assertEquals( + "Target index should contain template", + targetTemplate, + ismRollup.targetIndex, + ) + + // Verify that the temporary rollup is created with the templated values + assertEquals( + "Temporary rollup should have templated source_index", + sourceTemplate, + tempRollup.sourceIndex, + ) + assertEquals( + "Temporary rollup should have templated target_index", + targetTemplate, + tempRollup.targetIndex, + ) + } + + fun `test target_index template resolution is called with correct parameters`() { + // Create ISMRollup with templated target_index + val managedIndexName = "logs-2024-01" + val targetTemplate = "rollup_{{ctx.index}}" + val ismRollup = randomISMRollup().copy( + sourceIndex = null, // Use default (managed index) + targetIndex = targetTemplate, + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create temporary rollup for template resolution context + val tempRollup = ismRollup.toRollup(managedIndexName) + + // Verify that target_index contains template before resolution + assertEquals( + "Target index should contain template", + targetTemplate, + ismRollup.targetIndex, + ) + + // Verify that the temporary rollup is created with the templated target_index + assertEquals( + "Temporary rollup should have templated target_index", + targetTemplate, + tempRollup.targetIndex, + ) + + // Verify that source_index defaults to managed index when null + assertEquals( + "Source index should default to managed index", + managedIndexName, + tempRollup.sourceIndex, + ) + } + + fun `test target_index template resolution with complex template`() { + // Create ISMRollup with complex target_index template using multiple variables + val managedIndexName = "rollup_tier1_logs-2024-01" + val targetTemplate = "rollup_tier2_{{ctx.index}}_aggregated" + val ismRollup = randomISMRollup().copy( + sourceIndex = "{{ctx.index}}", + targetIndex = targetTemplate, + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create temporary rollup for template resolution context + val tempRollup = ismRollup.toRollup(managedIndexName) + + // Verify that target_index contains complex template + assertEquals( + "Target index should contain complex template", + targetTemplate, + ismRollup.targetIndex, + ) + + // Verify that the temporary rollup is created with the templated target_index + assertEquals( + "Temporary rollup should have templated target_index", + targetTemplate, + tempRollup.targetIndex, + ) + } + + fun `test target_index template resolution with source_index template`() { + // Create ISMRollup where both source_index and target_index use templates + val managedIndexName = "logs-2024-01" + val sourceTemplate = "{{ctx.index}}" + val targetTemplate = "rollup_{{ctx.source_index}}" + val ismRollup = randomISMRollup().copy( + sourceIndex = sourceTemplate, + targetIndex = targetTemplate, + ) + val action = RollupAction(ismRollup = ismRollup, index = 0) + + // Create temporary rollup for template resolution context + val tempRollup = ismRollup.toRollup(managedIndexName) + + // Verify that both fields contain templates + assertEquals( + "Source index should contain template", + sourceTemplate, + ismRollup.sourceIndex, + ) + assertEquals( + "Target index should contain template with source_index variable", + targetTemplate, + ismRollup.targetIndex, + ) + + // Verify that the temporary rollup is created with templated values + assertEquals( + "Temporary rollup should have templated source_index", + sourceTemplate, + tempRollup.sourceIndex, + ) + assertEquals( + "Temporary rollup should have templated target_index", + targetTemplate, + tempRollup.targetIndex, + ) + } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollupTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollupTests.kt index 0799d71ad..96d665120 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollupTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollupTests.kt @@ -6,10 +6,15 @@ package org.opensearch.indexmanagement.rollup.model import org.apache.commons.codec.digest.DigestUtils +import org.opensearch.Version +import org.opensearch.common.io.stream.BytesStreamOutput +import org.opensearch.common.xcontent.XContentType +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.index.seqno.SequenceNumbers import org.opensearch.indexmanagement.rollup.randomDateHistogram import org.opensearch.indexmanagement.rollup.randomISMRollup import org.opensearch.indexmanagement.rollup.randomTerms +import org.opensearch.indexmanagement.rollup.toJsonString import org.opensearch.indexmanagement.util.IndexUtils import org.opensearch.jobscheduler.spi.schedule.IntervalSchedule import org.opensearch.test.OpenSearchTestCase @@ -90,4 +95,213 @@ class ISMRollupTests : OpenSearchTestCase() { assertTrue(rollup.roles.isEmpty()) assertTrue(rollup.isEnabled) } + + fun `test ism rollup serialization without source index`() { + val ismRollup = randomISMRollup().copy(sourceIndex = null) + val jsonString = ismRollup.toJsonString() + + // Verify source_index is not present in JSON when null + assertFalse(jsonString.contains("source_index")) + + // Parse back and verify + val parser = createParser(XContentType.JSON.xContent(), jsonString) + parser.nextToken() + val parsed = ISMRollup.parse(parser) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + assertEquals(ismRollup.dimensions, parsed.dimensions) + assertEquals(ismRollup.metrics, parsed.metrics) + assertNull(parsed.sourceIndex) + } + + fun `test ism rollup serialization with source index`() { + val sourceIndex = "my-source-index" + val ismRollup = randomISMRollup().copy(sourceIndex = sourceIndex) + val jsonString = ismRollup.toJsonString() + + // Verify source_index is present in JSON + assertTrue(jsonString.contains("source_index")) + assertTrue(jsonString.contains(sourceIndex)) + + // Parse back and verify + val parser = createParser(XContentType.JSON.xContent(), jsonString) + parser.nextToken() + val parsed = ISMRollup.parse(parser) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + assertEquals(ismRollup.dimensions, parsed.dimensions) + assertEquals(ismRollup.metrics, parsed.metrics) + assertEquals(sourceIndex, parsed.sourceIndex) + } + + fun `test ism rollup deserialization without source index field`() { + val ismRollup = randomISMRollup() + val jsonString = ismRollup.toJsonString() + + // Manually remove source_index if present to simulate old format + val cleanedJson = jsonString.replace(Regex(",?\"source_index\":\"[^\"]*\""), "") + + val parser = createParser(XContentType.JSON.xContent(), cleanedJson) + parser.nextToken() + val parsed = ISMRollup.parse(parser) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + assertNull(parsed.sourceIndex) + } + + fun `test ism rollup stream serialization without source index on V3_0_0`() { + val ismRollup = randomISMRollup().copy(sourceIndex = null) + + val output = BytesStreamOutput() + output.version = Version.V_3_0_0 + ismRollup.writeTo(output) + + val input = StreamInput.wrap(output.bytes().toBytesRef().bytes) + input.version = Version.V_3_0_0 + val parsed = ISMRollup(input) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + assertEquals(ismRollup.dimensions, parsed.dimensions) + assertEquals(ismRollup.metrics, parsed.metrics) + assertNull(parsed.sourceIndex) + } + + fun `test ism rollup stream serialization with source index on V3_0_0`() { + val sourceIndex = "my-source-index" + val ismRollup = randomISMRollup().copy(sourceIndex = sourceIndex) + + val output = BytesStreamOutput() + output.version = Version.V_3_0_0 + ismRollup.writeTo(output) + + val input = StreamInput.wrap(output.bytes().toBytesRef().bytes) + input.version = Version.V_3_0_0 + val parsed = ISMRollup(input) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + assertEquals(ismRollup.dimensions, parsed.dimensions) + assertEquals(ismRollup.metrics, parsed.metrics) + assertEquals(sourceIndex, parsed.sourceIndex) + } + + fun `test ism rollup stream serialization backward compatibility with V2_18_0`() { + val ismRollup = randomISMRollup().copy(sourceIndex = "my-source-index") + + // Write with V2_18_0 (before source_index support) + val output = BytesStreamOutput() + output.version = Version.V_2_18_0 + ismRollup.writeTo(output) + + // Read with V2_18_0 - source_index should not be written + val input = StreamInput.wrap(output.bytes().toBytesRef().bytes) + input.version = Version.V_2_18_0 + val parsed = ISMRollup(input) + + assertEquals(ismRollup.description, parsed.description) + assertEquals(ismRollup.targetIndex, parsed.targetIndex) + assertEquals(ismRollup.pageSize, parsed.pageSize) + // sourceIndex should be null when reading from older version + assertNull(parsed.sourceIndex) + } + + fun `test ism rollup toRollup with explicit source index`() { + val explicitSourceIndex = "explicit-source-index" + val managedIndex = "managed-index" + val ismRollup = randomISMRollup().copy(sourceIndex = explicitSourceIndex) + + val rollup = ismRollup.toRollup(managedIndex) + + // Should use explicit source_index, not managed index + assertEquals(explicitSourceIndex, rollup.sourceIndex) + assertEquals(ismRollup.targetIndex, rollup.targetIndex) + + // Verify ID is computed with explicit source index + val expectedId = DigestUtils.sha1Hex(explicitSourceIndex + ismRollup.toString()) + assertEquals(expectedId, rollup.id) + } + + fun `test ism rollup toRollup without explicit source index uses managed index`() { + val managedIndex = "managed-index" + val ismRollup = randomISMRollup().copy(sourceIndex = null) + + val rollup = ismRollup.toRollup(managedIndex) + + // Should use managed index as fallback + assertEquals(managedIndex, rollup.sourceIndex) + assertEquals(ismRollup.targetIndex, rollup.targetIndex) + + // Verify ID is computed with managed index + val expectedId = DigestUtils.sha1Hex(managedIndex + ismRollup.toString()) + assertEquals(expectedId, rollup.id) + } + + fun `test ism rollup null source index handling`() { + val ismRollup = randomISMRollup().copy(sourceIndex = null) + + // Should not throw exception with null source index + assertNull(ismRollup.sourceIndex) + + // Should serialize and deserialize correctly + val jsonString = ismRollup.toJsonString() + val parser = createParser(XContentType.JSON.xContent(), jsonString) + parser.nextToken() + val parsed = ISMRollup.parse(parser) + + assertNull(parsed.sourceIndex) + } + + fun `test ism rollup with empty string source index is allowed`() { + // Empty string is technically allowed by the model (validation happens elsewhere) + val ismRollup = randomISMRollup().copy(sourceIndex = "") + + assertEquals("", ismRollup.sourceIndex) + + // Should serialize and deserialize correctly + val jsonString = ismRollup.toJsonString() + val parser = createParser(XContentType.JSON.xContent(), jsonString) + parser.nextToken() + val parsed = ISMRollup.parse(parser) + + assertEquals("", parsed.sourceIndex) + } + + fun `test ism rollup round trip serialization with source index`() { + val sourceIndex = "test-source-index" + val original = randomISMRollup().copy(sourceIndex = sourceIndex) + + // XContent round trip + val jsonString = original.toJsonString() + val parser = createParser(XContentType.JSON.xContent(), jsonString) + parser.nextToken() + val parsedFromXContent = ISMRollup.parse(parser) + + assertEquals(original.description, parsedFromXContent.description) + assertEquals(original.targetIndex, parsedFromXContent.targetIndex) + assertEquals(original.sourceIndex, parsedFromXContent.sourceIndex) + assertEquals(original.pageSize, parsedFromXContent.pageSize) + + // Stream round trip + val output = BytesStreamOutput() + output.version = Version.V_3_0_0 + original.writeTo(output) + + val input = StreamInput.wrap(output.bytes().toBytesRef().bytes) + input.version = Version.V_3_0_0 + val parsedFromStream = ISMRollup(input) + + assertEquals(original.description, parsedFromStream.description) + assertEquals(original.targetIndex, parsedFromStream.targetIndex) + assertEquals(original.sourceIndex, parsedFromStream.sourceIndex) + assertEquals(original.pageSize, parsedFromStream.pageSize) + } } diff --git a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolverTests.kt b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolverTests.kt index 8f1a97ae9..5ceb13982 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolverTests.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolverTests.kt @@ -54,4 +54,104 @@ class RollupFieldValueExpressionResolverTests : OpenSearchTestCase() { val targetIndexResolvedName = RollupFieldValueExpressionResolver.resolve(rollup, rollup.targetIndex) assertEquals("{{ctx.source_index}}", targetIndexResolvedName) } + + fun `test resolving ctx index in source_index template`() { + val managedIndexName = "logs-2024-01" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(managedIndexName)) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = "test_index_123") + val resolvedSourceIndex = RollupFieldValueExpressionResolver.resolve(rollup, "{{ctx.index}}", managedIndexName) + assertEquals(managedIndexName, resolvedSourceIndex) + } + + fun `test resolving ctx index in target_index template`() { + val managedIndexName = "logs-2024-01" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory("rollup_$managedIndexName")) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = "test_index_123") + val resolvedTargetIndex = RollupFieldValueExpressionResolver.resolve(rollup, "rollup_{{ctx.index}}", managedIndexName) + assertEquals("rollup_$managedIndexName", resolvedTargetIndex) + } + + fun `test resolving ctx index with different managed index names`() { + val managedIndexNames = listOf("index-1", "index-2", "data-stream-000001") + managedIndexNames.forEach { managedIndexName -> + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(managedIndexName)) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = "test_index") + val resolvedIndex = RollupFieldValueExpressionResolver.resolve(rollup, "{{ctx.index}}", managedIndexName) + assertEquals(managedIndexName, resolvedIndex) + } + } + + fun `test resolving ctx source_index in source_index template`() { + val sourceIndexName = "raw-data-index" + val managedIndexName = "managed-index" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(sourceIndexName)) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = sourceIndexName) + val resolvedSourceIndex = RollupFieldValueExpressionResolver.resolve(rollup, "{{ctx.source_index}}", managedIndexName) + assertEquals(sourceIndexName, resolvedSourceIndex) + } + + fun `test resolving ctx source_index in target_index template`() { + val sourceIndexName = "raw-data-index" + val managedIndexName = "managed-index" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory("rollup_$sourceIndexName")) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = sourceIndexName) + val resolvedTargetIndex = RollupFieldValueExpressionResolver.resolve(rollup, "rollup_{{ctx.source_index}}", managedIndexName) + assertEquals("rollup_$sourceIndexName", resolvedTargetIndex) + } + + fun `test resolving template with both ctx index and ctx source_index`() { + val sourceIndexName = "raw-data" + val managedIndexName = "logs-2024-01" + val expectedResolved = "rollup_${managedIndexName}_from_$sourceIndexName" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(expectedResolved)) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = sourceIndexName) + val resolvedIndex = RollupFieldValueExpressionResolver.resolve( + rollup, + "rollup_{{ctx.index}}_from_{{ctx.source_index}}", + managedIndexName, + ) + assertEquals(expectedResolved, resolvedIndex) + } + + fun `test resolving template with ctx index for multi-tier rollup`() { + // Simulates second tier rollup where managed index is the first tier rollup output + val managedIndexName = "rollup_tier1_logs-2024-01" + val expectedResolved = "rollup_tier2_$managedIndexName" + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(expectedResolved)) + whenever(indexAliasUtils.isAlias(anyString())).doReturn(false) + val rollup = randomRollup().copy(sourceIndex = managedIndexName) + val resolvedTargetIndex = RollupFieldValueExpressionResolver.resolve( + rollup, + "rollup_tier2_{{ctx.index}}", + managedIndexName, + ) + assertEquals(expectedResolved, resolvedTargetIndex) + } + + fun `test alias resolution with templated source_index`() { + // Test where the resolved value from template is an alias + val managedIndexName = "logs-alias" + val backingIndexName = "logs-2024-01-backing" + + // Mock the template to resolve to the alias name + whenever(scriptService.compile(any(), eq(TemplateScript.CONTEXT))).doReturn(TestTemplateService.MockTemplateScript.Factory(managedIndexName)) + + // Mock that the resolved value is an alias + whenever(indexAliasUtils.isAlias(eq(managedIndexName))).doReturn(true) + + // Mock the write index resolution + whenever(indexAliasUtils.getWriteIndexNameForAlias(eq(managedIndexName))).doReturn(backingIndexName) + + val rollup = randomRollup().copy(sourceIndex = "test_index") + val resolvedSourceIndex = RollupFieldValueExpressionResolver.resolve(rollup, "{{ctx.index}}", managedIndexName) + + // Verify it resolves to the backing write index, not the alias + assertEquals(backingIndexName, resolvedSourceIndex) + } } From 5e510cc85009b1c163cb373e889c844e4c08267a Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Tue, 2 Dec 2025 15:02:58 +0530 Subject: [PATCH 5/6] Upgrading the schema in the cached config file Signed-off-by: Kshitij Tandon --- .../indexmanagement/IndexManagementRestTestCase.kt | 2 +- .../mappings/cached-opendistro-ism-config.json | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/test/kotlin/org/opensearch/indexmanagement/IndexManagementRestTestCase.kt b/src/test/kotlin/org/opensearch/indexmanagement/IndexManagementRestTestCase.kt index 4a18d5c9a..ee9d4d2a5 100644 --- a/src/test/kotlin/org/opensearch/indexmanagement/IndexManagementRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/indexmanagement/IndexManagementRestTestCase.kt @@ -41,7 +41,7 @@ import javax.management.remote.JMXConnectorFactory import javax.management.remote.JMXServiceURL abstract class IndexManagementRestTestCase : ODFERestTestCase() { - val configSchemaVersion = 24 + val configSchemaVersion = 25 val historySchemaVersion = 7 // Having issues with tests leaking into other tests and mappings being incorrect and they are not caught by any pending task wait check as diff --git a/src/test/resources/mappings/cached-opendistro-ism-config.json b/src/test/resources/mappings/cached-opendistro-ism-config.json index 6250ba15d..ea5e7d5e0 100644 --- a/src/test/resources/mappings/cached-opendistro-ism-config.json +++ b/src/test/resources/mappings/cached-opendistro-ism-config.json @@ -1,6 +1,6 @@ { "_meta" : { - "schema_version": 24 + "schema_version": 25 }, "dynamic": "strict", "properties": { @@ -374,6 +374,15 @@ "description": { "type": "text" }, + "source_index": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, "target_index": { "type": "text", "fields": { From c266f5bcf8ed5142348a0c144327ff11018e2b90 Mon Sep 17 00:00:00 2001 From: Kshitij Tandon Date: Tue, 2 Dec 2025 19:54:19 +0530 Subject: [PATCH 6/6] Removing some redundant comments Signed-off-by: Kshitij Tandon --- .../step/rollup/AttemptCreateRollupJobStep.kt | 10 ---------- .../rollup/interceptor/RollupInterceptor.kt | 5 +---- .../indexmanagement/rollup/model/ISMRollup.kt | 4 ---- .../rollup/util/RollupFieldValueExpressionResolver.kt | 6 ------ 4 files changed, 1 insertion(+), 24 deletions(-) diff --git a/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt b/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt index 5bd089774..048ff1a2e 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/indexstatemanagement/step/rollup/AttemptCreateRollupJobStep.kt @@ -60,9 +60,6 @@ class AttemptCreateRollupJobStep(private val action: RollupAction) : Step(name) } // Resolve target_index template. - // Common patterns: - // - "rollup_{{ctx.index}}" -> "rollup_logs-2024-01" - // - "rollup_tier2_{{ctx.index}}" -> "rollup_tier2_rollup_tier1_logs-2024-01" val resolvedTargetIndex = RollupFieldValueExpressionResolver.resolve( tempRollup, action.ismRollup.targetIndex, @@ -79,8 +76,6 @@ class AttemptCreateRollupJobStep(private val action: RollupAction) : Step(name) ) // Create the final rollup job with resolved source_index and target_index. - // The copy() ensures that template variables are replaced with actual index names - // before the rollup job is persisted and executed. val rollup = action.ismRollup.toRollup(indexName, context.user) .copy(sourceIndex = resolvedSourceIndex, targetIndex = resolvedTargetIndex) rollupId = rollup.id @@ -123,11 +118,6 @@ class AttemptCreateRollupJobStep(private val action: RollupAction) : Step(name) /** * Validates that resolved source and target indices are valid and different. * - * This validation ensures that: - * 1. The resolved source_index is not empty or whitespace-only - * 2. The resolved target_index is not empty or whitespace-only - * 3. The source and target indices are different (prevents self-rollup) - * * @param sourceIndex The resolved source index name (after template resolution) * @param targetIndex The resolved target index name (after template resolution) * @throws IllegalArgumentException if any validation rule fails, with a descriptive error message 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 2fb9c2e6c..b8d747899 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/interceptor/RollupInterceptor.kt @@ -69,7 +69,6 @@ class RollupInterceptor( /** * Bypass level that skips all rollup search validations and rewriting. * Used when the system needs to query rollup indices directly using composite aggregation. - * Passed via FetchSourceContext marker for multi-node support. */ const val BYPASS_ROLLUP_SEARCH = 1 @@ -77,7 +76,6 @@ class RollupInterceptor( * Bypass level that allows non-zero size in rollup searches. * Used for internal operations like continuous rollup initialization that need to fetch * actual documents from rollup indices (e.g., getEarliestTimestampFromRollupIndex). - * Passed via FetchSourceContext marker for multi-node support. */ const val BYPASS_SIZE_CHECK = 2 } @@ -85,7 +83,6 @@ class RollupInterceptor( /** * Reads the bypass value from the request's FetchSourceContext. * Returns the bypass level if the marker is present in includes array, null otherwise. - * This enables bypass mechanism to work in multi-node clusters. */ internal fun getBypassFromFetchSource(request: ShardSearchRequest): Int? { val includes = request.source()?.fetchSource()?.includes() @@ -121,7 +118,7 @@ class RollupInterceptor( val index = request.shardId().indexName val isRollupIndex = isRollupIndex(index, clusterService.state()) if (isRollupIndex) { - // Check bypass from FetchSourceContext for multi-node support + // Check bypass from FetchSourceContext val bypassLevel = getBypassFromFetchSource(request) logger.debug("RollupInterceptor bypass check - bypassLevel: $bypassLevel") diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt index d4cc95655..48bf56d1e 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/model/ISMRollup.kt @@ -74,10 +74,6 @@ data class ISMRollup( /** * Converts ISMRollup configuration to a Rollup job. * - * Source index resolution priority: - * 1. If ISMRollup.sourceIndex is set, use it (for multi-tier rollups) - * 2. Otherwise, use the sourceIndex parameter (managed index from ISM context) - * * @param sourceIndex The managed index from ISM context (fallback if ISMRollup.sourceIndex is null) * @param user Optional user context for the rollup job * @return Rollup job configuration diff --git a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt index 4d2de9a5f..fc5c193ed 100644 --- a/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt +++ b/src/main/kotlin/org/opensearch/indexmanagement/rollup/util/RollupFieldValueExpressionResolver.kt @@ -26,7 +26,6 @@ object RollupFieldValueExpressionResolver { /** * Resolves template variables in a field value using the rollup object as context. - * This method is kept for backward compatibility and only supports {{ctx.source_index}}. * * @param rollup The rollup object providing context for template resolution * @param fieldValue The field value that may contain template variables (e.g., "{{ctx.source_index}}") @@ -57,19 +56,14 @@ object RollupFieldValueExpressionResolver { * Resolves template variables in a field value using the rollup object and managed index name as context. * This is the primary method used by ISM rollup actions to resolve source_index and target_index templates. * - * This method extends the basic resolve() by adding the managed index name to the template context, - * enabling the use of {{ctx.index}} in addition to {{ctx.source_index}}. - * * @param rollup The rollup object providing context for template resolution. The rollup's source_index * field is made available as {{ctx.source_index}} in templates. * @param fieldValue The field value that may contain Mustache template variables. Common patterns: * - "{{ctx.index}}" - resolves to the managed index name * - "{{ctx.source_index}}" - resolves to the rollup's source index - * - "rollup_{{ctx.index}}" - adds a prefix to the managed index name * - Literal values without templates are returned unchanged * @param managedIndexName The name of the index being managed by ISM. This is the index to which * the ISM policy is applied, and it's made available as {{ctx.index}} in templates. - * For multi-tier rollups, this would be the output of the previous tier. * @return The resolved field value with all template variables replaced. If the resolved value is an alias, * it's automatically resolved to the write index name. If resolution produces an empty or null result, * the original fieldValue is returned unchanged.