From 41d5949b3c7f69604231fc14c168321023676921 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Wed, 8 Jan 2025 14:04:02 -0800 Subject: [PATCH 01/21] add a feature that flattens custom result index when enabled Signed-off-by: Jackie Han --- gradle.properties | 2 +- gradle/wrapper/gradle-wrapper.properties | 2 +- .../ad/indices/ADIndexManagement.java | 1 + .../ad/settings/AnomalyDetectorSettings.java | 1 + .../ADResultBulkTransportAction.java | 77 ++++-- .../timeseries/indices/IndexManagement.java | 42 ++++ .../AbstractTimeSeriesActionHandler.java | 224 +++++++++++++++++- .../transport/ResultBulkTransportAction.java | 1 + .../mappings/anomaly-results-flattened.json | 173 ++++++++++++++ 9 files changed, 492 insertions(+), 31 deletions(-) create mode 100644 src/main/resources/mappings/anomaly-results-flattened.json diff --git a/gradle.properties b/gradle.properties index d2eba77fc..afebc3db1 100644 --- a/gradle.properties +++ b/gradle.properties @@ -27,4 +27,4 @@ systemProp.org.gradle.warning.mode=fail systemProp.jdk.tls.client.protocols=TLSv1.2 # jvm args for faster test execution by default -systemProp.tests.jvm.argline=-XX:TieredStopAtLevel=1 -XX:ReservedCodeCacheSize=64m +systemProp.tests.jvm.argline=-XX:TieredStopAtLevel=1 -XX:ReservedCodeCacheSize=64m \ No newline at end of file diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 1af9e0930..7cf08140d 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -4,4 +4,4 @@ distributionUrl=https\://services.gradle.org/distributions/gradle-8.5-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME -zipStorePath=wrapper/dists +zipStorePath=wrapper/dists \ No newline at end of file diff --git a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java index 161044d29..2ee771e14 100644 --- a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java +++ b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java @@ -19,6 +19,7 @@ import static org.opensearch.ad.settings.AnomalyDetectorSettings.ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE; import static org.opensearch.ad.settings.AnomalyDetectorSettings.ANOMALY_RESULTS_INDEX_MAPPING_FILE; import static org.opensearch.ad.settings.AnomalyDetectorSettings.CHECKPOINT_INDEX_MAPPING_FILE; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE; import java.io.IOException; import java.util.EnumMap; diff --git a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java index ffd88ae9d..e1235707d 100644 --- a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java +++ b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java @@ -190,6 +190,7 @@ private AnomalyDetectorSettings() {} ); public static final String ANOMALY_RESULTS_INDEX_MAPPING_FILE = "mappings/anomaly-results.json"; + public static final String FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE = "mappings/anomaly-results-flattened.json"; public static final String ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE = "mappings/anomaly-detection-state.json"; public static final String CHECKPOINT_INDEX_MAPPING_FILE = "mappings/anomaly-checkpoint.json"; diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index 2de6b07e3..205b63a37 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -39,6 +39,8 @@ public class ADResultBulkTransportAction extends ResultBulkTransportAction { private static final Logger LOG = LogManager.getLogger(ADResultBulkTransportAction.class); + private final ClusterService clusterService; + private final Client client; @Inject public ADResultBulkTransportAction( @@ -61,39 +63,82 @@ public ADResultBulkTransportAction( ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, ADResultBulkRequest::new ); + this.clusterService = clusterService; + this.client = client; clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_INDEX_PRESSURE_SOFT_LIMIT, it -> softLimit = it); clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_INDEX_PRESSURE_HARD_LIMIT, it -> hardLimit = it); } + /** + * Prepares a {@link BulkRequest} for indexing anomaly detection results. + * + * This method processes a list of anomaly detection results provided in the {@link ADResultBulkRequest}. + * Each result is evaluated based on the current indexing pressure and result priority. If a flattened + * result index exists for the result, the result is also added to the flattened index. + * + * @param indexingPressurePercent the current percentage of indexing pressure. This value influences + * whether a result is indexed based on predefined thresholds and probabilities. + * @param request the {@link ADResultBulkRequest} containing anomaly detection results + * to be processed. + * @return a {@link BulkRequest} containing all results that are eligible for indexing. + * + *

Behavior:

+ *
    + *
  • Results are added to the bulk request if the indexing pressure is within acceptable limits + * or the result has high priority.
  • + *
  • If a flattened result index exists for a result, it is added to the flattened index in addition + * to the primary index.
  • + *
+ * + *

Indexing Pressure Thresholds:

+ *
    + *
  • Below the soft limit: All results are added.
  • + *
  • Between the soft limit and the hard limit: High-priority results are always added, and + * other results are added based on a probability that decreases with increasing pressure.
  • + *
  • Above the hard limit: Only high-priority results are added.
  • + *
+ * + * @see ADResultBulkRequest + * @see BulkRequest + * @see ADResultWriteRequest + */ @Override protected BulkRequest prepareBulkRequest(float indexingPressurePercent, ADResultBulkRequest request) { BulkRequest bulkRequest = new BulkRequest(); List results = request.getResults(); - if (indexingPressurePercent <= softLimit) { - for (ADResultWriteRequest resultWriteRequest : results) { - addResult(bulkRequest, resultWriteRequest.getResult(), resultWriteRequest.getResultIndex()); + for (ADResultWriteRequest resultWriteRequest : results) { + AnomalyResult result = resultWriteRequest.getResult(); + String resultIndex = resultWriteRequest.getResultIndex(); + + if (shouldAddResult(indexingPressurePercent, result)) { + addResult(bulkRequest, result, resultIndex); + addToFlattenedIndexIfExists(bulkRequest, result, resultIndex); } + } + + return bulkRequest; + } + + private boolean shouldAddResult(float indexingPressurePercent, AnomalyResult result) { + if (indexingPressurePercent <= softLimit) { + // Always add when below soft limit + return true; } else if (indexingPressurePercent <= hardLimit) { // exceed soft limit (60%) but smaller than hard limit (90%) float acceptProbability = 1 - indexingPressurePercent; - for (ADResultWriteRequest resultWriteRequest : results) { - AnomalyResult result = resultWriteRequest.getResult(); - if (result.isHighPriority() || random.nextFloat() < acceptProbability) { - addResult(bulkRequest, result, resultWriteRequest.getResultIndex()); - } - } + return result.isHighPriority() || random.nextFloat() < acceptProbability; } else { // if exceeding hard limit, only index non-zero grade or error result - for (ADResultWriteRequest resultWriteRequest : results) { - AnomalyResult result = resultWriteRequest.getResult(); - if (result.isHighPriority()) { - addResult(bulkRequest, result, resultWriteRequest.getResultIndex()); - } - } + return result.isHighPriority(); } + } - return bulkRequest; + private void addToFlattenedIndexIfExists(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { + String flattenedResultIndexName = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(); + if (clusterService.state().metadata().hasIndex(flattenedResultIndexName)) { + addResult(bulkRequest, result, flattenedResultIndexName); + } } private void addResult(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 6e47c8838..1ac646b4e 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -11,6 +11,7 @@ package org.opensearch.timeseries.indices; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE; import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; import static org.opensearch.timeseries.util.RestHandlerUtils.createXContentParserFromRegistry; @@ -1008,6 +1009,47 @@ public void initCustomResultIndexAndExecute(String resultIndexOrAlias, Execu } } + /** + * creates flattened result index + * @param indexName the index name + * @param actionListener the action listener + * @throws IOException + */ + public void initFlattenedResultIndex(String indexName, ActionListener actionListener) throws IOException { + logger.info("Initializing flattened result index: {}", indexName); + + CreateIndexRequest request = new CreateIndexRequest(indexName) + .mapping(getFlattenedResultIndexMappings(), XContentType.JSON) + .settings(settings); + choosePrimaryShards(request, false); + + adminClient.indices().create(request, ActionListener.wrap( + response -> { + if (response.isAcknowledged()) { + logger.info("Successfully created flattened result index: {}", indexName); + actionListener.onResponse(response); + } else { + String errorMsg = "Index creation not acknowledged for index: " + indexName; + logger.error(errorMsg); + actionListener.onFailure(new IllegalStateException(errorMsg)); + } + }, + exception -> { + logger.error("Failed to create flattened result index: {}", indexName, exception); + actionListener.onFailure(exception); + } + )); + } + + /** + * Get flattened result index mapping json content + * @return flattened result index mapping + * @throws IOException + */ + public String getFlattenedResultIndexMappings() throws IOException { + return getMappings(FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE); + } + public void validateCustomIndexForBackendJob( String resultIndexOrAlias, String securityLogId, diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 8cc9675a6..0c3532092 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -34,6 +34,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.WriteRequest; import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.ad.transport.IndexAnomalyDetectorResponse; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -398,22 +399,219 @@ protected void validateTimeField(boolean indexingDryRun, ActionListener liste } /** - * Prepare for indexing a new config. - * @param indexingDryRun if this is dryrun for indexing; when validation, it is true; when create/update, it is false + * Prepares for indexing a new configuration. + * + * This method handles the preparation of indexing a configuration, either during validation (dry run) + * or for create/update operations. It supports both PUT and POST REST request methods. + * + * @param indexingDryRun indicates whether this is a dry run for indexing. + * If {@code true}, the operation performs validation without creating/updating the configuration. + * If {@code false}, the configuration is created or updated. + * @param listener the {@link ActionListener} to handle the response or failure of the operation. + * @throws IOException if an I/O error occurs during the operation. + * + *

Behavior:

+ *
    + *
  • For {@code RestRequest.Method.PUT}: Validates that the job is not already running before proceeding + * with updating the configuration. It updates the configuration and manages the result index mapping + * if necessary.
  • + *
  • For {@code RestRequest.Method.POST}: Creates a new configuration. If a custom result index or alias is specified: + *
      + *
    • If flattening of the result index mapping is enabled, it initializes a flattened result index, + * sets up an ingest pipeline, and updates the flattened result index settings to bind the ingest pipeline + * with the flattened result index, enabling the writing of flattened nested fields into the flattened result index.
    • + *
    • If flattening is not enabled, directly returns the creation response.
    • + *
    + * If no custom result index or alias is specified, returns the creation response directly.
  • + *
+ * + *

Notes:

+ *
    + *
  • If the configuration has a custom result index or alias and flattening is enabled, + * the flattened result index name is suffixed with the detector ID in lowercase.
  • + *
  • The ingest pipeline ID is uniquely generated based on the detector ID in lowercase.
  • + *
+ * + *

Exceptions:

+ *
    + *
  • If the {@code createConfigResponse} is of an unexpected type, which indicates create config call has failed, + * then an {@link IllegalStateException} is thrown.
  • + *
*/ protected void prepareConfigIndexing(boolean indexingDryRun, ActionListener listener) { if (method == RestRequest.Method.PUT) { - handler - .confirmJobRunning( - clusterService, - client, - id, - listener, - () -> updateConfig(id, indexingDryRun, listener), - xContentRegistry - ); + handlePutRequest(indexingDryRun, listener); + } else { + handlePostRequest(indexingDryRun, listener); + } + } + + private void handlePutRequest(boolean indexingDryRun, ActionListener listener) { + handler.confirmJobRunning(clusterService, client, id, listener, () -> { + handleFlattenResultIndexMappingUpdate(listener); + updateConfig(id, indexingDryRun, listener); + }, xContentRegistry); + } + + private void handlePostRequest(boolean indexingDryRun, ActionListener listener) { + createConfig(indexingDryRun, ActionListener.wrap( + createConfigResponse -> { + if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { + IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; + String detectorId = response.getId(); + String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); + String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); + + timeSeriesIndices.initFlattenedResultIndex(indexName, ActionListener.wrap( + initResponse -> setupIngestPipeline(detectorId, ActionListener.wrap( + pipelineResponse -> { + updateResultIndexSetting(pipelineId, indexName, ActionListener.wrap( + updateResponse -> listener.onResponse(createConfigResponse), + listener::onFailure + )); + }, + listener::onFailure + )), + listener::onFailure + )); + } else { + listener.onResponse(createConfigResponse); + } + }, + listener::onFailure + )); + } + + private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConfigResponse) { + return !indexingDryRun + && config.getCustomResultIndexOrAlias() != null + && config.getFlattenResultIndexMapping() + && createConfigResponse instanceof IndexAnomalyDetectorResponse; + } + + protected void setupIngestPipeline(String detectorId, ActionListener listener) { + String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); + String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); + + try { + BytesReference pipelineSource = createPipelineDefinition(indexName); + + PutPipelineRequest putPipelineRequest = new PutPipelineRequest(pipelineId, pipelineSource, XContentType.JSON); + + client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap( + response -> { + if (response.isAcknowledged()) { + logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); + listener.onResponse(null); + } else { + String errorMessage = "Ingest pipeline creation was not acknowledged for pipelineId: " + pipelineId; + logger.error(errorMessage); + listener.onFailure(new OpenSearchStatusException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, + exception -> { + logger.error("Error while creating ingest pipeline for pipelineId: {}", pipelineId, exception); + listener.onFailure(exception); + } + )); + + } catch (IOException e) { + logger.error("Exception while building ingest pipeline definition for pipeline ID: {}", pipelineId, e); + listener.onFailure(e); + } + } + + private BytesReference createPipelineDefinition(String indexName) throws IOException { + XContentBuilder pipelineBuilder = XContentFactory.jsonBuilder(); + pipelineBuilder.startObject(); + { + pipelineBuilder.field("description", "Ingest pipeline for anomaly detector with result index: " + indexName); + pipelineBuilder.startArray("processors"); + { + pipelineBuilder.startObject(); + { + pipelineBuilder.startObject("script"); + { + pipelineBuilder.field("lang", "painless"); + String flattenScript = getScripts(TimeSeriesSettings.FLATTEN_CUSTOM_RESULT_INDEX_PAINLESS); + pipelineBuilder.field("source", flattenScript); + } + pipelineBuilder.endObject(); + } + pipelineBuilder.endObject(); + } + pipelineBuilder.endArray(); + } + pipelineBuilder.endObject(); + return BytesReference.bytes(pipelineBuilder); + } + + protected void updateResultIndexSetting(String pipelineId, String flattenedResultIndex, ActionListener listener) { + UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); + updateSettingsRequest.indices(flattenedResultIndex); + + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put("index.default_pipeline", pipelineId); + + updateSettingsRequest.settings(settingsBuilder); + + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( + response -> { + if (response.isAcknowledged()) { + logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId); + listener.onResponse(null); + } else { + String errorMsg = "Settings update not acknowledged for index: " + flattenedResultIndex; + logger.error(errorMsg); + listener.onFailure(new OpenSearchStatusException(errorMsg, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, + exception -> { + logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId, exception); + listener.onFailure(exception); + } + )); + } + + private void handleFlattenResultIndexMappingUpdate(ActionListener listener) { + if (config.getCustomResultIndexOrAlias() == null) { + return; + } + if (config.getFlattenResultIndexMapping() != null && config.getFlattenResultIndexMapping()) { + // if field value is true, create the pipeline. No need to get and compare with previous value + setupIngestPipeline(id, listener); } else { - createConfig(indexingDryRun, listener); + String pipelineId = "anomaly_detection_ingest_pipeline_" + config.getId(); + client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { + + @Override + public void onResponse(AcknowledgedResponse acknowledgedResponse) { + if (acknowledgedResponse.isAcknowledged()) { + logger.info("Ingest pipeline deleted successfully for pipelineId: {}", pipelineId); + } else { + logger.error("Failed to delete ingest pipeline for pipelineId: {}", pipelineId); + listener + .onFailure( + new OpenSearchStatusException( + "Ingest pipeline deletion was not acknowledged for pipelineId: " + pipelineId, + RestStatus.INTERNAL_SERVER_ERROR + ) + ); + } + } + + @Override + public void onFailure(Exception e) { + if (e instanceof OpenSearchStatusException && ((OpenSearchStatusException) e).status() == RestStatus.NOT_FOUND) { + logger.info("Ingest pipeline [{}] not found, skipping deletion.", pipelineId); + listener.onResponse(null); + } else { + logger.error("Error while deleting ingest pipeline for pipelineId: {}", pipelineId, e); + listener.onFailure(e); + } + } + }); +>>>>>>> 2a322387 (add a feature that flattens custom result index when enabled) } } @@ -820,7 +1018,7 @@ public void onFailure(Exception e) { }); } - protected void onCreateMappingsResponse(CreateIndexResponse response, boolean indexingDryRun, ActionListener listener) { + protected void onCreateMappingsResponse(CreateIndexResponse response, boolean indexingDryRun, ActionListener listener) throws IOException { if (response.isAcknowledged()) { logger.info("Created {} with mappings.", CommonName.CONFIG_INDEX); prepareConfigIndexing(indexingDryRun, listener); diff --git a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java index 61efd6104..5b765f0a3 100644 --- a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java @@ -26,6 +26,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.Writeable; diff --git a/src/main/resources/mappings/anomaly-results-flattened.json b/src/main/resources/mappings/anomaly-results-flattened.json new file mode 100644 index 000000000..38c087648 --- /dev/null +++ b/src/main/resources/mappings/anomaly-results-flattened.json @@ -0,0 +1,173 @@ +{ + "dynamic": true, + "_meta": { + "schema_version": 7 + }, + "properties": { + "detector_id": { + "type": "keyword" + }, + "is_anomaly": { + "type": "boolean" + }, + "anomaly_score": { + "type": "double" + }, + "anomaly_grade": { + "type": "double" + }, + "confidence": { + "type": "double" + }, + "feature_data": { + "type": "nested", + "properties": { + "feature_id": { + "type": "keyword" + }, + "feature_name": { + "type": "keyword" + }, + "data": { + "type": "double" + } + } + }, + "data_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "data_end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "execution_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "execution_end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "error": { + "type": "text" + }, + "user": { + "type": "nested", + "properties": { + "name": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword", + "ignore_above": 256 + } + } + }, + "backend_roles": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword" + } + } + }, + "roles": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword" + } + } + }, + "custom_attribute_names": { + "type": "text", + "fields": { + "keyword": { + "type": "keyword" + } + } + } + } + }, + "entity": { + "type": "nested", + "properties": { + "name": { + "type": "keyword" + }, + "value": { + "type": "keyword" + } + } + }, + "schema_version": { + "type": "integer" + }, + "task_id": { + "type": "keyword" + }, + "model_id": { + "type": "keyword" + }, + "approx_anomaly_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "relevant_attribution": { + "type": "nested", + "properties": { + "feature_id": { + "type": "keyword" + }, + "data": { + "type": "double" + } + } + }, + "past_values": { + "type": "nested", + "properties": { + "feature_id": { + "type": "keyword" + }, + "data": { + "type": "double" + } + } + }, + "expected_values": { + "type": "nested", + "properties": { + "likelihood": { + "type": "double" + }, + "value_list": { + "type": "nested", + "properties": { + "feature_id": { + "type": "keyword" + }, + "data": { + "type": "double" + } + } + } + } + }, + "threshold": { + "type": "double" + }, + "feature_imputed": { + "type": "nested", + "properties": { + "feature_id": { + "type": "keyword" + }, + "imputed": { + "type": "boolean" + } + } + } + } +} From 352075b7ecf6cca02539bbe96ff658d645b8f250 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Wed, 15 Jan 2025 13:18:57 -0800 Subject: [PATCH 02/21] clean up Signed-off-by: Jackie Han --- gradle.properties | 2 +- gradle/wrapper/gradle-wrapper.properties | 2 +- .../ad/indices/ADIndexManagement.java | 1 - .../timeseries/indices/IndexManagement.java | 38 +++--- .../AbstractTimeSeriesActionHandler.java | 124 +++++++++--------- .../settings/TimeSeriesSettings.java | 2 + .../transport/ResultBulkTransportAction.java | 1 - .../flatten-custom-result-index-painless.txt | 67 ++++++++++ 8 files changed, 152 insertions(+), 85 deletions(-) create mode 100644 src/main/resources/scripts/flatten-custom-result-index-painless.txt diff --git a/gradle.properties b/gradle.properties index afebc3db1..d2eba77fc 100644 --- a/gradle.properties +++ b/gradle.properties @@ -27,4 +27,4 @@ systemProp.org.gradle.warning.mode=fail systemProp.jdk.tls.client.protocols=TLSv1.2 # jvm args for faster test execution by default -systemProp.tests.jvm.argline=-XX:TieredStopAtLevel=1 -XX:ReservedCodeCacheSize=64m \ No newline at end of file +systemProp.tests.jvm.argline=-XX:TieredStopAtLevel=1 -XX:ReservedCodeCacheSize=64m diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 7cf08140d..1af9e0930 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -4,4 +4,4 @@ distributionUrl=https\://services.gradle.org/distributions/gradle-8.5-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME -zipStorePath=wrapper/dists \ No newline at end of file +zipStorePath=wrapper/dists diff --git a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java index 2ee771e14..161044d29 100644 --- a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java +++ b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java @@ -19,7 +19,6 @@ import static org.opensearch.ad.settings.AnomalyDetectorSettings.ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE; import static org.opensearch.ad.settings.AnomalyDetectorSettings.ANOMALY_RESULTS_INDEX_MAPPING_FILE; import static org.opensearch.ad.settings.AnomalyDetectorSettings.CHECKPOINT_INDEX_MAPPING_FILE; -import static org.opensearch.ad.settings.AnomalyDetectorSettings.FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE; import java.io.IOException; import java.util.EnumMap; diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 1ac646b4e..d06c17ea1 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -273,6 +273,11 @@ protected static String getMappings(String mappingFileRelativePath) throws IOExc return Resources.toString(url, Charsets.UTF_8); } + public static String getScripts(String scriptFileRelativePath) throws IOException { + URL url = IndexManagement.class.getClassLoader().getResource(scriptFileRelativePath); + return Resources.toString(url, Charsets.UTF_8); + } + protected void choosePrimaryShards(CreateIndexRequest request, boolean hiddenIndex) { request .settings( @@ -1019,26 +1024,23 @@ public void initFlattenedResultIndex(String indexName, ActionListener { - if (response.isAcknowledged()) { - logger.info("Successfully created flattened result index: {}", indexName); - actionListener.onResponse(response); - } else { - String errorMsg = "Index creation not acknowledged for index: " + indexName; - logger.error(errorMsg); - actionListener.onFailure(new IllegalStateException(errorMsg)); - } - }, - exception -> { - logger.error("Failed to create flattened result index: {}", indexName, exception); - actionListener.onFailure(exception); - } - )); + adminClient.indices().create(request, ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Successfully created flattened result index: {}", indexName); + actionListener.onResponse(response); + } else { + String errorMsg = "Index creation not acknowledged for index: " + indexName; + logger.error(errorMsg); + actionListener.onFailure(new IllegalStateException(errorMsg)); + } + }, exception -> { + logger.error("Failed to create flattened result index: {}", indexName, exception); + actionListener.onFailure(exception); + })); } /** diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 0c3532092..e16d5ad89 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -8,6 +8,7 @@ import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; import static org.opensearch.timeseries.constant.CommonMessages.CATEGORICAL_FIELD_TYPE_ERR_MSG; import static org.opensearch.timeseries.constant.CommonMessages.TIMESTAMP_VALIDATION_FAILED; +import static org.opensearch.timeseries.indices.IndexManagement.getScripts; import static org.opensearch.timeseries.util.ParseUtils.parseAggregators; import static org.opensearch.timeseries.util.RestHandlerUtils.XCONTENT_WITH_TYPE; import static org.opensearch.timeseries.util.RestHandlerUtils.isExceptionCausedByInvalidQuery; @@ -25,14 +26,18 @@ import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsAction; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsRequest; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsResponse; +import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.ingest.DeletePipelineRequest; +import org.opensearch.action.ingest.PutPipelineRequest; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.support.replication.ReplicationResponse; import org.opensearch.ad.transport.IndexAnomalyDetectorResponse; import org.opensearch.client.Client; @@ -41,11 +46,14 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; import org.opensearch.commons.authuser.User; import org.opensearch.core.action.ActionListener; import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.query.BoolQueryBuilder; import org.opensearch.index.query.QueryBuilder; @@ -68,6 +76,7 @@ import org.opensearch.timeseries.model.TimeSeriesTask; import org.opensearch.timeseries.model.ValidationAspect; import org.opensearch.timeseries.model.ValidationIssueType; +import org.opensearch.timeseries.settings.TimeSeriesSettings; import org.opensearch.timeseries.task.TaskCacheManager; import org.opensearch.timeseries.task.TaskManager; import org.opensearch.timeseries.util.*; @@ -454,39 +463,35 @@ private void handlePutRequest(boolean indexingDryRun, ActionListener listener } private void handlePostRequest(boolean indexingDryRun, ActionListener listener) { - createConfig(indexingDryRun, ActionListener.wrap( - createConfigResponse -> { - if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { - IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; - String detectorId = response.getId(); - String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); - String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); - - timeSeriesIndices.initFlattenedResultIndex(indexName, ActionListener.wrap( - initResponse -> setupIngestPipeline(detectorId, ActionListener.wrap( - pipelineResponse -> { - updateResultIndexSetting(pipelineId, indexName, ActionListener.wrap( - updateResponse -> listener.onResponse(createConfigResponse), - listener::onFailure - )); - }, - listener::onFailure - )), - listener::onFailure - )); - } else { - listener.onResponse(createConfigResponse); - } - }, - listener::onFailure - )); + createConfig(indexingDryRun, ActionListener.wrap(createConfigResponse -> { + if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { + IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; + String detectorId = response.getId(); + String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); + String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); + + timeSeriesIndices + .initFlattenedResultIndex( + indexName, + ActionListener.wrap(initResponse -> setupIngestPipeline(detectorId, ActionListener.wrap(pipelineResponse -> { + updateResultIndexSetting( + pipelineId, + indexName, + ActionListener.wrap(updateResponse -> listener.onResponse(createConfigResponse), listener::onFailure) + ); + }, listener::onFailure)), listener::onFailure) + ); + } else { + listener.onResponse(createConfigResponse); + } + }, listener::onFailure)); } private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConfigResponse) { return !indexingDryRun - && config.getCustomResultIndexOrAlias() != null - && config.getFlattenResultIndexMapping() - && createConfigResponse instanceof IndexAnomalyDetectorResponse; + && config.getCustomResultIndexOrAlias() != null + && config.getFlattenResultIndexMapping() + && createConfigResponse instanceof IndexAnomalyDetectorResponse; } protected void setupIngestPipeline(String detectorId, ActionListener listener) { @@ -498,22 +503,19 @@ protected void setupIngestPipeline(String detectorId, ActionListener listener PutPipelineRequest putPipelineRequest = new PutPipelineRequest(pipelineId, pipelineSource, XContentType.JSON); - client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap( - response -> { - if (response.isAcknowledged()) { - logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); - listener.onResponse(null); - } else { - String errorMessage = "Ingest pipeline creation was not acknowledged for pipelineId: " + pipelineId; - logger.error(errorMessage); - listener.onFailure(new OpenSearchStatusException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR)); - } - }, - exception -> { - logger.error("Error while creating ingest pipeline for pipelineId: {}", pipelineId, exception); - listener.onFailure(exception); - } - )); + client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); + listener.onResponse(null); + } else { + String errorMessage = "Ingest pipeline creation was not acknowledged for pipelineId: " + pipelineId; + logger.error(errorMessage); + listener.onFailure(new OpenSearchStatusException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, exception -> { + logger.error("Error while creating ingest pipeline for pipelineId: {}", pipelineId, exception); + listener.onFailure(exception); + })); } catch (IOException e) { logger.error("Exception while building ingest pipeline definition for pipeline ID: {}", pipelineId, e); @@ -555,22 +557,19 @@ protected void updateResultIndexSetting(String pipelineId, String flattenedResul updateSettingsRequest.settings(settingsBuilder); - client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( - response -> { - if (response.isAcknowledged()) { - logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId); - listener.onResponse(null); - } else { - String errorMsg = "Settings update not acknowledged for index: " + flattenedResultIndex; - logger.error(errorMsg); - listener.onFailure(new OpenSearchStatusException(errorMsg, RestStatus.INTERNAL_SERVER_ERROR)); - } - }, - exception -> { - logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId, exception); - listener.onFailure(exception); - } - )); + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId); + listener.onResponse(null); + } else { + String errorMsg = "Settings update not acknowledged for index: " + flattenedResultIndex; + logger.error(errorMsg); + listener.onFailure(new OpenSearchStatusException(errorMsg, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, exception -> { + logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId, exception); + listener.onFailure(exception); + })); } private void handleFlattenResultIndexMappingUpdate(ActionListener listener) { @@ -611,7 +610,6 @@ public void onFailure(Exception e) { } } }); ->>>>>>> 2a322387 (add a feature that flattens custom result index when enabled) } } @@ -1018,7 +1016,7 @@ public void onFailure(Exception e) { }); } - protected void onCreateMappingsResponse(CreateIndexResponse response, boolean indexingDryRun, ActionListener listener) throws IOException { + protected void onCreateMappingsResponse(CreateIndexResponse response, boolean indexingDryRun, ActionListener listener) { if (response.isAcknowledged()) { logger.info("Created {} with mappings.", CommonName.CONFIG_INDEX); prepareConfigIndexing(indexingDryRun, listener); diff --git a/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java b/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java index 34bf7835f..d44d52f16 100644 --- a/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java +++ b/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java @@ -289,4 +289,6 @@ public class TimeSeriesSettings { // max entities to track per detector public static final int MAX_TRACKING_ENTITIES = 1000000; + + public static final String FLATTEN_CUSTOM_RESULT_INDEX_PAINLESS = "scripts/flatten-custom-result-index-painless.txt"; } diff --git a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java index 5b765f0a3..61efd6104 100644 --- a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java @@ -26,7 +26,6 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; import org.opensearch.client.Client; -import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.Writeable; diff --git a/src/main/resources/scripts/flatten-custom-result-index-painless.txt b/src/main/resources/scripts/flatten-custom-result-index-painless.txt new file mode 100644 index 000000000..22516646f --- /dev/null +++ b/src/main/resources/scripts/flatten-custom-result-index-painless.txt @@ -0,0 +1,67 @@ +// Create a map to store the relationship between feature_id and feature_name from feature_data +def featureNameMap = [:]; + +// Populate the map from feature_data +if (ctx.containsKey('feature_data') && ctx.feature_data != null) { + for (int i = 0; i < ctx.feature_data.length; i++) { + def feature = ctx.feature_data[i]; + if (feature != null && feature.containsKey('feature_id') && feature.containsKey('feature_name')) { + featureNameMap[feature.feature_id] = feature.feature_name; + ctx['feature_data_' + feature.feature_name] = feature.data; // Flatten feature_data as before + } + } +} + +// Flatten nested entity field +if (ctx.containsKey('entity') && ctx.entity != null) { + for (int i = 0; i < ctx.entity.length; i++) { + def entity = ctx.entity[i]; + if (entity != null && entity.containsKey('name') && entity.containsKey('value')) { + ctx['entity_' + entity.name] = entity.value; + } + } +} + +// Flatten nested relevant_attribution field +if (ctx.containsKey('relevant_attribution') && ctx.relevant_attribution != null) { + for (int i = 0; i < ctx.relevant_attribution.length; i++) { + def attribution = ctx.relevant_attribution[i]; + if (attribution != null && attribution.containsKey('feature_id') && attribution.containsKey('data')) { + def featureName = featureNameMap[attribution.feature_id]; + if (featureName != null) { + ctx['relevant_attribution_' + featureName] = attribution.data; + } + } + } +} + +// Flatten nested expected_values field +if (ctx.containsKey('expected_values') && ctx.expected_values != null) { + for (int i = 0; i < ctx.expected_values.length; i++) { + def expected = ctx.expected_values[i]; + if (expected != null && expected.containsKey('value_list') && expected.value_list != null) { + for (int j = 0; j < expected.value_list.length; j++) { + def value = expected.value_list[j]; + if (value != null && value.containsKey('feature_id') && value.containsKey('data')) { + def featureName = featureNameMap[value.feature_id]; + if (featureName != null) { + ctx['expected_values_' + featureName] = value.data; + } + } + } + } + } +} + +// Flatten nested past_values field +if (ctx.containsKey('past_values') && ctx.past_values != null) { + for (int i = 0; i < ctx.past_values.length; i++) { + def pastValue = ctx.past_values[i]; + if (pastValue != null && pastValue.containsKey('feature_id') && pastValue.containsKey('data')) { + def featureName = featureNameMap[pastValue.feature_id]; + if (featureName != null) { + ctx['past_value_' + featureName] = pastValue.data; + } + } + } +} \ No newline at end of file From b131f9aa3ac75a7e97fe61732d43a63bbb1bf7cb Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Thu, 23 Jan 2025 15:39:53 -0800 Subject: [PATCH 03/21] add IT Signed-off-by: Jackie Han --- .../ADResultBulkTransportAction.java | 3 +- .../AbstractTimeSeriesActionHandler.java | 9 ++-- .../ad/rest/AnomalyDetectorRestApiIT.java | 52 ++++++++++++++++++- .../opensearch/timeseries/TestHelpers.java | 34 ++++++++++++ 4 files changed, 90 insertions(+), 8 deletions(-) diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index 205b63a37..7340adc3f 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -17,6 +17,7 @@ import java.io.IOException; import java.util.List; +import java.util.Locale; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -135,7 +136,7 @@ private boolean shouldAddResult(float indexingPressurePercent, AnomalyResult res } private void addToFlattenedIndexIfExists(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { - String flattenedResultIndexName = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(); + String flattenedResultIndexName = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); if (clusterService.state().metadata().hasIndex(flattenedResultIndexName)) { addResult(bulkRequest, result, flattenedResultIndexName); } diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index e16d5ad89..1a6e9199b 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -417,7 +417,6 @@ protected void validateTimeField(boolean indexingDryRun, ActionListener liste * If {@code true}, the operation performs validation without creating/updating the configuration. * If {@code false}, the configuration is created or updated. * @param listener the {@link ActionListener} to handle the response or failure of the operation. - * @throws IOException if an I/O error occurs during the operation. * *

Behavior:

*
    @@ -467,8 +466,8 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; String detectorId = response.getId(); - String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); - String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); + String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(Locale.ROOT); + String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(Locale.ROOT); timeSeriesIndices .initFlattenedResultIndex( @@ -495,8 +494,8 @@ private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConf } protected void setupIngestPipeline(String detectorId, ActionListener listener) { - String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(); - String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(); + String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(Locale.ROOT); + String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(Locale.ROOT); try { BytesReference pipelineSource = createPipelineDefinition(indexName); diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index d4e2ca4ec..220778e18 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -122,11 +122,20 @@ private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List< } private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List features, boolean useDateNanos) - throws IOException { + throws IOException { + return createIndexAndGetAnomalyDetector(indexName, features, useDateNanos, false); + } + + private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List features, boolean useDateNanos, + boolean useFlattenResultIndex) throws IOException { TestHelpers.createIndexWithTimeField(client(), indexName, TIME_FIELD, useDateNanos); String testIndexData = "{\"keyword-field\": \"field-1\", \"ip-field\": \"1.2.3.4\", \"timestamp\": 1}"; TestHelpers.ingestDataToIndex(client(), indexName, TestHelpers.toHttpEntity(testIndexData)); - AnomalyDetector detector = TestHelpers.randomAnomalyDetector(TIME_FIELD, indexName, features); + + AnomalyDetector detector = useFlattenResultIndex + ? TestHelpers.randomAnomalyDetectorWithFlattenResultIndex(TIME_FIELD, indexName, features) + : TestHelpers.randomAnomalyDetector(TIME_FIELD, indexName, features); + return detector; } @@ -180,6 +189,45 @@ public void testCreateAnomalyDetectorWithDuplicateName() throws Exception { ); } + public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception { + AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME, + ImmutableList.of(TestHelpers.randomFeature(true)), false, true); + + // test behavior when AD is disabled + updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); + Exception ex = expectThrows( + ResponseException.class, + () -> TestHelpers + .makeRequest( + client(), + "POST", + TestHelpers.AD_BASE_DETECTORS_URI, + ImmutableMap.of(), + TestHelpers.toHttpEntity(detector), + null + ) + ); + assertThat(ex.getMessage(), containsString(ADCommonMessages.DISABLED_ERR_MSG)); + + // test behavior when AD is enabled + updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); + Response response = TestHelpers + .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + assertEquals("Create anomaly detector with flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); + Map responseMap = entityAsMap(response); + String id = (String) responseMap.get("_id"); + int version = (int) responseMap.get("_version"); + assertNotEquals("response is missing Id", AnomalyDetector.NO_ID, id); + assertTrue("incorrect version", version > 0); + // ensure the flattened result index was created + String expectedFlattenedIndex = String.format( + "opensearch-ad-plugin-result-test_flattened_%s", + id.toLowerCase(Locale.ROOT) + ); + boolean indexExists = indexExists(expectedFlattenedIndex); + assertTrue(indexExists); + } + public void testCreateAnomalyDetector() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME); updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index 0c8f45a63..4168cfed2 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -511,6 +511,40 @@ public static AnomalyDetector randomAnomalyDetector(String timefield, String ind ); } + public static AnomalyDetector randomAnomalyDetectorWithFlattenResultIndex(String timefield, String indexName, List features) throws IOException { + return new AnomalyDetector( + randomAlphaOfLength(10), + randomLong(), + randomAlphaOfLength(20), + randomAlphaOfLength(30), + timefield, + ImmutableList.of(indexName.toLowerCase(Locale.ROOT)), + features, + randomQuery(), + randomIntervalTimeConfiguration(), + randomIntervalTimeConfiguration(), + randomIntBetween(1, TimeSeriesSettings.MAX_SHINGLE_SIZE), + null, + randomInt(), + Instant.now(), + null, + randomUser(), + ADCommonName.CUSTOM_RESULT_INDEX_PREFIX + "test", + TestHelpers.randomImputationOption(features), + // timeDecay (reverse of recencyEmphasis) should be less than 1. + // so we start with 2. + randomIntBetween(2, 10000), + randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), + randomIntBetween(1, 1000), + null, + null, + null, + null, + true, + Instant.now() + ); + } + public static AnomalyDetector randomAnomalyDetectorWithEmptyFeature() throws IOException { return new AnomalyDetector( randomAlphaOfLength(10), From 9fced72f9d21576c7559784bdcd2578489641a21 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Thu, 23 Jan 2025 15:52:10 -0800 Subject: [PATCH 04/21] cleanup Signed-off-by: Jackie Han --- .../ad/rest/AnomalyDetectorRestApiIT.java | 50 ++++++++------- .../opensearch/timeseries/TestHelpers.java | 61 ++++++++++--------- 2 files changed, 59 insertions(+), 52 deletions(-) diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 220778e18..8edde712b 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -122,19 +122,23 @@ private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List< } private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List features, boolean useDateNanos) - throws IOException { + throws IOException { return createIndexAndGetAnomalyDetector(indexName, features, useDateNanos, false); } - private AnomalyDetector createIndexAndGetAnomalyDetector(String indexName, List features, boolean useDateNanos, - boolean useFlattenResultIndex) throws IOException { + private AnomalyDetector createIndexAndGetAnomalyDetector( + String indexName, + List features, + boolean useDateNanos, + boolean useFlattenResultIndex + ) throws IOException { TestHelpers.createIndexWithTimeField(client(), indexName, TIME_FIELD, useDateNanos); String testIndexData = "{\"keyword-field\": \"field-1\", \"ip-field\": \"1.2.3.4\", \"timestamp\": 1}"; TestHelpers.ingestDataToIndex(client(), indexName, TestHelpers.toHttpEntity(testIndexData)); AnomalyDetector detector = useFlattenResultIndex - ? TestHelpers.randomAnomalyDetectorWithFlattenResultIndex(TIME_FIELD, indexName, features) - : TestHelpers.randomAnomalyDetector(TIME_FIELD, indexName, features); + ? TestHelpers.randomAnomalyDetectorWithFlattenResultIndex(TIME_FIELD, indexName, features) + : TestHelpers.randomAnomalyDetector(TIME_FIELD, indexName, features); return detector; } @@ -190,29 +194,33 @@ public void testCreateAnomalyDetectorWithDuplicateName() throws Exception { } public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception { - AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME, - ImmutableList.of(TestHelpers.randomFeature(true)), false, true); + AnomalyDetector detector = createIndexAndGetAnomalyDetector( + INDEX_NAME, + ImmutableList.of(TestHelpers.randomFeature(true)), + false, + true + ); // test behavior when AD is disabled updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); Exception ex = expectThrows( - ResponseException.class, - () -> TestHelpers - .makeRequest( - client(), - "POST", - TestHelpers.AD_BASE_DETECTORS_URI, - ImmutableMap.of(), - TestHelpers.toHttpEntity(detector), - null - ) + ResponseException.class, + () -> TestHelpers + .makeRequest( + client(), + "POST", + TestHelpers.AD_BASE_DETECTORS_URI, + ImmutableMap.of(), + TestHelpers.toHttpEntity(detector), + null + ) ); assertThat(ex.getMessage(), containsString(ADCommonMessages.DISABLED_ERR_MSG)); // test behavior when AD is enabled updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); Response response = TestHelpers - .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); assertEquals("Create anomaly detector with flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); Map responseMap = entityAsMap(response); String id = (String) responseMap.get("_id"); @@ -220,10 +228,8 @@ public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception assertNotEquals("response is missing Id", AnomalyDetector.NO_ID, id); assertTrue("incorrect version", version > 0); // ensure the flattened result index was created - String expectedFlattenedIndex = String.format( - "opensearch-ad-plugin-result-test_flattened_%s", - id.toLowerCase(Locale.ROOT) - ); + String expectedFlattenedIndex = String + .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); boolean indexExists = indexExists(expectedFlattenedIndex); assertTrue(indexExists); } diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index 4168cfed2..e53f0bfe4 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -511,37 +511,38 @@ public static AnomalyDetector randomAnomalyDetector(String timefield, String ind ); } - public static AnomalyDetector randomAnomalyDetectorWithFlattenResultIndex(String timefield, String indexName, List features) throws IOException { + public static AnomalyDetector randomAnomalyDetectorWithFlattenResultIndex(String timefield, String indexName, List features) + throws IOException { return new AnomalyDetector( - randomAlphaOfLength(10), - randomLong(), - randomAlphaOfLength(20), - randomAlphaOfLength(30), - timefield, - ImmutableList.of(indexName.toLowerCase(Locale.ROOT)), - features, - randomQuery(), - randomIntervalTimeConfiguration(), - randomIntervalTimeConfiguration(), - randomIntBetween(1, TimeSeriesSettings.MAX_SHINGLE_SIZE), - null, - randomInt(), - Instant.now(), - null, - randomUser(), - ADCommonName.CUSTOM_RESULT_INDEX_PREFIX + "test", - TestHelpers.randomImputationOption(features), - // timeDecay (reverse of recencyEmphasis) should be less than 1. - // so we start with 2. - randomIntBetween(2, 10000), - randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), - randomIntBetween(1, 1000), - null, - null, - null, - null, - true, - Instant.now() + randomAlphaOfLength(10), + randomLong(), + randomAlphaOfLength(20), + randomAlphaOfLength(30), + timefield, + ImmutableList.of(indexName.toLowerCase(Locale.ROOT)), + features, + randomQuery(), + randomIntervalTimeConfiguration(), + randomIntervalTimeConfiguration(), + randomIntBetween(1, TimeSeriesSettings.MAX_SHINGLE_SIZE), + null, + randomInt(), + Instant.now(), + null, + randomUser(), + ADCommonName.CUSTOM_RESULT_INDEX_PREFIX + "test", + TestHelpers.randomImputationOption(features), + // timeDecay (reverse of recencyEmphasis) should be less than 1. + // so we start with 2. + randomIntBetween(2, 10000), + randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), + randomIntBetween(1, 1000), + null, + null, + null, + null, + true, + Instant.now() ); } From 3ce0734d1429c2240543cc3d897db32761a16c3c Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 00:07:04 -0800 Subject: [PATCH 05/21] address comments Signed-off-by: Jackie Han --- .../ad/indices/ADIndexManagement.java | 21 +++++++++++++ .../ADResultBulkTransportAction.java | 6 ++-- .../timeseries/indices/IndexManagement.java | 28 ++++++++--------- .../opensearch/timeseries/model/Config.java | 2 +- .../AbstractTimeSeriesActionHandler.java | 31 ++++++++++++------- 5 files changed, 58 insertions(+), 30 deletions(-) diff --git a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java index 161044d29..8248d4520 100644 --- a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java +++ b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.EnumMap; +import java.util.Map; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -45,6 +46,8 @@ import org.opensearch.timeseries.indices.IndexManagement; import org.opensearch.timeseries.util.DiscoveryNodeFilterer; +import com.fasterxml.jackson.databind.ObjectMapper; + /** * This class provides utility methods for various anomaly detection indices. */ @@ -57,6 +60,8 @@ public class ADIndexManagement extends IndexManagement { // The index name pattern to query all AD result, history and current AD result public static final String ALL_AD_RESULTS_INDEX_PATTERN = ".opendistro-anomaly-results*"; + // private static final ObjectMapper objectMapper = new ObjectMapper(); + /** * Constructor function * @@ -122,6 +127,22 @@ public static String getResultMappings() throws IOException { return getMappings(ANOMALY_RESULTS_INDEX_MAPPING_FILE); } + /** + * Retrieves the JSON mapping for the flattened result index with the "dynamic" field set to true + * @return JSON mapping for the flattened result index. + * @throws IOException if the mapping file cannot be read. + */ + public static String getFlattenedResultMappings() throws IOException { + ObjectMapper objectMapper = new ObjectMapper(); + + Map mapping = objectMapper + .readValue(ADIndexManagement.class.getClassLoader().getResourceAsStream(ANOMALY_RESULTS_INDEX_MAPPING_FILE), Map.class); + + mapping.put("dynamic", true); + + return objectMapper.writeValueAsString(mapping); + } + /** * Get anomaly detector state index mapping json content. * diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index 7340adc3f..da90bf930 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -136,9 +136,9 @@ private boolean shouldAddResult(float indexingPressurePercent, AnomalyResult res } private void addToFlattenedIndexIfExists(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { - String flattenedResultIndexName = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); - if (clusterService.state().metadata().hasIndex(flattenedResultIndexName)) { - addResult(bulkRequest, result, flattenedResultIndexName); + String flattenedResultIndexAlias = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); + if (clusterService.state().metadata().hasAlias(flattenedResultIndexAlias)) { + addResult(bulkRequest, result, flattenedResultIndexAlias); } } diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index d06c17ea1..57e25fb79 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -11,7 +11,7 @@ package org.opensearch.timeseries.indices; -import static org.opensearch.ad.settings.AnomalyDetectorSettings.FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE; +import static org.opensearch.ad.indices.ADIndexManagement.getFlattenedResultMappings; import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; import static org.opensearch.timeseries.util.RestHandlerUtils.createXContentParserFromRegistry; @@ -90,6 +90,7 @@ import org.opensearch.timeseries.settings.TimeSeriesSettings; import org.opensearch.timeseries.util.DiscoveryNodeFilterer; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; import com.google.common.io.Resources; @@ -137,6 +138,7 @@ public abstract class IndexManagement & TimeSe private NamedXContentRegistry xContentRegistry; protected BiCheckedFunction configParser; protected String customResultIndexPrefix; + private final ObjectMapper objectMapper = new ObjectMapper(); protected class IndexState { // keep track of whether the mapping version is up-to-date @@ -1016,21 +1018,28 @@ public void initCustomResultIndexAndExecute(String resultIndexOrAlias, Execu /** * creates flattened result index - * @param indexName the index name + * @param flattenedResultIndexAlias the flattened result index alias * @param actionListener the action listener * @throws IOException */ - public void initFlattenedResultIndex(String indexName, ActionListener actionListener) throws IOException { + public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionListener actionListener) + throws IOException { + String indexName = getCustomResultIndexPattern(flattenedResultIndexAlias); logger.info("Initializing flattened result index: {}", indexName); CreateIndexRequest request = new CreateIndexRequest(indexName) - .mapping(getFlattenedResultIndexMappings(), XContentType.JSON) + .mapping(getFlattenedResultMappings(), XContentType.JSON) .settings(settings); + + if (flattenedResultIndexAlias != null) { + request.alias(new Alias(flattenedResultIndexAlias)); + } + choosePrimaryShards(request, false); adminClient.indices().create(request, ActionListener.wrap(response -> { if (response.isAcknowledged()) { - logger.info("Successfully created flattened result index: {}", indexName); + logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); actionListener.onResponse(response); } else { String errorMsg = "Index creation not acknowledged for index: " + indexName; @@ -1043,15 +1052,6 @@ public void initFlattenedResultIndex(String indexName, ActionListener void validateCustomIndexForBackendJob( String resultIndexOrAlias, String securityLogId, diff --git a/src/main/java/org/opensearch/timeseries/model/Config.java b/src/main/java/org/opensearch/timeseries/model/Config.java index d61807528..81f88c1ba 100644 --- a/src/main/java/org/opensearch/timeseries/model/Config.java +++ b/src/main/java/org/opensearch/timeseries/model/Config.java @@ -79,7 +79,7 @@ public abstract class Config implements Writeable, ToXContentObject { public static final String RESULT_INDEX_FIELD_MIN_SIZE = "result_index_min_size"; public static final String RESULT_INDEX_FIELD_MIN_AGE = "result_index_min_age"; public static final String RESULT_INDEX_FIELD_TTL = "result_index_ttl"; - public static final String FLATTEN_RESULT_INDEX_MAPPING = "flatten_result_index_mapping"; + public static final String FLATTEN_RESULT_INDEX_MAPPING = "flatten_custom_result_index"; // Changing categorical field, feature attributes, interval, windowDelay, time field, horizon, indices, // result index would force us to display results only from the most recent update. Otherwise, // the UI appear cluttered and unclear. diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 1a6e9199b..c7ba96283 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -454,6 +454,14 @@ protected void prepareConfigIndexing(boolean indexingDryRun, ActionListener l } } + private String getFlattenedResultIndexAlias(String configId) { + return config.getCustomResultIndexOrAlias() + "_flattened_" + configId.toLowerCase(Locale.ROOT); + } + + private String getFlattenResultIndexIngestPipelineId(String configId) { + return "flatten_result_index_ingest_pipeline" + configId.toLowerCase(Locale.ROOT); + } + private void handlePutRequest(boolean indexingDryRun, ActionListener listener) { handler.confirmJobRunning(clusterService, client, id, listener, () -> { handleFlattenResultIndexMappingUpdate(listener); @@ -465,17 +473,17 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene createConfig(indexingDryRun, ActionListener.wrap(createConfigResponse -> { if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; - String detectorId = response.getId(); - String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(Locale.ROOT); - String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(Locale.ROOT); + String configId = response.getId(); + String flattenedResultIndexAlias = getFlattenedResultIndexAlias(configId); + String pipelineId = getFlattenResultIndexIngestPipelineId(configId); timeSeriesIndices .initFlattenedResultIndex( - indexName, - ActionListener.wrap(initResponse -> setupIngestPipeline(detectorId, ActionListener.wrap(pipelineResponse -> { + flattenedResultIndexAlias, + ActionListener.wrap(initResponse -> setupIngestPipeline(configId, ActionListener.wrap(pipelineResponse -> { updateResultIndexSetting( pipelineId, - indexName, + flattenedResultIndexAlias, ActionListener.wrap(updateResponse -> listener.onResponse(createConfigResponse), listener::onFailure) ); }, listener::onFailure)), listener::onFailure) @@ -493,12 +501,12 @@ private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConf && createConfigResponse instanceof IndexAnomalyDetectorResponse; } - protected void setupIngestPipeline(String detectorId, ActionListener listener) { - String indexName = config.getCustomResultIndexOrAlias() + "_flattened_" + detectorId.toLowerCase(Locale.ROOT); - String pipelineId = "anomaly_detection_ingest_pipeline_" + detectorId.toLowerCase(Locale.ROOT); + protected void setupIngestPipeline(String configId, ActionListener listener) { + String flattenedResultIndexAlias = getFlattenedResultIndexAlias(configId); + String pipelineId = getFlattenResultIndexIngestPipelineId(configId); try { - BytesReference pipelineSource = createPipelineDefinition(indexName); + BytesReference pipelineSource = createPipelineDefinition(flattenedResultIndexAlias); PutPipelineRequest putPipelineRequest = new PutPipelineRequest(pipelineId, pipelineSource, XContentType.JSON); @@ -576,10 +584,9 @@ private void handleFlattenResultIndexMappingUpdate(ActionListener listener) { return; } if (config.getFlattenResultIndexMapping() != null && config.getFlattenResultIndexMapping()) { - // if field value is true, create the pipeline. No need to get and compare with previous value setupIngestPipeline(id, listener); } else { - String pipelineId = "anomaly_detection_ingest_pipeline_" + config.getId(); + String pipelineId = getFlattenResultIndexIngestPipelineId(config.getId()); client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { @Override From fb715c37ebd8b765e82993749c1ada3033b48c5a Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 00:53:46 -0800 Subject: [PATCH 06/21] update IT Signed-off-by: Jackie Han --- .../ad/rest/AnomalyDetectorRestApiIT.java | 19 ++++++++++++++++--- .../opensearch/timeseries/TestHelpers.java | 1 + 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 8edde712b..a660ec752 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -196,7 +196,7 @@ public void testCreateAnomalyDetectorWithDuplicateName() throws Exception { public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector( INDEX_NAME, - ImmutableList.of(TestHelpers.randomFeature(true)), + ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), false, true ); @@ -230,8 +230,21 @@ public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception // ensure the flattened result index was created String expectedFlattenedIndex = String .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); - boolean indexExists = indexExists(expectedFlattenedIndex); - assertTrue(indexExists); + boolean aliasExists = aliasExists(expectedFlattenedIndex); + assertTrue(aliasExists); + // ensure that the flattened field "feature_data_feature_bytes" exists in the mappings + String startDetectorEndpoint = String.format(Locale.ROOT, TestHelpers.AD_BASE_START_DETECTOR_URL, id); + Response startDetectorResponse = TestHelpers + .makeRequest(client(), "POST", startDetectorEndpoint, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + String getFlattenedResultIndexEndpoint = String + .format(Locale.ROOT, "/opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); + Response getIndexResponse = TestHelpers.makeRequest(client(), "GET", getFlattenedResultIndexEndpoint, ImmutableMap.of(), "", null); + Map flattenedResultIndex = entityAsMap(getIndexResponse); + Map mappings = (Map) flattenedResultIndex + .get("opensearch-ad-plugin-result-test_flattened_" + id.toLowerCase(Locale.ROOT)); + Map properties = (Map) ((Map) mappings.get("mappings")).get("properties"); + assertTrue("Flattened field 'feature_data_feature_bytes' does not exist", properties.containsKey("feature_data_feature_bytes")); + } public void testCreateAnomalyDetector() throws Exception { diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index e53f0bfe4..b0df971f2 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -171,6 +171,7 @@ public class TestHelpers { public static final String AD_BASE_RESULT_URI = AD_BASE_DETECTORS_URI + "/results"; public static final String AD_BASE_PREVIEW_URI = AD_BASE_DETECTORS_URI + "/%s/_preview"; public static final String AD_BASE_STATS_URI = "/_plugins/_anomaly_detection/stats"; + public static final String AD_BASE_START_DETECTOR_URL = AD_BASE_DETECTORS_URI + "/%s/_start"; public static ImmutableSet HISTORICAL_ANALYSIS_RUNNING_STATS = ImmutableSet .of(TaskState.CREATED.name(), TaskState.INIT.name(), TaskState.RUNNING.name()); // Task may fail if memory circuit breaker triggered. From 499e7eadd782b969e8df7275dd79557114b57ad1 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 01:55:22 -0800 Subject: [PATCH 07/21] update IT Signed-off-by: Jackie Han --- .../ad/settings/AnomalyDetectorSettings.java | 1 - .../AbstractTimeSeriesActionHandler.java | 4 +- .../mappings/anomaly-results-flattened.json | 173 ------------------ .../ad/rest/AnomalyDetectorRestApiIT.java | 21 ++- 4 files changed, 19 insertions(+), 180 deletions(-) delete mode 100644 src/main/resources/mappings/anomaly-results-flattened.json diff --git a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java index e1235707d..ffd88ae9d 100644 --- a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java +++ b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java @@ -190,7 +190,6 @@ private AnomalyDetectorSettings() {} ); public static final String ANOMALY_RESULTS_INDEX_MAPPING_FILE = "mappings/anomaly-results.json"; - public static final String FLATTENED_ANOMALY_RESULTS_INDEX_MAPPING_FILE = "mappings/anomaly-results-flattened.json"; public static final String ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE = "mappings/anomaly-detection-state.json"; public static final String CHECKPOINT_INDEX_MAPPING_FILE = "mappings/anomaly-checkpoint.json"; diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index c7ba96283..69cf0ddcc 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -495,9 +495,11 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene } private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConfigResponse) { + Boolean flattenResultIndexMapping = config.getFlattenResultIndexMapping(); + return !indexingDryRun && config.getCustomResultIndexOrAlias() != null - && config.getFlattenResultIndexMapping() + && Boolean.TRUE.equals(flattenResultIndexMapping) && createConfigResponse instanceof IndexAnomalyDetectorResponse; } diff --git a/src/main/resources/mappings/anomaly-results-flattened.json b/src/main/resources/mappings/anomaly-results-flattened.json deleted file mode 100644 index 38c087648..000000000 --- a/src/main/resources/mappings/anomaly-results-flattened.json +++ /dev/null @@ -1,173 +0,0 @@ -{ - "dynamic": true, - "_meta": { - "schema_version": 7 - }, - "properties": { - "detector_id": { - "type": "keyword" - }, - "is_anomaly": { - "type": "boolean" - }, - "anomaly_score": { - "type": "double" - }, - "anomaly_grade": { - "type": "double" - }, - "confidence": { - "type": "double" - }, - "feature_data": { - "type": "nested", - "properties": { - "feature_id": { - "type": "keyword" - }, - "feature_name": { - "type": "keyword" - }, - "data": { - "type": "double" - } - } - }, - "data_start_time": { - "type": "date", - "format": "strict_date_time||epoch_millis" - }, - "data_end_time": { - "type": "date", - "format": "strict_date_time||epoch_millis" - }, - "execution_start_time": { - "type": "date", - "format": "strict_date_time||epoch_millis" - }, - "execution_end_time": { - "type": "date", - "format": "strict_date_time||epoch_millis" - }, - "error": { - "type": "text" - }, - "user": { - "type": "nested", - "properties": { - "name": { - "type": "text", - "fields": { - "keyword": { - "type": "keyword", - "ignore_above": 256 - } - } - }, - "backend_roles": { - "type": "text", - "fields": { - "keyword": { - "type": "keyword" - } - } - }, - "roles": { - "type": "text", - "fields": { - "keyword": { - "type": "keyword" - } - } - }, - "custom_attribute_names": { - "type": "text", - "fields": { - "keyword": { - "type": "keyword" - } - } - } - } - }, - "entity": { - "type": "nested", - "properties": { - "name": { - "type": "keyword" - }, - "value": { - "type": "keyword" - } - } - }, - "schema_version": { - "type": "integer" - }, - "task_id": { - "type": "keyword" - }, - "model_id": { - "type": "keyword" - }, - "approx_anomaly_start_time": { - "type": "date", - "format": "strict_date_time||epoch_millis" - }, - "relevant_attribution": { - "type": "nested", - "properties": { - "feature_id": { - "type": "keyword" - }, - "data": { - "type": "double" - } - } - }, - "past_values": { - "type": "nested", - "properties": { - "feature_id": { - "type": "keyword" - }, - "data": { - "type": "double" - } - } - }, - "expected_values": { - "type": "nested", - "properties": { - "likelihood": { - "type": "double" - }, - "value_list": { - "type": "nested", - "properties": { - "feature_id": { - "type": "keyword" - }, - "data": { - "type": "double" - } - } - } - } - }, - "threshold": { - "type": "double" - }, - "feature_imputed": { - "type": "nested", - "properties": { - "feature_id": { - "type": "keyword" - }, - "imputed": { - "type": "boolean" - } - } - } - } -} diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index a660ec752..65516a241 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -27,6 +27,7 @@ import java.util.stream.Collectors; import org.apache.hc.core5.http.ContentType; +import org.apache.hc.core5.http.HttpEntity; import org.apache.hc.core5.http.io.entity.StringEntity; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -235,16 +236,26 @@ public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception // ensure that the flattened field "feature_data_feature_bytes" exists in the mappings String startDetectorEndpoint = String.format(Locale.ROOT, TestHelpers.AD_BASE_START_DETECTOR_URL, id); Response startDetectorResponse = TestHelpers - .makeRequest(client(), "POST", startDetectorEndpoint, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + .makeRequest(client(), "POST", startDetectorEndpoint, ImmutableMap.of(), (HttpEntity) null, null); String getFlattenedResultIndexEndpoint = String .format(Locale.ROOT, "/opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); + // wait for the detector starts writing result + try { + Thread.sleep(60 * 1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Thread was interrupted while waiting", e); + } Response getIndexResponse = TestHelpers.makeRequest(client(), "GET", getFlattenedResultIndexEndpoint, ImmutableMap.of(), "", null); Map flattenedResultIndex = entityAsMap(getIndexResponse); - Map mappings = (Map) flattenedResultIndex - .get("opensearch-ad-plugin-result-test_flattened_" + id.toLowerCase(Locale.ROOT)); - Map properties = (Map) ((Map) mappings.get("mappings")).get("properties"); - assertTrue("Flattened field 'feature_data_feature_bytes' does not exist", properties.containsKey("feature_data_feature_bytes")); + String indexKey = flattenedResultIndex.keySet().stream().findFirst().orElse(null); + Map indexDetails = (Map) flattenedResultIndex.get(indexKey); + Map mappings = (Map) indexDetails.get("mappings"); + Object dynamicValue = mappings.get("dynamic"); + assertEquals("Dynamic field is not set to true", "true", dynamicValue.toString()); + Map properties = (Map) mappings.get("properties"); + assertTrue("Flattened field 'feature_data_feature_bytes' does not exist", properties.containsKey("feature_data_feature_bytes")); } public void testCreateAnomalyDetector() throws Exception { From f079bafb2c691d32b85d657c5d8a64dfaa8e7f91 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 09:19:00 -0800 Subject: [PATCH 08/21] clean up Signed-off-by: Jackie Han --- .../timeseries/indices/IndexManagement.java | 8 ++++++++ .../AbstractTimeSeriesActionHandler.java | 19 ++++++------------- .../ad/model/AnomalyDetectorTests.java | 2 +- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 57e25fb79..39b64c0a9 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1052,6 +1052,14 @@ public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionLis })); } + public String getFlattenedResultIndexAlias(String indexOrAliasName, String configId) { + return indexOrAliasName + "_flattened_" + configId.toLowerCase(Locale.ROOT); + } + + public String getFlattenResultIndexIngestPipelineId(String configId) { + return "flatten_result_index_ingest_pipeline" + configId.toLowerCase(Locale.ROOT); + } + public void validateCustomIndexForBackendJob( String resultIndexOrAlias, String securityLogId, diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 69cf0ddcc..61c47888d 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -454,14 +454,6 @@ protected void prepareConfigIndexing(boolean indexingDryRun, ActionListener l } } - private String getFlattenedResultIndexAlias(String configId) { - return config.getCustomResultIndexOrAlias() + "_flattened_" + configId.toLowerCase(Locale.ROOT); - } - - private String getFlattenResultIndexIngestPipelineId(String configId) { - return "flatten_result_index_ingest_pipeline" + configId.toLowerCase(Locale.ROOT); - } - private void handlePutRequest(boolean indexingDryRun, ActionListener listener) { handler.confirmJobRunning(clusterService, client, id, listener, () -> { handleFlattenResultIndexMappingUpdate(listener); @@ -474,8 +466,9 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; String configId = response.getId(); - String flattenedResultIndexAlias = getFlattenedResultIndexAlias(configId); - String pipelineId = getFlattenResultIndexIngestPipelineId(configId); + String flattenedResultIndexAlias = timeSeriesIndices + .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); timeSeriesIndices .initFlattenedResultIndex( @@ -504,8 +497,8 @@ private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConf } protected void setupIngestPipeline(String configId, ActionListener listener) { - String flattenedResultIndexAlias = getFlattenedResultIndexAlias(configId); - String pipelineId = getFlattenResultIndexIngestPipelineId(configId); + String flattenedResultIndexAlias = timeSeriesIndices.getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); try { BytesReference pipelineSource = createPipelineDefinition(flattenedResultIndexAlias); @@ -588,7 +581,7 @@ private void handleFlattenResultIndexMappingUpdate(ActionListener listener) { if (config.getFlattenResultIndexMapping() != null && config.getFlattenResultIndexMapping()) { setupIngestPipeline(id, listener); } else { - String pipelineId = getFlattenResultIndexIngestPipelineId(config.getId()); + String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { @Override diff --git a/src/test/java/org/opensearch/ad/model/AnomalyDetectorTests.java b/src/test/java/org/opensearch/ad/model/AnomalyDetectorTests.java index 4bc0e6b9e..bc634a869 100644 --- a/src/test/java/org/opensearch/ad/model/AnomalyDetectorTests.java +++ b/src/test/java/org/opensearch/ad/model/AnomalyDetectorTests.java @@ -1007,7 +1007,7 @@ public void testParseAnomalyDetector_withCustomIndex_withFlattenResultIndexMappi + "\"aggregation_query\":{\"XzewX\":{\"value_count\":{\"field\":\"ok\"}}}}],\"recency_emphasis\":3342," + "\"history\":62,\"last_update_time\":1717192049845,\"category_field\":[\"Tcqcb\"],\"result_index\":" + "\"opensearch-ad-plugin-result-test\",\"imputation_option\":{\"method\":\"ZERO\"},\"suggested_seasonality\":64,\"detection_interval\":{\"period\":" - + "{\"interval\":5,\"unit\":\"Minutes\"}},\"detector_type\":\"MULTI_ENTITY\",\"rules\":[],\"flatten_result_index_mapping\":true}"; + + "{\"interval\":5,\"unit\":\"Minutes\"}},\"detector_type\":\"MULTI_ENTITY\",\"rules\":[],\"flatten_custom_result_index\":true}"; AnomalyDetector parsedDetector = AnomalyDetector.parse(TestHelpers.parser(detectorString), "id", 1L, null, null); assertEquals(true, (boolean) parsedDetector.getFlattenResultIndexMapping()); } From edee31cb3b6ca504cd202c5231be5710cf582ecf Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 11:05:54 -0800 Subject: [PATCH 09/21] add more IT Signed-off-by: Jackie Han --- .../ad/rest/AnomalyDetectorRestApiIT.java | 75 ++++++++++++++++++- 1 file changed, 74 insertions(+), 1 deletion(-) diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 65516a241..550d6ef32 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -194,7 +194,7 @@ public void testCreateAnomalyDetectorWithDuplicateName() throws Exception { ); } - public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception { + public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector( INDEX_NAME, ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), @@ -258,6 +258,79 @@ public void testCreateAnomalyDetectorWithFlattenedResultIndex() throws Exception assertTrue("Flattened field 'feature_data_feature_bytes' does not exist", properties.containsKey("feature_data_feature_bytes")); } + public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipeline() throws Exception { + AnomalyDetector detector = createIndexAndGetAnomalyDetector( + INDEX_NAME, + ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), + false, + true + ); + + // test behavior when AD is enabled + updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); + Response response = TestHelpers + .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + assertEquals("Create anomaly detector with flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); + Map responseMap = entityAsMap(response); + String id = (String) responseMap.get("_id"); + String expectedFlattenedIndex = String + .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); + String expectedPipelineId = String.format(Locale.ROOT, "flatten_result_index_ingest_pipeline%s", id.toLowerCase(Locale.ROOT)); + String getIngestPipelineEndpoint = String.format(Locale.ROOT, "_ingest/pipeline/%s", expectedPipelineId); + Response getPipelineResponse = TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null); + assertEquals( + "Expected 200 response but got: " + getPipelineResponse.getStatusLine().getStatusCode(), + 200, + getPipelineResponse.getStatusLine().getStatusCode() + ); + List features = detector.getFeatureAttributes(); + AnomalyDetector newDetector = new AnomalyDetector( + id, + detector.getVersion(), + detector.getName(), + detector.getDescription(), + detector.getTimeField(), + detector.getIndices(), + features, + detector.getFilterQuery(), + detector.getInterval(), + detector.getWindowDelay(), + detector.getShingleSize(), + detector.getUiMetadata(), + detector.getSchemaVersion(), + detector.getLastUpdateTime(), + null, + detector.getUser(), + detector.getCustomResultIndexOrAlias(), + TestHelpers.randomImputationOption(features), + randomIntBetween(1, 10000), + randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), + randomIntBetween(1, 1000), + null, + null, + null, + null, + false, + detector.getLastBreakingUIChangeTime() + ); + Response updateResponse = TestHelpers + .makeRequest( + client(), + "PUT", + TestHelpers.AD_BASE_DETECTORS_URI + "/" + id + "?refresh=true", + ImmutableMap.of(), + TestHelpers.toHttpEntity(newDetector), + null + ); + assertEquals("Update anomaly detector failed", RestStatus.OK, TestHelpers.restStatus(updateResponse)); + ResponseException responseException = expectThrows( + ResponseException.class, + () -> TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null) + ); + int statusCode = responseException.getResponse().getStatusLine().getStatusCode(); + assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); + } + public void testCreateAnomalyDetector() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME); updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); From 9b371ba496a390e2a89497bd3f9ff783f1d68ad9 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Mon, 27 Jan 2025 23:01:28 -0800 Subject: [PATCH 10/21] add more IT Signed-off-by: Jackie Han --- .../timeseries/indices/IndexManagement.java | 53 +++++++------- .../AbstractTimeSeriesActionHandler.java | 41 +++++++---- .../timeseries/util/RestHandlerUtils.java | 20 ++++++ .../ad/rest/AnomalyDetectorRestApiIT.java | 72 +++++++++++++++++++ 4 files changed, 147 insertions(+), 39 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 39b64c0a9..0d6778661 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1020,36 +1020,39 @@ public void initCustomResultIndexAndExecute(String resultIndexOrAlias, Execu * creates flattened result index * @param flattenedResultIndexAlias the flattened result index alias * @param actionListener the action listener - * @throws IOException */ - public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionListener actionListener) - throws IOException { - String indexName = getCustomResultIndexPattern(flattenedResultIndexAlias); - logger.info("Initializing flattened result index: {}", indexName); + public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionListener actionListener) { + try { + String indexName = getCustomResultIndexPattern(flattenedResultIndexAlias); + logger.info("Initializing flattened result index: {}", indexName); - CreateIndexRequest request = new CreateIndexRequest(indexName) - .mapping(getFlattenedResultMappings(), XContentType.JSON) - .settings(settings); + CreateIndexRequest request = new CreateIndexRequest(indexName) + .mapping(getFlattenedResultMappings(), XContentType.JSON) + .settings(settings); - if (flattenedResultIndexAlias != null) { - request.alias(new Alias(flattenedResultIndexAlias)); - } + if (flattenedResultIndexAlias != null) { + request.alias(new Alias(flattenedResultIndexAlias)); + } - choosePrimaryShards(request, false); + choosePrimaryShards(request, false); - adminClient.indices().create(request, ActionListener.wrap(response -> { - if (response.isAcknowledged()) { - logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); - actionListener.onResponse(response); - } else { - String errorMsg = "Index creation not acknowledged for index: " + indexName; - logger.error(errorMsg); - actionListener.onFailure(new IllegalStateException(errorMsg)); - } - }, exception -> { - logger.error("Failed to create flattened result index: {}", indexName, exception); - actionListener.onFailure(exception); - })); + adminClient.indices().create(request, ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); + actionListener.onResponse(response); + } else { + String errorMsg = "Index creation not acknowledged for index: " + indexName; + logger.error(errorMsg); + actionListener.onFailure(new IllegalStateException(errorMsg)); + } + }, exception -> { + logger.error("Failed to create flattened result index: {}", indexName, exception); + actionListener.onFailure(exception); + })); + } catch (Exception e) { + logger.error("Error while initializing flattened result index: {}", flattenedResultIndexAlias, e); + actionListener.onFailure(e); + } } public String getFlattenedResultIndexAlias(String indexOrAliasName, String configId) { diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 61c47888d..2b713ea2d 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -39,7 +39,6 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.support.replication.ReplicationResponse; -import org.opensearch.ad.transport.IndexAnomalyDetectorResponse; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -456,16 +455,14 @@ protected void prepareConfigIndexing(boolean indexingDryRun, ActionListener l private void handlePutRequest(boolean indexingDryRun, ActionListener listener) { handler.confirmJobRunning(clusterService, client, id, listener, () -> { - handleFlattenResultIndexMappingUpdate(listener); updateConfig(id, indexingDryRun, listener); }, xContentRegistry); } private void handlePostRequest(boolean indexingDryRun, ActionListener listener) { createConfig(indexingDryRun, ActionListener.wrap(createConfigResponse -> { - if (shouldHandleFlattening(indexingDryRun, createConfigResponse)) { - IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) createConfigResponse; - String configId = response.getId(); + if (shouldHandleFlattening(indexingDryRun)) { + String configId = RestHandlerUtils.getConfigIdFromIndexResponse(createConfigResponse); String flattenedResultIndexAlias = timeSeriesIndices .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); @@ -487,13 +484,10 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene }, listener::onFailure)); } - private boolean shouldHandleFlattening(boolean indexingDryRun, Object createConfigResponse) { + private boolean shouldHandleFlattening(boolean indexingDryRun) { Boolean flattenResultIndexMapping = config.getFlattenResultIndexMapping(); - return !indexingDryRun - && config.getCustomResultIndexOrAlias() != null - && Boolean.TRUE.equals(flattenResultIndexMapping) - && createConfigResponse instanceof IndexAnomalyDetectorResponse; + return !indexingDryRun && config.getCustomResultIndexOrAlias() != null && Boolean.TRUE.equals(flattenResultIndexMapping); } protected void setupIngestPipeline(String configId, ActionListener listener) { @@ -574,13 +568,13 @@ protected void updateResultIndexSetting(String pipelineId, String flattenedResul })); } - private void handleFlattenResultIndexMappingUpdate(ActionListener listener) { + private void handleFlattenResultIndexMappingUpdate(Config existingConfig, ActionListener listener) { if (config.getCustomResultIndexOrAlias() == null) { return; } - if (config.getFlattenResultIndexMapping() != null && config.getFlattenResultIndexMapping()) { - setupIngestPipeline(id, listener); - } else { + if (Boolean.TRUE.equals(existingConfig.getFlattenResultIndexMapping()) + && Boolean.FALSE.equals(config.getFlattenResultIndexMapping()) + && existingConfig.getCustomResultIndexOrAlias() != null) { String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { @@ -611,6 +605,24 @@ public void onFailure(Exception e) { } } }); + } else if (Boolean.FALSE.equals(existingConfig.getFlattenResultIndexMapping()) + && Boolean.TRUE.equals(config.getFlattenResultIndexMapping()) + && existingConfig.getCustomResultIndexOrAlias() != null + ) { + String flattenedResultIndexAlias = timeSeriesIndices + .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), config.getId()); + String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); + timeSeriesIndices + .initFlattenedResultIndex( + flattenedResultIndexAlias, + ActionListener.wrap(initResponse -> setupIngestPipeline(config.getId(), ActionListener.wrap(pipelineResponse -> { + updateResultIndexSetting( + pipelineId, + flattenedResultIndexAlias, + ActionListener.wrap(updateResponse -> listener.onResponse(updateResponse), listener::onFailure) + ); + }, listener::onFailure)), listener::onFailure) + ); } } @@ -661,6 +673,7 @@ private void onGetConfigResponse(GetResponse response, boolean indexingDryRun, S breakingUIChange = true; } } + handleFlattenResultIndexMappingUpdate(existingConfig, listener); ActionListener confirmBatchRunningListener = ActionListener .wrap( diff --git a/src/main/java/org/opensearch/timeseries/util/RestHandlerUtils.java b/src/main/java/org/opensearch/timeseries/util/RestHandlerUtils.java index ec45ff02c..4d9924c60 100644 --- a/src/main/java/org/opensearch/timeseries/util/RestHandlerUtils.java +++ b/src/main/java/org/opensearch/timeseries/util/RestHandlerUtils.java @@ -26,17 +26,20 @@ import org.opensearch.OpenSearchStatusException; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.ShardSearchFailure; +import org.opensearch.ad.transport.IndexAnomalyDetectorResponse; import org.opensearch.common.Nullable; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.action.ActionResponse; import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.forecast.transport.IndexForecasterResponse; import org.opensearch.index.IndexNotFoundException; import org.opensearch.indices.InvalidIndexNameException; import org.opensearch.rest.RestChannel; @@ -290,4 +293,21 @@ public static Entity buildEntity(RestRequest request, String detectorId) throws // not a valid profile request with correct entity information return null; } + + public static String getConfigIdFromIndexResponse(ActionResponse actionResponse) { + String configId; + if (actionResponse instanceof IndexAnomalyDetectorResponse) { + IndexAnomalyDetectorResponse response = (IndexAnomalyDetectorResponse) actionResponse; + configId = response.getId(); + logger.info("Handling IndexAnomalyDetectorResponse for configId: {}", configId); + } else if (actionResponse instanceof IndexForecasterResponse) { + IndexForecasterResponse response = (IndexForecasterResponse) actionResponse; + configId = response.getId(); + logger.info("Handling IndexForecasterResponse for configId: {}", configId); + } else { + throw new IllegalStateException("Unexpected response type: " + actionResponse.getClass().getName()); + } + return configId; + } + } diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 550d6ef32..58227a123 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -331,6 +331,78 @@ public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipe assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); } + public void testUpdateAnomalyDetector_enableFlattenResultIndex_shouldCreatePipeline() throws Exception { + AnomalyDetector detector = createIndexAndGetAnomalyDetector( + INDEX_NAME, + ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), + false, + false + ); + // test behavior when AD is enabled + updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); + // create a detector with flatten result index disabled, shouldn't find related ingest pipeline + Response response = TestHelpers + .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + assertEquals("Create anomaly detector without flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); + Map responseMap = entityAsMap(response); + String id = (String) responseMap.get("_id"); + String expectedPipelineId = String.format(Locale.ROOT, "flatten_result_index_ingest_pipeline%s", id.toLowerCase(Locale.ROOT)); + String getIngestPipelineEndpoint = String.format(Locale.ROOT, "_ingest/pipeline/%s", expectedPipelineId); + ResponseException responseException = expectThrows( + ResponseException.class, + () -> TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null) + ); + int statusCode = responseException.getResponse().getStatusLine().getStatusCode(); + assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); + // update the detector with flatten result index enabled, should be able to find ingest pipeline + List features = detector.getFeatureAttributes(); + AnomalyDetector newDetector = new AnomalyDetector( + id, + detector.getVersion(), + detector.getName(), + detector.getDescription(), + detector.getTimeField(), + detector.getIndices(), + features, + detector.getFilterQuery(), + detector.getInterval(), + detector.getWindowDelay(), + detector.getShingleSize(), + detector.getUiMetadata(), + detector.getSchemaVersion(), + detector.getLastUpdateTime(), + null, + detector.getUser(), + detector.getCustomResultIndexOrAlias(), + TestHelpers.randomImputationOption(features), + randomIntBetween(1, 10000), + randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), + randomIntBetween(1, 1000), + null, + null, + null, + null, + true, + detector.getLastBreakingUIChangeTime() + ); + Response updateResponse = TestHelpers + .makeRequest( + client(), + "PUT", + TestHelpers.AD_BASE_DETECTORS_URI + "/" + id + "?refresh=true", + ImmutableMap.of(), + TestHelpers.toHttpEntity(newDetector), + null + ); + assertEquals("Update anomaly detector failed", RestStatus.OK, TestHelpers.restStatus(updateResponse)); + Response getPipelineResponse = TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null); + assertEquals( + "Expected 200 response but got: " + getPipelineResponse.getStatusLine().getStatusCode(), + 200, + getPipelineResponse.getStatusLine().getStatusCode() + ); + } + public void testCreateAnomalyDetector() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME); updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); From c8449222c9d30c3e443ad5de75b94a1b24376fac Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 01:44:12 -0800 Subject: [PATCH 11/21] add more IT Signed-off-by: Jackie Han --- .../timeseries/constant/CommonMessages.java | 1 + .../timeseries/indices/IndexManagement.java | 4 +- .../AbstractTimeSeriesActionHandler.java | 37 ++++------ .../ad/rest/AnomalyDetectorRestApiIT.java | 72 ------------------- 4 files changed, 18 insertions(+), 96 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java b/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java index 8ffed43b7..460be7d22 100644 --- a/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java +++ b/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java @@ -44,6 +44,7 @@ public static String getTooManyCategoricalFieldErr(int limit) { public static String FAIL_TO_FIND_CONFIG_MSG = "Can't find config with id: "; public static final String CAN_NOT_CHANGE_CATEGORY_FIELD = "Can't change category field"; public static final String CAN_NOT_CHANGE_CUSTOM_RESULT_INDEX = "Can't change custom result index"; + public static final String CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX = "Can't change flatten result index"; public static final String CATEGORICAL_FIELD_TYPE_ERR_MSG = "Categorical field %s must be of type keyword or ip."; // Modifying message for FEATURE below may break the parseADValidationException method of ValidateAnomalyDetectorTransportAction public static final String FEATURE_INVALID_MSG_PREFIX = "Feature has an invalid query"; diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 0d6778661..7a504aabb 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1027,8 +1027,8 @@ public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionLis logger.info("Initializing flattened result index: {}", indexName); CreateIndexRequest request = new CreateIndexRequest(indexName) - .mapping(getFlattenedResultMappings(), XContentType.JSON) - .settings(settings); + .mapping(getFlattenedResultMappings(), XContentType.JSON) + .settings(settings); if (flattenedResultIndexAlias != null) { request.alias(new Alias(flattenedResultIndexAlias)); diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 2b713ea2d..f6af23609 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -454,9 +454,15 @@ protected void prepareConfigIndexing(boolean indexingDryRun, ActionListener l } private void handlePutRequest(boolean indexingDryRun, ActionListener listener) { - handler.confirmJobRunning(clusterService, client, id, listener, () -> { - updateConfig(id, indexingDryRun, listener); - }, xContentRegistry); + handler + .confirmJobRunning( + clusterService, + client, + id, + listener, + () -> { updateConfig(id, indexingDryRun, listener); }, + xContentRegistry + ); } private void handlePostRequest(boolean indexingDryRun, ActionListener listener) { @@ -573,8 +579,8 @@ private void handleFlattenResultIndexMappingUpdate(Config existingConfig, Action return; } if (Boolean.TRUE.equals(existingConfig.getFlattenResultIndexMapping()) - && Boolean.FALSE.equals(config.getFlattenResultIndexMapping()) - && existingConfig.getCustomResultIndexOrAlias() != null) { + && Boolean.FALSE.equals(config.getFlattenResultIndexMapping()) + && existingConfig.getCustomResultIndexOrAlias() != null) { String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { @@ -606,23 +612,10 @@ public void onFailure(Exception e) { } }); } else if (Boolean.FALSE.equals(existingConfig.getFlattenResultIndexMapping()) - && Boolean.TRUE.equals(config.getFlattenResultIndexMapping()) - && existingConfig.getCustomResultIndexOrAlias() != null - ) { - String flattenedResultIndexAlias = timeSeriesIndices - .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), config.getId()); - String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); - timeSeriesIndices - .initFlattenedResultIndex( - flattenedResultIndexAlias, - ActionListener.wrap(initResponse -> setupIngestPipeline(config.getId(), ActionListener.wrap(pipelineResponse -> { - updateResultIndexSetting( - pipelineId, - flattenedResultIndexAlias, - ActionListener.wrap(updateResponse -> listener.onResponse(updateResponse), listener::onFailure) - ); - }, listener::onFailure)), listener::onFailure) - ); + && Boolean.TRUE.equals(config.getFlattenResultIndexMapping()) + && existingConfig.getCustomResultIndexOrAlias() != null) { + listener.onFailure(new OpenSearchStatusException(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX, RestStatus.BAD_REQUEST)); + return; } } diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 58227a123..550d6ef32 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -331,78 +331,6 @@ public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipe assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); } - public void testUpdateAnomalyDetector_enableFlattenResultIndex_shouldCreatePipeline() throws Exception { - AnomalyDetector detector = createIndexAndGetAnomalyDetector( - INDEX_NAME, - ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), - false, - false - ); - // test behavior when AD is enabled - updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); - // create a detector with flatten result index disabled, shouldn't find related ingest pipeline - Response response = TestHelpers - .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); - assertEquals("Create anomaly detector without flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); - Map responseMap = entityAsMap(response); - String id = (String) responseMap.get("_id"); - String expectedPipelineId = String.format(Locale.ROOT, "flatten_result_index_ingest_pipeline%s", id.toLowerCase(Locale.ROOT)); - String getIngestPipelineEndpoint = String.format(Locale.ROOT, "_ingest/pipeline/%s", expectedPipelineId); - ResponseException responseException = expectThrows( - ResponseException.class, - () -> TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null) - ); - int statusCode = responseException.getResponse().getStatusLine().getStatusCode(); - assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); - // update the detector with flatten result index enabled, should be able to find ingest pipeline - List features = detector.getFeatureAttributes(); - AnomalyDetector newDetector = new AnomalyDetector( - id, - detector.getVersion(), - detector.getName(), - detector.getDescription(), - detector.getTimeField(), - detector.getIndices(), - features, - detector.getFilterQuery(), - detector.getInterval(), - detector.getWindowDelay(), - detector.getShingleSize(), - detector.getUiMetadata(), - detector.getSchemaVersion(), - detector.getLastUpdateTime(), - null, - detector.getUser(), - detector.getCustomResultIndexOrAlias(), - TestHelpers.randomImputationOption(features), - randomIntBetween(1, 10000), - randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), - randomIntBetween(1, 1000), - null, - null, - null, - null, - true, - detector.getLastBreakingUIChangeTime() - ); - Response updateResponse = TestHelpers - .makeRequest( - client(), - "PUT", - TestHelpers.AD_BASE_DETECTORS_URI + "/" + id + "?refresh=true", - ImmutableMap.of(), - TestHelpers.toHttpEntity(newDetector), - null - ); - assertEquals("Update anomaly detector failed", RestStatus.OK, TestHelpers.restStatus(updateResponse)); - Response getPipelineResponse = TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null); - assertEquals( - "Expected 200 response but got: " + getPipelineResponse.getStatusLine().getStatusCode(), - 200, - getPipelineResponse.getStatusLine().getStatusCode() - ); - } - public void testCreateAnomalyDetector() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME); updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); From d4b9262752412d809743dc5b379247604fb8f71a Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 11:52:37 -0800 Subject: [PATCH 12/21] address comments --- .../ad/indices/ADIndexManagement.java | 2 - .../opensearch/ad/model/AnomalyDetector.java | 2 +- .../opensearch/forecast/model/Forecaster.java | 2 +- .../opensearch/timeseries/model/Config.java | 8 +-- .../AbstractTimeSeriesActionHandler.java | 10 +-- .../ad/rest/AnomalyDetectorRestApiIT.java | 65 +++++++++++++++++++ 6 files changed, 76 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java index 8248d4520..9b057ad45 100644 --- a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java +++ b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java @@ -60,8 +60,6 @@ public class ADIndexManagement extends IndexManagement { // The index name pattern to query all AD result, history and current AD result public static final String ALL_AD_RESULTS_INDEX_PATTERN = ".opendistro-anomaly-results*"; - // private static final ObjectMapper objectMapper = new ObjectMapper(); - /** * Constructor function * diff --git a/src/main/java/org/opensearch/ad/model/AnomalyDetector.java b/src/main/java/org/opensearch/ad/model/AnomalyDetector.java index 7b3f573ff..9f9b46f32 100644 --- a/src/main/java/org/opensearch/ad/model/AnomalyDetector.java +++ b/src/main/java/org/opensearch/ad/model/AnomalyDetector.java @@ -592,7 +592,7 @@ public static AnomalyDetector parse( case RESULT_INDEX_FIELD_TTL: customResultIndexTTL = onlyParseNumberValue(parser); break; - case FLATTEN_RESULT_INDEX_MAPPING: + case FLATTEN_CUSTOM_RESULT_INDEX: flattenResultIndexMapping = onlyParseBooleanValue(parser); break; case BREAKING_UI_CHANGE_TIME: diff --git a/src/main/java/org/opensearch/forecast/model/Forecaster.java b/src/main/java/org/opensearch/forecast/model/Forecaster.java index 756b5c4e0..5f0333cdb 100644 --- a/src/main/java/org/opensearch/forecast/model/Forecaster.java +++ b/src/main/java/org/opensearch/forecast/model/Forecaster.java @@ -437,7 +437,7 @@ public static Forecaster parse( case RESULT_INDEX_FIELD_TTL: customResultIndexTTL = parser.intValue(); break; - case FLATTEN_RESULT_INDEX_MAPPING: + case FLATTEN_CUSTOM_RESULT_INDEX: flattenResultIndexMapping = parser.booleanValue(); break; case BREAKING_UI_CHANGE_TIME: diff --git a/src/main/java/org/opensearch/timeseries/model/Config.java b/src/main/java/org/opensearch/timeseries/model/Config.java index 81f88c1ba..eda7a32ec 100644 --- a/src/main/java/org/opensearch/timeseries/model/Config.java +++ b/src/main/java/org/opensearch/timeseries/model/Config.java @@ -79,7 +79,7 @@ public abstract class Config implements Writeable, ToXContentObject { public static final String RESULT_INDEX_FIELD_MIN_SIZE = "result_index_min_size"; public static final String RESULT_INDEX_FIELD_MIN_AGE = "result_index_min_age"; public static final String RESULT_INDEX_FIELD_TTL = "result_index_ttl"; - public static final String FLATTEN_RESULT_INDEX_MAPPING = "flatten_custom_result_index"; + public static final String FLATTEN_CUSTOM_RESULT_INDEX = "flatten_custom_result_index"; // Changing categorical field, feature attributes, interval, windowDelay, time field, horizon, indices, // result index would force us to display results only from the most recent update. Otherwise, // the UI appear cluttered and unclear. @@ -533,7 +533,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(RESULT_INDEX_FIELD_TTL, customResultIndexTTL); } if (flattenResultIndexMapping != null) { - builder.field(FLATTEN_RESULT_INDEX_MAPPING, flattenResultIndexMapping); + builder.field(FLATTEN_CUSTOM_RESULT_INDEX, flattenResultIndexMapping); } if (lastUIBreakingChangeTime != null) { builder.field(BREAKING_UI_CHANGE_TIME, lastUIBreakingChangeTime.toEpochMilli()); @@ -746,8 +746,8 @@ public Integer getCustomResultIndexTTL() { return customResultIndexTTL; } - public Boolean getFlattenResultIndexMapping() { - return flattenResultIndexMapping; + public boolean getFlattenResultIndexMapping() { + return flattenResultIndexMapping != null ? flattenResultIndexMapping : false; } public Instant getLastBreakingUIChangeTime() { diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index f6af23609..b6f1353a9 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -529,7 +529,7 @@ private BytesReference createPipelineDefinition(String indexName) throws IOExcep XContentBuilder pipelineBuilder = XContentFactory.jsonBuilder(); pipelineBuilder.startObject(); { - pipelineBuilder.field("description", "Ingest pipeline for anomaly detector with result index: " + indexName); + pipelineBuilder.field("description", "Ingest pipeline for flattening result index: " + indexName); pipelineBuilder.startArray("processors"); { pipelineBuilder.startObject(); @@ -578,8 +578,8 @@ private void handleFlattenResultIndexMappingUpdate(Config existingConfig, Action if (config.getCustomResultIndexOrAlias() == null) { return; } - if (Boolean.TRUE.equals(existingConfig.getFlattenResultIndexMapping()) - && Boolean.FALSE.equals(config.getFlattenResultIndexMapping()) + if (existingConfig.getFlattenResultIndexMapping() + && !config.getFlattenResultIndexMapping() && existingConfig.getCustomResultIndexOrAlias() != null) { String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { @@ -611,8 +611,8 @@ public void onFailure(Exception e) { } } }); - } else if (Boolean.FALSE.equals(existingConfig.getFlattenResultIndexMapping()) - && Boolean.TRUE.equals(config.getFlattenResultIndexMapping()) + } else if (!existingConfig.getFlattenResultIndexMapping() + && config.getFlattenResultIndexMapping() && existingConfig.getCustomResultIndexOrAlias() != null) { listener.onFailure(new OpenSearchStatusException(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX, RestStatus.BAD_REQUEST)); return; diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 550d6ef32..85c67788b 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -331,6 +331,71 @@ public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipe assertEquals("Expected 404 response but got: " + statusCode, 404, statusCode); } + public void testUpdateAnomalyDetectorFlattenResultIndexField() throws Exception { + TestHelpers.createIndexWithTimeField(client(), INDEX_NAME, TIME_FIELD, false); + String testIndexData = "{\"keyword-field\": \"field-1\", \"ip-field\": \"1.2.3.4\", \"timestamp\": 1}"; + TestHelpers.ingestDataToIndex(client(), INDEX_NAME, TestHelpers.toHttpEntity(testIndexData)); + AnomalyDetector detector = TestHelpers + .randomDetector( + ImmutableList.of(TestHelpers.randomFeature("feature_bytes", "agg", true)), + INDEX_NAME, + 5, + TIME_FIELD, + null, + ADCommonName.CUSTOM_RESULT_INDEX_PREFIX + "test" + ); + Response response = TestHelpers + .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); + assertEquals("Create anomaly detector failed", RestStatus.CREATED, TestHelpers.restStatus(response)); + Map responseMap = entityAsMap(response); + String id = (String) responseMap.get("_id"); + List features = detector.getFeatureAttributes(); + long expectedFeatures = features.stream().filter(Feature::getEnabled).count(); + AnomalyDetector newDetector = new AnomalyDetector( + id, + null, + detector.getName(), + detector.getDescription(), + detector.getTimeField(), + detector.getIndices(), + features, + detector.getFilterQuery(), + detector.getInterval(), + detector.getWindowDelay(), + detector.getShingleSize(), + detector.getUiMetadata(), + detector.getSchemaVersion(), + detector.getLastUpdateTime(), + detector.getCategoryFields(), + detector.getUser(), + detector.getCustomResultIndexOrAlias(), + TestHelpers.randomImputationOption(features), + randomIntBetween(1, 10000), + randomInt(TimeSeriesSettings.MAX_SHINGLE_SIZE / 2), + randomIntBetween(1, 1000), + detector.getRules(), + null, + null, + null, + true, + detector.getLastBreakingUIChangeTime() + ); + + Exception ex = expectThrows( + ResponseException.class, + () -> TestHelpers + .makeRequest( + client(), + "PUT", + TestHelpers.AD_BASE_DETECTORS_URI + "/" + id + "?refresh=true", + ImmutableMap.of(), + TestHelpers.toHttpEntity(newDetector), + null + ) + ); + assertThat(ex.getMessage(), containsString(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX)); + } + public void testCreateAnomalyDetector() throws Exception { AnomalyDetector detector = createIndexAndGetAnomalyDetector(INDEX_NAME); updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); From 781d2da3d45887063e5b0954af21683da30c2ef1 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 12:29:25 -0800 Subject: [PATCH 13/21] address comments Signed-off-by: Jackie Han --- .../timeseries/indices/IndexManagement.java | 18 ++++++++++++------ .../AbstractTimeSeriesActionHandler.java | 3 +++ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 7a504aabb..a5750fb5d 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1307,14 +1307,20 @@ protected void rolloverAndDeleteHistoryIndex( } // perform rollover and delete on found custom result index alias - candidateResultAliases.forEach(config -> handleCustomResultIndex(config, resultIndex)); + candidateResultAliases.forEach(config -> { + handleResultIndexRolloverAndDelete(config.getCustomResultIndexOrAlias(), config, resultIndex); + if (config.getFlattenResultIndexMapping()) { + String flattenedResultIndexAlias = getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), config.getId()); + handleResultIndexRolloverAndDelete(flattenedResultIndexAlias, config, resultIndex); + } + }); }, e -> { logger.error("Failed to get configs with custom result index alias.", e); })); } - private void handleCustomResultIndex(Config config, IndexType resultIndex) { + private void handleResultIndexRolloverAndDelete(String indexAlias, Config config, IndexType resultIndex) { RolloverRequest rolloverRequest = buildRolloverRequest( - config.getCustomResultIndexOrAlias(), - getCustomResultIndexPattern(config.getCustomResultIndexOrAlias()) + indexAlias, + getCustomResultIndexPattern(indexAlias) ); // add rollover conditions if found in config @@ -1327,9 +1333,9 @@ private void handleCustomResultIndex(Config config, IndexType resultIndex) { // perform rollover and delete on custom result index alias proceedWithRolloverAndDelete( - config.getCustomResultIndexOrAlias(), + indexAlias, rolloverRequest, - getAllCustomResultIndexPattern(config.getCustomResultIndexOrAlias()), + getAllCustomResultIndexPattern(indexAlias), resultIndex, config.getCustomResultIndexTTL() ); diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index b6f1353a9..f4c6e63f6 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -614,6 +614,9 @@ public void onFailure(Exception e) { } else if (!existingConfig.getFlattenResultIndexMapping() && config.getFlattenResultIndexMapping() && existingConfig.getCustomResultIndexOrAlias() != null) { + // customers can choose to use a flattened result index for newly created detectors and disable it for those detectors. + // however, since enabling the flattened result index creates additional resources and due to bwc concerns, + // we do not allow customers to enable this feature for existing running detectors. listener.onFailure(new OpenSearchStatusException(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX, RestStatus.BAD_REQUEST)); return; } From a623fcf524f1343b8a55c99276e2096096357262 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 14:46:47 -0800 Subject: [PATCH 14/21] address comments Signed-off-by: Jackie Han --- .../timeseries/indices/IndexManagement.java | 5 +- .../AbstractTimeSeriesActionHandler.java | 141 +++++++++++------- 2 files changed, 86 insertions(+), 60 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index a5750fb5d..398123928 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1318,10 +1318,7 @@ protected void rolloverAndDeleteHistoryIndex( } private void handleResultIndexRolloverAndDelete(String indexAlias, Config config, IndexType resultIndex) { - RolloverRequest rolloverRequest = buildRolloverRequest( - indexAlias, - getCustomResultIndexPattern(indexAlias) - ); + RolloverRequest rolloverRequest = buildRolloverRequest(indexAlias, getCustomResultIndexPattern(indexAlias)); // add rollover conditions if found in config if (config.getCustomResultIndexMinAge() != null) { diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index f4c6e63f6..ed264c7f5 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -22,6 +22,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.OpenSearchStatusException; +import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest; +import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsAction; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsRequest; @@ -37,7 +39,6 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.WriteRequest; -import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.support.replication.ReplicationResponse; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; @@ -574,54 +575,6 @@ protected void updateResultIndexSetting(String pipelineId, String flattenedResul })); } - private void handleFlattenResultIndexMappingUpdate(Config existingConfig, ActionListener listener) { - if (config.getCustomResultIndexOrAlias() == null) { - return; - } - if (existingConfig.getFlattenResultIndexMapping() - && !config.getFlattenResultIndexMapping() - && existingConfig.getCustomResultIndexOrAlias() != null) { - String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(config.getId()); - client.admin().cluster().deletePipeline(new DeletePipelineRequest(pipelineId), new ActionListener() { - - @Override - public void onResponse(AcknowledgedResponse acknowledgedResponse) { - if (acknowledgedResponse.isAcknowledged()) { - logger.info("Ingest pipeline deleted successfully for pipelineId: {}", pipelineId); - } else { - logger.error("Failed to delete ingest pipeline for pipelineId: {}", pipelineId); - listener - .onFailure( - new OpenSearchStatusException( - "Ingest pipeline deletion was not acknowledged for pipelineId: " + pipelineId, - RestStatus.INTERNAL_SERVER_ERROR - ) - ); - } - } - - @Override - public void onFailure(Exception e) { - if (e instanceof OpenSearchStatusException && ((OpenSearchStatusException) e).status() == RestStatus.NOT_FOUND) { - logger.info("Ingest pipeline [{}] not found, skipping deletion.", pipelineId); - listener.onResponse(null); - } else { - logger.error("Error while deleting ingest pipeline for pipelineId: {}", pipelineId, e); - listener.onFailure(e); - } - } - }); - } else if (!existingConfig.getFlattenResultIndexMapping() - && config.getFlattenResultIndexMapping() - && existingConfig.getCustomResultIndexOrAlias() != null) { - // customers can choose to use a flattened result index for newly created detectors and disable it for those detectors. - // however, since enabling the flattened result index creates additional resources and due to bwc concerns, - // we do not allow customers to enable this feature for existing running detectors. - listener.onFailure(new OpenSearchStatusException(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX, RestStatus.BAD_REQUEST)); - return; - } - } - protected void updateConfig(String id, boolean indexingDryRun, ActionListener listener) { GetRequest request = new GetRequest(CommonName.CONFIG_INDEX, id); client @@ -663,28 +616,104 @@ private void onGetConfigResponse(GetResponse response, boolean indexingDryRun, S ); return; } + if (!existingConfig.getFlattenResultIndexMapping() + && config.getFlattenResultIndexMapping() + && existingConfig.getCustomResultIndexOrAlias() != null) { + // customers can choose to use a flattened result index for newly created detectors and disable it for those detectors. + // however, since enabling the flattened result index creates additional resources and due to bwc concerns, + // we do not allow customers to enable this feature for existing running detectors. + listener + .onFailure( + new OpenSearchStatusException(CommonMessages.CAN_NOT_CHANGE_FLATTEN_RESULT_INDEX, RestStatus.BAD_REQUEST) + ); + return; + } } else { if (!ParseUtils.listEqualsWithoutConsideringOrder(existingConfig.getCategoryFields(), config.getCategoryFields()) || !Objects.equals(existingConfig.getCustomResultIndexOrAlias(), config.getCustomResultIndexOrAlias())) { breakingUIChange = true; } } - handleFlattenResultIndexMappingUpdate(existingConfig, listener); + ActionListener confirmBatchRunningListener; - ActionListener confirmBatchRunningListener = ActionListener - .wrap( - r -> searchConfigInputIndices(id, indexingDryRun, listener), - // can't update config if there is task running - listener::onFailure - ); + // when unselecting flatten result index, need to clean up alias and ingest pipeline resources + if (existingConfig.getFlattenResultIndexMapping() + && !config.getFlattenResultIndexMapping() + && existingConfig.getCustomResultIndexOrAlias() != null) { + confirmBatchRunningListener = ActionListener + .wrap( + r -> getFlattenResultAliasIndex(existingConfig, listener, id, indexingDryRun), + // can't update config if there is task running + listener::onFailure + ); + } else { + confirmBatchRunningListener = ActionListener + .wrap( + r -> searchConfigInputIndices(id, indexingDryRun, listener), + // can't update config if there is task running + listener::onFailure + ); + } handler.confirmBatchRunning(id, batchTasks, confirmBatchRunningListener); } catch (Exception e) { String message = "Failed to parse config " + id; logger.error(message, e); listener.onFailure(new OpenSearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR)); } + } + + private void getFlattenResultAliasIndex(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { + String flattenResultIndexAlias = timeSeriesIndices + .getFlattenedResultIndexAlias(existingConfig.getCustomResultIndexOrAlias(), existingConfig.getId()); + GetAliasesRequest getAliasesRequest = new GetAliasesRequest(flattenResultIndexAlias); + client.admin().indices().getAliases(getAliasesRequest, ActionListener.wrap(getAliasesResponse -> { + Set indices = getAliasesResponse.getAliases().keySet(); + if (indices.isEmpty()) { + return; + } + String indexName = indices.iterator().next(); + deleteAlias(indexName, flattenResultIndexAlias, existingConfig, listener, id, indexingDryRun); + }, exception -> listener.onFailure(exception))); + } + + private void deleteAlias( + String indexName, + String aliasName, + Config existingConfig, + ActionListener listener, + String id, + boolean indexingDryRun + ) { + IndicesAliasesRequest indicesAliasesRequest = new IndicesAliasesRequest(); + indicesAliasesRequest.addAliasAction(IndicesAliasesRequest.AliasActions.remove().index(indexName).alias(aliasName)); + client + .admin() + .indices() + .aliases( + indicesAliasesRequest, + ActionListener + .wrap( + deleteAliasResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), + exception -> listener.onFailure(exception) + ) + ); + } + + private void deleteIngestPipeline(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { + String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(existingConfig.getId()); + client + .admin() + .cluster() + .deletePipeline( + new DeletePipelineRequest(pipelineId), + ActionListener + .wrap( + deleteIngestPipelineResponse -> searchConfigInputIndices(id, indexingDryRun, listener), + exception -> listener.onFailure(exception) + ) + ); } protected void validateAgainstExistingHCConfig(String configId, boolean indexingDryRun, ActionListener listener) { From cf8bed4fabec7233f282a6a008c9b3615baf038f Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 17:16:03 -0800 Subject: [PATCH 15/21] utlizing a node state manager when writing results into flattened result index Signed-off-by: Jackie Han --- .../ADResultBulkTransportAction.java | 23 +++++++++++++++---- .../ForecastResultBulkTransportAction.java | 7 ++++-- .../transport/ResultBulkTransportAction.java | 6 ++++- .../ad/rest/AnomalyDetectorRestApiIT.java | 2 +- 4 files changed, 29 insertions(+), 9 deletions(-) diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index da90bf930..3922e7a0b 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -31,8 +31,12 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexingPressure; +import org.opensearch.timeseries.AnalysisType; +import org.opensearch.timeseries.NodeStateManager; +import org.opensearch.timeseries.model.Config; import org.opensearch.timeseries.transport.ResultBulkTransportAction; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportService; @@ -50,7 +54,8 @@ public ADResultBulkTransportAction( IndexingPressure indexingPressure, Settings settings, ClusterService clusterService, - Client client + Client client, + NodeStateManager stateManager ) { super( ADResultBulkAction.NAME, @@ -62,7 +67,8 @@ public ADResultBulkTransportAction( AD_INDEX_PRESSURE_SOFT_LIMIT.get(settings), AD_INDEX_PRESSURE_HARD_LIMIT.get(settings), ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, - ADResultBulkRequest::new + ADResultBulkRequest::new, + stateManager ); this.clusterService = clusterService; this.client = client; @@ -137,9 +143,16 @@ private boolean shouldAddResult(float indexingPressurePercent, AnomalyResult res private void addToFlattenedIndexIfExists(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { String flattenedResultIndexAlias = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); - if (clusterService.state().metadata().hasAlias(flattenedResultIndexAlias)) { - addResult(bulkRequest, result, flattenedResultIndexAlias); - } + String configId = result.getConfigId(); + nodeStateManager.getConfig(configId, AnalysisType.AD, ActionListener.wrap(configOptional -> { + if (configOptional.isEmpty()) { + return; + } + Config config = configOptional.get(); + if (config.getFlattenResultIndexMapping()) { + addResult(bulkRequest, result, flattenedResultIndexAlias); + } + }, e -> LOG.error("Fail to get config", e))); } private void addResult(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { diff --git a/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java b/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java index dcdd0680a..d4fce9c63 100644 --- a/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java @@ -26,6 +26,7 @@ import org.opensearch.forecast.model.ForecastResult; import org.opensearch.forecast.ratelimit.ForecastResultWriteRequest; import org.opensearch.index.IndexingPressure; +import org.opensearch.timeseries.NodeStateManager; import org.opensearch.timeseries.transport.ResultBulkTransportAction; import org.opensearch.transport.TransportService; @@ -39,7 +40,8 @@ public ForecastResultBulkTransportAction( IndexingPressure indexingPressure, Settings settings, ClusterService clusterService, - Client client + Client client, + NodeStateManager stateManager ) { super( ForecastResultBulkAction.NAME, @@ -51,7 +53,8 @@ public ForecastResultBulkTransportAction( FORECAST_INDEX_PRESSURE_SOFT_LIMIT.get(settings), FORECAST_INDEX_PRESSURE_HARD_LIMIT.get(settings), ForecastIndex.RESULT.getIndexName(), - ForecastResultBulkRequest::new + ForecastResultBulkRequest::new, + stateManager ); clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_INDEX_PRESSURE_SOFT_LIMIT, it -> softLimit = it); clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_INDEX_PRESSURE_HARD_LIMIT, it -> hardLimit = it); diff --git a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java index 61efd6104..9b1571337 100644 --- a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java @@ -34,6 +34,7 @@ import org.opensearch.index.IndexingPressure; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; +import org.opensearch.timeseries.NodeStateManager; import org.opensearch.timeseries.model.IndexableResult; import org.opensearch.timeseries.ratelimit.ResultWriteRequest; import org.opensearch.timeseries.util.BulkUtil; @@ -51,6 +52,7 @@ public abstract class ResultBulkTransportAction requestReader + Writeable.Reader requestReader, + NodeStateManager nodeStateManager ) { super(actionName, transportService, actionFilters, requestReader, ThreadPool.Names.SAME); this.indexingPressure = indexingPressure; @@ -72,6 +75,7 @@ public ResultBulkTransportAction( this.softLimit = softLimit; this.hardLimit = hardLimit; this.indexName = indexName; + this.nodeStateManager = nodeStateManager; // random seed is 42. Can be any number this.random = new Random(42); diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 85c67788b..48ecb6317 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -241,7 +241,7 @@ public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exceptio .format(Locale.ROOT, "/opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); // wait for the detector starts writing result try { - Thread.sleep(60 * 1000); + Thread.sleep(30 * 1000); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException("Thread was interrupted while waiting", e); From d557c500d69a4cd4b23e5edc76d6a74d705103fa Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 22:18:16 -0800 Subject: [PATCH 16/21] build flatten resuilt index enabled into the ResultWriteRequest Signed-off-by: Jackie Han --- .../ad/ratelimit/ADResultWriteRequest.java | 5 +++-- .../ad/ratelimit/ADResultWriteWorker.java | 5 +++-- .../ad/ratelimit/ADSaveResultStrategy.java | 3 ++- .../ADResultBulkTransportAction.java | 22 ++++--------------- .../ADSingleStreamResultTransportAction.java | 3 ++- .../ratelimit/ForecastResultWriteRequest.java | 5 +++-- .../ratelimit/ForecastResultWriteWorker.java | 5 +++-- .../ratelimit/ForecastSaveResultStrategy.java | 3 ++- ...castSingleStreamResultTransportAction.java | 3 ++- .../ratelimit/ResultWriteRequest.java | 18 +++++++++++++-- .../ratelimit/ResultWriteWorker.java | 6 +++-- 11 files changed, 44 insertions(+), 34 deletions(-) diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java index 912396ebd..c02242380 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java @@ -25,9 +25,10 @@ public ADResultWriteRequest( String detectorId, RequestPriority priority, AnomalyResult result, - String resultIndex + String resultIndex, + boolean flattenResultIndex ) { - super(expirationEpochMs, detectorId, priority, result, resultIndex); + super(expirationEpochMs, detectorId, priority, result, resultIndex, flattenResultIndex); } public ADResultWriteRequest(StreamInput in) throws IOException { diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java index b57e99f1c..608be3e4c 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java @@ -103,8 +103,9 @@ protected ADResultWriteRequest createResultWriteRequest( String configId, RequestPriority priority, AnomalyResult result, - String resultIndex + String resultIndex, + boolean flattenResultIndex ) { - return new ADResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex); + return new ADResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex, flattenResultIndex); } } diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java b/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java index cac437523..fdecbf236 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java @@ -86,7 +86,8 @@ public void saveResult(AnomalyResult result, Config config) { config.getId(), result.getAnomalyGrade() > 0 ? RequestPriority.HIGH : RequestPriority.MEDIUM, result, - config.getCustomResultIndexOrAlias() + config.getCustomResultIndexOrAlias(), + config.getFlattenResultIndexMapping() ) ); } diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index 3922e7a0b..ec35344b2 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -31,12 +31,9 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; -import org.opensearch.core.action.ActionListener; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexingPressure; -import org.opensearch.timeseries.AnalysisType; import org.opensearch.timeseries.NodeStateManager; -import org.opensearch.timeseries.model.Config; import org.opensearch.timeseries.transport.ResultBulkTransportAction; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportService; @@ -120,7 +117,10 @@ protected BulkRequest prepareBulkRequest(float indexingPressurePercent, ADResult if (shouldAddResult(indexingPressurePercent, result)) { addResult(bulkRequest, result, resultIndex); - addToFlattenedIndexIfExists(bulkRequest, result, resultIndex); + if (resultWriteRequest.getFlattenResultIndex()) { + String flattenedResultIndexAlias = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); + addResult(bulkRequest, result, flattenedResultIndexAlias); + } } } @@ -141,20 +141,6 @@ private boolean shouldAddResult(float indexingPressurePercent, AnomalyResult res } } - private void addToFlattenedIndexIfExists(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { - String flattenedResultIndexAlias = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); - String configId = result.getConfigId(); - nodeStateManager.getConfig(configId, AnalysisType.AD, ActionListener.wrap(configOptional -> { - if (configOptional.isEmpty()) { - return; - } - Config config = configOptional.get(); - if (config.getFlattenResultIndexMapping()) { - addResult(bulkRequest, result, flattenedResultIndexAlias); - } - }, e -> LOG.error("Fail to get config", e))); - } - private void addResult(BulkRequest bulkRequest, AnomalyResult result, String resultIndex) { String index = resultIndex == null ? indexName : resultIndex; try (XContentBuilder builder = jsonBuilder()) { diff --git a/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java index 83283468e..67d323418 100644 --- a/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java @@ -72,7 +72,8 @@ public ADResultWriteRequest createResultWriteRequest(Config config, AnomalyResul config.getId(), RequestPriority.MEDIUM, result, - config.getCustomResultIndexOrAlias() + config.getCustomResultIndexOrAlias(), + config.getFlattenResultIndexMapping() ); } diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java index 54c33f5bb..9dedb08a6 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java @@ -25,9 +25,10 @@ public ForecastResultWriteRequest( String forecasterId, RequestPriority priority, ForecastResult result, - String resultIndex + String resultIndex, + boolean flattenResultIndex ) { - super(expirationEpochMs, forecasterId, priority, result, resultIndex); + super(expirationEpochMs, forecasterId, priority, result, resultIndex, flattenResultIndex); } public ForecastResultWriteRequest(StreamInput in) throws IOException { diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java index 7f991bcf6..87e23b5a3 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java @@ -103,8 +103,9 @@ protected ForecastResultWriteRequest createResultWriteRequest( String configId, RequestPriority priority, ForecastResult result, - String resultIndex + String resultIndex, + boolean flattenResultIndex ) { - return new ForecastResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex); + return new ForecastResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex, flattenResultIndex); } } diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java index 10f256278..a35f52b84 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java @@ -83,7 +83,8 @@ public void saveResult(ForecastResult result, Config config) { config.getId(), RequestPriority.MEDIUM, result, - config.getCustomResultIndexOrAlias() + config.getCustomResultIndexOrAlias(), + config.getFlattenResultIndexMapping() ) ); } diff --git a/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java b/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java index 6b3a09835..e09fe36b6 100644 --- a/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java +++ b/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java @@ -76,7 +76,8 @@ public ForecastResultWriteRequest createResultWriteRequest(Config config, Foreca config.getId(), RequestPriority.MEDIUM, result, - config.getCustomResultIndexOrAlias() + config.getCustomResultIndexOrAlias(), + config.getFlattenResultIndexMapping() ); } } diff --git a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java index 6d5a069f1..084487062 100644 --- a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java +++ b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java @@ -22,16 +22,26 @@ public abstract class ResultWriteRequest ext private final ResultType result; // If resultIndex is null, result will be stored in default result index. private final String resultIndex; + private final boolean flattenResultIndex; - public ResultWriteRequest(long expirationEpochMs, String configId, RequestPriority priority, ResultType result, String resultIndex) { + public ResultWriteRequest( + long expirationEpochMs, + String configId, + RequestPriority priority, + ResultType result, + String resultIndex, + boolean flattenResultIndex + ) { super(expirationEpochMs, configId, priority); this.result = result; this.resultIndex = resultIndex; + this.flattenResultIndex = flattenResultIndex; } - public ResultWriteRequest(StreamInput in, Writeable.Reader resultReader) throws IOException { + public ResultWriteRequest(StreamInput in, Reader resultReader) throws IOException { this.result = resultReader.read(in); this.resultIndex = in.readOptionalString(); + this.flattenResultIndex = in.readBoolean(); } @Override @@ -47,4 +57,8 @@ public ResultType getResult() { public String getResultIndex() { return resultIndex; } + + public boolean getFlattenResultIndex() { + return flattenResultIndex; + } } diff --git a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java index 772062fef..2c7cdcc59 100644 --- a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java +++ b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java @@ -199,7 +199,8 @@ private ActionListener> onGetConfig( id, resultToRetry.isHighPriority() ? RequestPriority.HIGH : RequestPriority.MEDIUM, resultToRetry, - config.getCustomResultIndexOrAlias() + config.getCustomResultIndexOrAlias(), + config.getFlattenResultIndexMapping() ) ); @@ -216,6 +217,7 @@ protected abstract ResultWriteRequestType createResultWriteRequest( String configId, RequestPriority priority, ResultType result, - String resultIndex + String resultIndex, + boolean flattenResultIndex ); } From b5240df52065b3fe461b40f0a87dbbe5423235ff Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 22:54:11 -0800 Subject: [PATCH 17/21] unbind ingest pipeline with flattened result index when it's disabled Signed-off-by: Jackie Han --- .../AbstractTimeSeriesActionHandler.java | 53 +++++++------------ 1 file changed, 19 insertions(+), 34 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index ed264c7f5..e0e9a6ee5 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -642,7 +642,7 @@ private void onGetConfigResponse(GetResponse response, boolean indexingDryRun, S && existingConfig.getCustomResultIndexOrAlias() != null) { confirmBatchRunningListener = ActionListener .wrap( - r -> getFlattenResultAliasIndex(existingConfig, listener, id, indexingDryRun), + r -> unbindIngestPipelineWithFlattenedResultIndex(existingConfig, listener, id, indexingDryRun), // can't update config if there is task running listener::onFailure ); @@ -663,41 +663,26 @@ private void onGetConfigResponse(GetResponse response, boolean indexingDryRun, S } } - private void getFlattenResultAliasIndex(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { - String flattenResultIndexAlias = timeSeriesIndices - .getFlattenedResultIndexAlias(existingConfig.getCustomResultIndexOrAlias(), existingConfig.getId()); - GetAliasesRequest getAliasesRequest = new GetAliasesRequest(flattenResultIndexAlias); - client.admin().indices().getAliases(getAliasesRequest, ActionListener.wrap(getAliasesResponse -> { - Set indices = getAliasesResponse.getAliases().keySet(); - if (indices.isEmpty()) { - return; - } - String indexName = indices.iterator().next(); - deleteAlias(indexName, flattenResultIndexAlias, existingConfig, listener, id, indexingDryRun); - }, exception -> listener.onFailure(exception))); + private UpdateSettingsRequest buildUpdateSettingsRequest(String defaultPipelineName, String configId) { + String flattenedResultIndex = timeSeriesIndices + .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); + updateSettingsRequest.indices(flattenedResultIndex); + + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put("index.default_pipeline", defaultPipelineName); + + updateSettingsRequest.settings(settingsBuilder); + + return updateSettingsRequest; } - private void deleteAlias( - String indexName, - String aliasName, - Config existingConfig, - ActionListener listener, - String id, - boolean indexingDryRun - ) { - IndicesAliasesRequest indicesAliasesRequest = new IndicesAliasesRequest(); - indicesAliasesRequest.addAliasAction(IndicesAliasesRequest.AliasActions.remove().index(indexName).alias(aliasName)); - client - .admin() - .indices() - .aliases( - indicesAliasesRequest, - ActionListener - .wrap( - deleteAliasResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), - exception -> listener.onFailure(exception) - ) - ); + private void unbindIngestPipelineWithFlattenedResultIndex(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { + UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest("_none", existingConfig.getId()); + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( + updateSettingsResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), + exception -> listener.onFailure(exception) + )); } private void deleteIngestPipeline(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { From ffd7d9493427c6b97b1120c754523ae9086b28f0 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Tue, 28 Jan 2025 23:23:34 -0800 Subject: [PATCH 18/21] test --- .../timeseries/indices/IndexManagement.java | 10 +-- .../AbstractTimeSeriesActionHandler.java | 68 ++++++++----------- 2 files changed, 32 insertions(+), 46 deletions(-) diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 398123928..84e8b5cb3 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1037,14 +1037,8 @@ public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionLis choosePrimaryShards(request, false); adminClient.indices().create(request, ActionListener.wrap(response -> { - if (response.isAcknowledged()) { - logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); - actionListener.onResponse(response); - } else { - String errorMsg = "Index creation not acknowledged for index: " + indexName; - logger.error(errorMsg); - actionListener.onFailure(new IllegalStateException(errorMsg)); - } + logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); + actionListener.onResponse(response); }, exception -> { logger.error("Failed to create flattened result index: {}", indexName, exception); actionListener.onFailure(exception); diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index e0e9a6ee5..2702a3745 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -472,19 +472,25 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene String configId = RestHandlerUtils.getConfigIdFromIndexResponse(createConfigResponse); String flattenedResultIndexAlias = timeSeriesIndices .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); - String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); timeSeriesIndices - .initFlattenedResultIndex( - flattenedResultIndexAlias, - ActionListener.wrap(initResponse -> setupIngestPipeline(configId, ActionListener.wrap(pipelineResponse -> { - updateResultIndexSetting( - pipelineId, + .initFlattenedResultIndex( flattenedResultIndexAlias, - ActionListener.wrap(updateResponse -> listener.onResponse(createConfigResponse), listener::onFailure) - ); - }, listener::onFailure)), listener::onFailure) - ); + ActionListener.wrap( + initResponse -> setupIngestPipeline(configId, listener, createConfigResponse), + listener::onFailure)); + +// timeSeriesIndices +// .initFlattenedResultIndex( +// flattenedResultIndexAlias, +// ActionListener.wrap(initResponse -> setupIngestPipeline(configId, ActionListener.wrap(pipelineResponse -> { +// updateResultIndexSetting( +// pipelineId, +// flattenedResultIndexAlias, +// ActionListener.wrap(updateResponse -> listener.onResponse(createConfigResponse), listener::onFailure) +// ); +// }, listener::onFailure)), listener::onFailure) +// ); } else { listener.onResponse(createConfigResponse); } @@ -497,7 +503,7 @@ private boolean shouldHandleFlattening(boolean indexingDryRun) { return !indexingDryRun && config.getCustomResultIndexOrAlias() != null && Boolean.TRUE.equals(flattenResultIndexMapping); } - protected void setupIngestPipeline(String configId, ActionListener listener) { + protected void setupIngestPipeline(String configId, ActionListener listener, T createConfigResponse) { String flattenedResultIndexAlias = timeSeriesIndices.getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); @@ -506,15 +512,11 @@ protected void setupIngestPipeline(String configId, ActionListener listener) PutPipelineRequest putPipelineRequest = new PutPipelineRequest(pipelineId, pipelineSource, XContentType.JSON); - client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap(response -> { - if (response.isAcknowledged()) { - logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); - listener.onResponse(null); - } else { - String errorMessage = "Ingest pipeline creation was not acknowledged for pipelineId: " + pipelineId; - logger.error(errorMessage); - listener.onFailure(new OpenSearchStatusException(errorMessage, RestStatus.INTERNAL_SERVER_ERROR)); - } + client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap( + putPipelineResponse -> { + logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); + bindIngestPipelineWithFlattenedResultIndex(pipelineId, configId, flattenedResultIndexAlias, listener, createConfigResponse); + }, exception -> { logger.error("Error while creating ingest pipeline for pipelineId: {}", pipelineId, exception); listener.onFailure(exception); @@ -551,26 +553,15 @@ private BytesReference createPipelineDefinition(String indexName) throws IOExcep return BytesReference.bytes(pipelineBuilder); } - protected void updateResultIndexSetting(String pipelineId, String flattenedResultIndex, ActionListener listener) { - UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); - updateSettingsRequest.indices(flattenedResultIndex); + protected void bindIngestPipelineWithFlattenedResultIndex(String pipelineId, String configId, String flattenedResultIndexAlias, ActionListener listener, T createConfigResponse) { + UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest(pipelineId, configId); - Settings.Builder settingsBuilder = Settings.builder(); - settingsBuilder.put("index.default_pipeline", pipelineId); - - updateSettingsRequest.settings(settingsBuilder); - - client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> { - if (response.isAcknowledged()) { - logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId); - listener.onResponse(null); - } else { - String errorMsg = "Settings update not acknowledged for index: " + flattenedResultIndex; - logger.error(errorMsg); - listener.onFailure(new OpenSearchStatusException(errorMsg, RestStatus.INTERNAL_SERVER_ERROR)); - } + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( + updateSettingsResponse -> { + logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId); + listener.onResponse(createConfigResponse); }, exception -> { - logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndex, pipelineId, exception); + logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId, exception); listener.onFailure(exception); })); } @@ -678,6 +669,7 @@ private UpdateSettingsRequest buildUpdateSettingsRequest(String defaultPipelineN } private void unbindIngestPipelineWithFlattenedResultIndex(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { + // The pipeline name _none specifies that the index does not have an ingest pipeline. UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest("_none", existingConfig.getId()); client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( updateSettingsResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), From 05f56a3b040465c0b5308bad922b0c6948efcb64 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Wed, 29 Jan 2025 01:49:56 -0800 Subject: [PATCH 19/21] address comments --- .../ADResultBulkTransportAction.java | 7 +- .../ForecastResultBulkTransportAction.java | 7 +- .../ratelimit/ResultWriteRequest.java | 1 + .../AbstractTimeSeriesActionHandler.java | 95 +++++++------- .../transport/ResultBulkTransportAction.java | 4 +- .../ad/ratelimit/ResultWriteWorkerTests.java | 14 ++- .../ad/rest/AnomalyDetectorRestApiIT.java | 116 ++++++++++++++---- .../MultiEntityResultHandlerTests.java | 3 +- .../opensearch/timeseries/TestHelpers.java | 5 +- 9 files changed, 161 insertions(+), 91 deletions(-) diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index ec35344b2..e27362097 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -33,7 +33,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexingPressure; -import org.opensearch.timeseries.NodeStateManager; import org.opensearch.timeseries.transport.ResultBulkTransportAction; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportService; @@ -51,8 +50,7 @@ public ADResultBulkTransportAction( IndexingPressure indexingPressure, Settings settings, ClusterService clusterService, - Client client, - NodeStateManager stateManager + Client client ) { super( ADResultBulkAction.NAME, @@ -64,8 +62,7 @@ public ADResultBulkTransportAction( AD_INDEX_PRESSURE_SOFT_LIMIT.get(settings), AD_INDEX_PRESSURE_HARD_LIMIT.get(settings), ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, - ADResultBulkRequest::new, - stateManager + ADResultBulkRequest::new ); this.clusterService = clusterService; this.client = client; diff --git a/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java b/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java index d4fce9c63..dcdd0680a 100644 --- a/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/forecast/transport/ForecastResultBulkTransportAction.java @@ -26,7 +26,6 @@ import org.opensearch.forecast.model.ForecastResult; import org.opensearch.forecast.ratelimit.ForecastResultWriteRequest; import org.opensearch.index.IndexingPressure; -import org.opensearch.timeseries.NodeStateManager; import org.opensearch.timeseries.transport.ResultBulkTransportAction; import org.opensearch.transport.TransportService; @@ -40,8 +39,7 @@ public ForecastResultBulkTransportAction( IndexingPressure indexingPressure, Settings settings, ClusterService clusterService, - Client client, - NodeStateManager stateManager + Client client ) { super( ForecastResultBulkAction.NAME, @@ -53,8 +51,7 @@ public ForecastResultBulkTransportAction( FORECAST_INDEX_PRESSURE_SOFT_LIMIT.get(settings), FORECAST_INDEX_PRESSURE_HARD_LIMIT.get(settings), ForecastIndex.RESULT.getIndexName(), - ForecastResultBulkRequest::new, - stateManager + ForecastResultBulkRequest::new ); clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_INDEX_PRESSURE_SOFT_LIMIT, it -> softLimit = it); clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_INDEX_PRESSURE_HARD_LIMIT, it -> hardLimit = it); diff --git a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java index 084487062..c56484920 100644 --- a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java +++ b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java @@ -48,6 +48,7 @@ public ResultWriteRequest(StreamInput in, Reader resultReader) throw public void writeTo(StreamOutput out) throws IOException { result.writeTo(out); out.writeOptionalString(resultIndex); + out.writeBoolean(flattenResultIndex); } public ResultType getResult() { diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index 2702a3745..a03491024 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -22,8 +22,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.OpenSearchStatusException; -import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest; -import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsAction; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsRequest; @@ -474,23 +472,11 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); timeSeriesIndices - .initFlattenedResultIndex( - flattenedResultIndexAlias, - ActionListener.wrap( - initResponse -> setupIngestPipeline(configId, listener, createConfigResponse), - listener::onFailure)); - -// timeSeriesIndices -// .initFlattenedResultIndex( -// flattenedResultIndexAlias, -// ActionListener.wrap(initResponse -> setupIngestPipeline(configId, ActionListener.wrap(pipelineResponse -> { -// updateResultIndexSetting( -// pipelineId, -// flattenedResultIndexAlias, -// ActionListener.wrap(updateResponse -> listener.onResponse(createConfigResponse), listener::onFailure) -// ); -// }, listener::onFailure)), listener::onFailure) -// ); + .initFlattenedResultIndex( + flattenedResultIndexAlias, + ActionListener + .wrap(initResponse -> setupIngestPipeline(configId, listener, createConfigResponse), listener::onFailure) + ); } else { listener.onResponse(createConfigResponse); } @@ -512,10 +498,9 @@ protected void setupIngestPipeline(String configId, ActionListener listener, PutPipelineRequest putPipelineRequest = new PutPipelineRequest(pipelineId, pipelineSource, XContentType.JSON); - client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap( - putPipelineResponse -> { - logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); - bindIngestPipelineWithFlattenedResultIndex(pipelineId, configId, flattenedResultIndexAlias, listener, createConfigResponse); + client.admin().cluster().putPipeline(putPipelineRequest, ActionListener.wrap(putPipelineResponse -> { + logger.info("Ingest pipeline created successfully for pipelineId: {}", pipelineId); + bindIngestPipelineWithFlattenedResultIndex(pipelineId, configId, flattenedResultIndexAlias, listener, createConfigResponse); }, exception -> { logger.error("Error while creating ingest pipeline for pipelineId: {}", pipelineId, exception); @@ -553,13 +538,31 @@ private BytesReference createPipelineDefinition(String indexName) throws IOExcep return BytesReference.bytes(pipelineBuilder); } - protected void bindIngestPipelineWithFlattenedResultIndex(String pipelineId, String configId, String flattenedResultIndexAlias, ActionListener listener, T createConfigResponse) { + private UpdateSettingsRequest buildUpdateSettingsRequest(String defaultPipelineName, String configId) { + String flattenedResultIndex = timeSeriesIndices.getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); + updateSettingsRequest.indices(flattenedResultIndex); + + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put("index.default_pipeline", defaultPipelineName); + + updateSettingsRequest.settings(settingsBuilder); + + return updateSettingsRequest; + } + + protected void bindIngestPipelineWithFlattenedResultIndex( + String pipelineId, + String configId, + String flattenedResultIndexAlias, + ActionListener listener, + T createConfigResponse + ) { UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest(pipelineId, configId); - client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( - updateSettingsResponse -> { - logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId); - listener.onResponse(createConfigResponse); + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(updateSettingsResponse -> { + logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId); + listener.onResponse(createConfigResponse); }, exception -> { logger.error("Failed to update settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId, exception); listener.onFailure(exception); @@ -654,27 +657,25 @@ private void onGetConfigResponse(GetResponse response, boolean indexingDryRun, S } } - private UpdateSettingsRequest buildUpdateSettingsRequest(String defaultPipelineName, String configId) { - String flattenedResultIndex = timeSeriesIndices - .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); - UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); - updateSettingsRequest.indices(flattenedResultIndex); - - Settings.Builder settingsBuilder = Settings.builder(); - settingsBuilder.put("index.default_pipeline", defaultPipelineName); - - updateSettingsRequest.settings(settingsBuilder); - - return updateSettingsRequest; - } - - private void unbindIngestPipelineWithFlattenedResultIndex(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { + private void unbindIngestPipelineWithFlattenedResultIndex( + Config existingConfig, + ActionListener listener, + String id, + boolean indexingDryRun + ) { // The pipeline name _none specifies that the index does not have an ingest pipeline. UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest("_none", existingConfig.getId()); - client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap( - updateSettingsResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), - exception -> listener.onFailure(exception) - )); + client + .admin() + .indices() + .updateSettings( + updateSettingsRequest, + ActionListener + .wrap( + updateSettingsResponse -> deleteIngestPipeline(existingConfig, listener, id, indexingDryRun), + exception -> listener.onFailure(exception) + ) + ); } private void deleteIngestPipeline(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { diff --git a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java index 9b1571337..6177eaae9 100644 --- a/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/timeseries/transport/ResultBulkTransportAction.java @@ -64,8 +64,7 @@ public ResultBulkTransportAction( float softLimit, float hardLimit, String indexName, - Writeable.Reader requestReader, - NodeStateManager nodeStateManager + Writeable.Reader requestReader ) { super(actionName, transportService, actionFilters, requestReader, ThreadPool.Names.SAME); this.indexingPressure = indexingPressure; @@ -75,7 +74,6 @@ public ResultBulkTransportAction( this.softLimit = softLimit; this.hardLimit = hardLimit; this.indexName = indexName; - this.nodeStateManager = nodeStateManager; // random seed is 42. Can be any number this.random = new Random(42); diff --git a/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java b/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java index 2d829722b..843c40bed 100644 --- a/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java +++ b/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java @@ -120,7 +120,8 @@ public void testRegular() { detectorId, RequestPriority.MEDIUM, detectResult, - null + null, + false ); request.add(resultWriteRequest); @@ -130,7 +131,7 @@ public void testRegular() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); // the request results one flush verify(resultHandler, times(1)).flush(any(), any()); @@ -152,7 +153,8 @@ public void testSingleRetryRequest() throws IOException { detectorId, RequestPriority.MEDIUM, detectResult, - null + null, + false ); request.add(resultWriteRequest); @@ -168,7 +170,7 @@ public void testSingleRetryRequest() throws IOException { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); // one flush from the original request; and one due to retry verify(resultHandler, times(2)).flush(any(), any()); @@ -188,7 +190,7 @@ public void testRetryException() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); // one flush from the original request; and one due to retry verify(resultHandler, times(2)).flush(any(), any()); verify(nodeStateManager, times(1)).setException(eq(detectorId), any(OpenSearchStatusException.class)); @@ -202,7 +204,7 @@ public void testOverloaded() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); // one flush from the original request; and one due to retry verify(resultHandler, times(1)).flush(any(), any()); verify(nodeStateManager, times(1)).setException(eq(detectorId), any(OpenSearchRejectedExecutionException.class)); diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 48ecb6317..951f236ae 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -202,7 +202,7 @@ public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exceptio true ); - // test behavior when AD is disabled + // Test behavior when AD is disabled updateClusterSettings(ADEnabledSetting.AD_ENABLED, false); Exception ex = expectThrows( ResponseException.class, @@ -218,44 +218,116 @@ public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exceptio ); assertThat(ex.getMessage(), containsString(ADCommonMessages.DISABLED_ERR_MSG)); - // test behavior when AD is enabled + // Test behavior when AD is enabled updateClusterSettings(ADEnabledSetting.AD_ENABLED, true); Response response = TestHelpers .makeRequest(client(), "POST", TestHelpers.AD_BASE_DETECTORS_URI, ImmutableMap.of(), TestHelpers.toHttpEntity(detector), null); assertEquals("Create anomaly detector with flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); + Map responseMap = entityAsMap(response); String id = (String) responseMap.get("_id"); int version = (int) responseMap.get("_version"); - assertNotEquals("response is missing Id", AnomalyDetector.NO_ID, id); - assertTrue("incorrect version", version > 0); - // ensure the flattened result index was created + assertNotEquals("Response is missing Id", AnomalyDetector.NO_ID, id); + assertTrue("Incorrect version", version > 0); + + // Ensure the flattened result index was created String expectedFlattenedIndex = String .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); - boolean aliasExists = aliasExists(expectedFlattenedIndex); - assertTrue(aliasExists); - // ensure that the flattened field "feature_data_feature_bytes" exists in the mappings + assertTrue("Alias for flattened result index does not exist", aliasExists(expectedFlattenedIndex)); + + // Start detector String startDetectorEndpoint = String.format(Locale.ROOT, TestHelpers.AD_BASE_START_DETECTOR_URL, id); - Response startDetectorResponse = TestHelpers - .makeRequest(client(), "POST", startDetectorEndpoint, ImmutableMap.of(), (HttpEntity) null, null); - String getFlattenedResultIndexEndpoint = String - .format(Locale.ROOT, "/opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); - // wait for the detector starts writing result - try { - Thread.sleep(30 * 1000); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Thread was interrupted while waiting", e); + TestHelpers.makeRequest(client(), "POST", startDetectorEndpoint, ImmutableMap.of(), (HttpEntity) null, null); + + // Wait for detector results, check every 1 second, max 60 seconds + boolean resultsAvailable = false; + int maxRetries = 60; + int retryIntervalMs = 1000; + + Map searchResults = null; + for (int attempt = 0; attempt < maxRetries; attempt++) { + try { + Response searchAllResponse = TestHelpers + .makeRequest( + client(), + "POST", + TestHelpers.AD_BASE_RESULT_URI + "/_search/" + expectedFlattenedIndex, + ImmutableMap.of(), + new StringEntity("{\"query\":{\"match_all\":{}}}", ContentType.APPLICATION_JSON), + null + ); + searchResults = entityAsMap(searchAllResponse); + List> hitsList = (List>) ((Map) searchResults.get("hits")) + .get("hits"); + + if (hitsList != null && !hitsList.isEmpty()) { + resultsAvailable = true; + break; + } + } catch (Exception e) { + // Log the retry attempt and continue retrying + System.out.println("Attempt " + (attempt + 1) + " failed, retrying..."); + } + Thread.sleep(retryIntervalMs); } - Response getIndexResponse = TestHelpers.makeRequest(client(), "GET", getFlattenedResultIndexEndpoint, ImmutableMap.of(), "", null); + + assertTrue("No anomaly detection results found within timeout period", resultsAvailable); + + // Extract feature name and value from search results + Map firstHit = ((List>) ((Map) searchResults.get("hits")).get("hits")).get(0); + Map source = (Map) firstHit.get("_source"); + assertNotNull("Source should not be null", source); + assertTrue("Source should contain 'feature_data'", source.containsKey("feature_data")); + + List> featureDataList = (List>) source.get("feature_data"); + assertFalse("Feature data list should not be empty", featureDataList.isEmpty()); + + Map firstFeature = featureDataList.get(0); + String featureName = (String) firstFeature.get("feature_name"); + Double featureValue = ((Number) firstFeature.get("data")).doubleValue(); + + // Validate flattened result index mappings + Response getIndexResponse = TestHelpers.makeRequest(client(), "GET", expectedFlattenedIndex, ImmutableMap.of(), "", null); Map flattenedResultIndex = entityAsMap(getIndexResponse); String indexKey = flattenedResultIndex.keySet().stream().findFirst().orElse(null); Map indexDetails = (Map) flattenedResultIndex.get(indexKey); Map mappings = (Map) indexDetails.get("mappings"); - Object dynamicValue = mappings.get("dynamic"); - assertEquals("Dynamic field is not set to true", "true", dynamicValue.toString()); + + assertEquals("Dynamic field is not set to true", "true", mappings.get("dynamic").toString()); + Map properties = (Map) mappings.get("properties"); - assertTrue("Flattened field 'feature_data_feature_bytes' does not exist", properties.containsKey("feature_data_feature_bytes")); + String expectedFieldKey = "feature_data_" + featureName; + assertTrue("Flattened field '" + expectedFieldKey + "' does not exist", properties.containsKey(expectedFieldKey)); + + // Search against flattened result index and validate value + Response searchFlattenResultIndexResponse = TestHelpers + .makeRequest( + client(), + "POST", + TestHelpers.AD_BASE_RESULT_URI + "/_search/" + expectedFlattenedIndex, + ImmutableMap.of(), + new StringEntity("{\"query\":{\"match_all\":{}}}", ContentType.APPLICATION_JSON), + null + ); + Map flattenedResultIndexSearchResults = entityAsMap(searchFlattenResultIndexResponse); + Map flattenedResultIndexHitsMap = (Map) flattenedResultIndexSearchResults.get("hits"); + List> flattenedResultIndexHitsList = (List>) flattenedResultIndexHitsMap.get("hits"); + + Map flattenedResultIndexFirstHit = flattenedResultIndexHitsList.get(0); + Map flattenedResultIndexSource = (Map) flattenedResultIndexFirstHit.get("_source"); + + assertTrue( + "Flattened result index does not contain '" + expectedFieldKey + "'", + flattenedResultIndexSource.containsKey(expectedFieldKey) + ); + + assertEquals( + "Flattened field value is not correct", + featureValue, + ((Number) flattenedResultIndexSource.get(expectedFieldKey)).doubleValue(), + 0.0001 + ); } public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipeline() throws Exception { diff --git a/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java b/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java index 02b5b31ac..a19442112 100644 --- a/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java +++ b/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java @@ -50,7 +50,8 @@ public void setUp() throws Exception { detectorId, RequestPriority.MEDIUM, TestHelpers.randomAnomalyDetectResult(), - null + null, + false ); request.add(resultWriteRequest); diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index b0df971f2..1dd709e12 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -1053,7 +1053,8 @@ public static ADResultWriteRequest randomADResultWriteRequest(String detectorId, detectorId, RequestPriority.MEDIUM, randomHCADAnomalyDetectResult(score, grade), - null + null, + false ); return resultWriteRequest; } @@ -2279,7 +2280,7 @@ public static ForecastResultWriteRequest randomForecastResultWriteRequest() { ForecastResult result = randomForecastResult(forecasterId); String resultIndex = random.nextBoolean() ? randomAlphaOfLength(10) : null; // Randomly decide to set resultIndex or not - return new ForecastResultWriteRequest(expirationEpochMs, forecasterId, priority, result, resultIndex); + return new ForecastResultWriteRequest(expirationEpochMs, forecasterId, priority, result, resultIndex, false); } public static ForecastResult randomForecastResult(String forecasterId) { From 64cc3bcca6e126a2a49479344906abc586005c90 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Wed, 29 Jan 2025 01:55:48 -0800 Subject: [PATCH 20/21] cleanup Signed-off-by: Jackie Han --- .../ad/rest/AnomalyDetectorRestApiIT.java | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 951f236ae..6cc8dc2f8 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -246,27 +246,21 @@ public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exceptio Map searchResults = null; for (int attempt = 0; attempt < maxRetries; attempt++) { - try { - Response searchAllResponse = TestHelpers - .makeRequest( - client(), - "POST", - TestHelpers.AD_BASE_RESULT_URI + "/_search/" + expectedFlattenedIndex, - ImmutableMap.of(), - new StringEntity("{\"query\":{\"match_all\":{}}}", ContentType.APPLICATION_JSON), - null - ); - searchResults = entityAsMap(searchAllResponse); - List> hitsList = (List>) ((Map) searchResults.get("hits")) - .get("hits"); - - if (hitsList != null && !hitsList.isEmpty()) { - resultsAvailable = true; - break; - } - } catch (Exception e) { - // Log the retry attempt and continue retrying - System.out.println("Attempt " + (attempt + 1) + " failed, retrying..."); + Response searchAllResponse = TestHelpers + .makeRequest( + client(), + "POST", + TestHelpers.AD_BASE_RESULT_URI + "/_search/" + expectedFlattenedIndex, + ImmutableMap.of(), + new StringEntity("{\"query\":{\"match_all\":{}}}", ContentType.APPLICATION_JSON), + null + ); + searchResults = entityAsMap(searchAllResponse); + List> hitsList = (List>) ((Map) searchResults.get("hits")).get("hits"); + + if (hitsList != null && !hitsList.isEmpty()) { + resultsAvailable = true; + break; } Thread.sleep(retryIntervalMs); } From 8a9429db8a50025e46ce2db9af8c6ac129959f07 Mon Sep 17 00:00:00 2001 From: Jackie Han Date: Wed, 29 Jan 2025 10:35:24 -0800 Subject: [PATCH 21/21] make flatten result index use detector name Signed-off-by: Jackie Han --- .../ad/ratelimit/ADResultWriteRequest.java | 2 +- .../ad/ratelimit/ADResultWriteWorker.java | 2 +- .../ad/ratelimit/ADSaveResultStrategy.java | 2 +- .../ADResultBulkTransportAction.java | 6 +-- .../ADSingleStreamResultTransportAction.java | 2 +- .../ratelimit/ForecastResultWriteRequest.java | 2 +- .../ratelimit/ForecastResultWriteWorker.java | 2 +- .../ratelimit/ForecastSaveResultStrategy.java | 2 +- ...castSingleStreamResultTransportAction.java | 2 +- .../timeseries/indices/IndexManagement.java | 20 +++++----- .../opensearch/timeseries/model/Config.java | 14 +++++++ .../ratelimit/ResultWriteRequest.java | 10 ++--- .../ratelimit/ResultWriteWorker.java | 4 +- .../AbstractTimeSeriesActionHandler.java | 37 +++++++++++++------ .../ad/ratelimit/ResultWriteWorkerTests.java | 12 +++--- .../ad/rest/AnomalyDetectorRestApiIT.java | 8 ++-- .../MultiEntityResultHandlerTests.java | 2 +- .../opensearch/timeseries/TestHelpers.java | 6 +-- 18 files changed, 78 insertions(+), 57 deletions(-) diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java index c02242380..5f26f9244 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteRequest.java @@ -26,7 +26,7 @@ public ADResultWriteRequest( RequestPriority priority, AnomalyResult result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ) { super(expirationEpochMs, detectorId, priority, result, resultIndex, flattenResultIndex); } diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java index 608be3e4c..f9b4edc75 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADResultWriteWorker.java @@ -104,7 +104,7 @@ protected ADResultWriteRequest createResultWriteRequest( RequestPriority priority, AnomalyResult result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ) { return new ADResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex, flattenResultIndex); } diff --git a/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java b/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java index fdecbf236..c84518890 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java +++ b/src/main/java/org/opensearch/ad/ratelimit/ADSaveResultStrategy.java @@ -87,7 +87,7 @@ public void saveResult(AnomalyResult result, Config config) { result.getAnomalyGrade() > 0 ? RequestPriority.HIGH : RequestPriority.MEDIUM, result, config.getCustomResultIndexOrAlias(), - config.getFlattenResultIndexMapping() + config.getFlattenResultIndexAlias() ) ); } diff --git a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java index e27362097..ef451208b 100644 --- a/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADResultBulkTransportAction.java @@ -17,7 +17,6 @@ import java.io.IOException; import java.util.List; -import java.util.Locale; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -114,9 +113,8 @@ protected BulkRequest prepareBulkRequest(float indexingPressurePercent, ADResult if (shouldAddResult(indexingPressurePercent, result)) { addResult(bulkRequest, result, resultIndex); - if (resultWriteRequest.getFlattenResultIndex()) { - String flattenedResultIndexAlias = resultIndex + "_flattened_" + result.getDetectorId().toLowerCase(Locale.ROOT); - addResult(bulkRequest, result, flattenedResultIndexAlias); + if (resultWriteRequest.getFlattenResultIndex() != null) { + addResult(bulkRequest, result, resultWriteRequest.getFlattenResultIndex()); } } } diff --git a/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java b/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java index 67d323418..66915513b 100644 --- a/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ADSingleStreamResultTransportAction.java @@ -73,7 +73,7 @@ public ADResultWriteRequest createResultWriteRequest(Config config, AnomalyResul RequestPriority.MEDIUM, result, config.getCustomResultIndexOrAlias(), - config.getFlattenResultIndexMapping() + config.getFlattenResultIndexAlias() ); } diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java index 9dedb08a6..3929a5e5f 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteRequest.java @@ -26,7 +26,7 @@ public ForecastResultWriteRequest( RequestPriority priority, ForecastResult result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ) { super(expirationEpochMs, forecasterId, priority, result, resultIndex, flattenResultIndex); } diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java index 87e23b5a3..7fba8ab03 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastResultWriteWorker.java @@ -104,7 +104,7 @@ protected ForecastResultWriteRequest createResultWriteRequest( RequestPriority priority, ForecastResult result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ) { return new ForecastResultWriteRequest(expirationEpochMs, configId, priority, result, resultIndex, flattenResultIndex); } diff --git a/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java b/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java index a35f52b84..07fc26462 100644 --- a/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java +++ b/src/main/java/org/opensearch/forecast/ratelimit/ForecastSaveResultStrategy.java @@ -84,7 +84,7 @@ public void saveResult(ForecastResult result, Config config) { RequestPriority.MEDIUM, result, config.getCustomResultIndexOrAlias(), - config.getFlattenResultIndexMapping() + config.getFlattenResultIndexAlias() ) ); } diff --git a/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java b/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java index e09fe36b6..2b4d15899 100644 --- a/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java +++ b/src/main/java/org/opensearch/forecast/transport/ForecastSingleStreamResultTransportAction.java @@ -77,7 +77,7 @@ public ForecastResultWriteRequest createResultWriteRequest(Config config, Foreca RequestPriority.MEDIUM, result, config.getCustomResultIndexOrAlias(), - config.getFlattenResultIndexMapping() + config.getFlattenResultIndexAlias() ); } } diff --git a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 84e8b5cb3..2606b1ee4 100644 --- a/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -1037,8 +1037,14 @@ public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionLis choosePrimaryShards(request, false); adminClient.indices().create(request, ActionListener.wrap(response -> { - logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); - actionListener.onResponse(response); + if (response.isAcknowledged()) { + logger.info("Successfully created flattened result index: {} with alias: {}", indexName, flattenedResultIndexAlias); + actionListener.onResponse(response); + } else { + String errorMsg = "Index creation not acknowledged for index: " + indexName; + logger.error(errorMsg); + actionListener.onFailure(new IllegalStateException(errorMsg)); + } }, exception -> { logger.error("Failed to create flattened result index: {}", indexName, exception); actionListener.onFailure(exception); @@ -1049,14 +1055,6 @@ public void initFlattenedResultIndex(String flattenedResultIndexAlias, ActionLis } } - public String getFlattenedResultIndexAlias(String indexOrAliasName, String configId) { - return indexOrAliasName + "_flattened_" + configId.toLowerCase(Locale.ROOT); - } - - public String getFlattenResultIndexIngestPipelineId(String configId) { - return "flatten_result_index_ingest_pipeline" + configId.toLowerCase(Locale.ROOT); - } - public void validateCustomIndexForBackendJob( String resultIndexOrAlias, String securityLogId, @@ -1304,7 +1302,7 @@ protected void rolloverAndDeleteHistoryIndex( candidateResultAliases.forEach(config -> { handleResultIndexRolloverAndDelete(config.getCustomResultIndexOrAlias(), config, resultIndex); if (config.getFlattenResultIndexMapping()) { - String flattenedResultIndexAlias = getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), config.getId()); + String flattenedResultIndexAlias = config.getFlattenResultIndexAlias(); handleResultIndexRolloverAndDelete(flattenedResultIndexAlias, config, resultIndex); } }); diff --git a/src/main/java/org/opensearch/timeseries/model/Config.java b/src/main/java/org/opensearch/timeseries/model/Config.java index eda7a32ec..9ff74c8ad 100644 --- a/src/main/java/org/opensearch/timeseries/model/Config.java +++ b/src/main/java/org/opensearch/timeseries/model/Config.java @@ -750,6 +750,20 @@ public boolean getFlattenResultIndexMapping() { return flattenResultIndexMapping != null ? flattenResultIndexMapping : false; } + public String getFlattenResultIndexAlias() { + if (getFlattenResultIndexMapping()) { + return (getCustomResultIndexOrAlias() + "_flattened_" + getName()).toLowerCase(Locale.ROOT); + } + return null; + } + + public String getFlattenResultIndexIngestPipelineName() { + if (getFlattenResultIndexMapping()) { + return ("flatten_result_index_ingest_pipeline_" + getName()).toLowerCase(Locale.ROOT); + } + return null; + } + public Instant getLastBreakingUIChangeTime() { return lastUIBreakingChangeTime; } diff --git a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java index c56484920..070344324 100644 --- a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java +++ b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteRequest.java @@ -22,7 +22,7 @@ public abstract class ResultWriteRequest ext private final ResultType result; // If resultIndex is null, result will be stored in default result index. private final String resultIndex; - private final boolean flattenResultIndex; + private final String flattenResultIndex; public ResultWriteRequest( long expirationEpochMs, @@ -30,7 +30,7 @@ public ResultWriteRequest( RequestPriority priority, ResultType result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ) { super(expirationEpochMs, configId, priority); this.result = result; @@ -41,14 +41,14 @@ public ResultWriteRequest( public ResultWriteRequest(StreamInput in, Reader resultReader) throws IOException { this.result = resultReader.read(in); this.resultIndex = in.readOptionalString(); - this.flattenResultIndex = in.readBoolean(); + this.flattenResultIndex = in.readOptionalString(); } @Override public void writeTo(StreamOutput out) throws IOException { result.writeTo(out); out.writeOptionalString(resultIndex); - out.writeBoolean(flattenResultIndex); + out.writeOptionalString(flattenResultIndex); } public ResultType getResult() { @@ -59,7 +59,7 @@ public String getResultIndex() { return resultIndex; } - public boolean getFlattenResultIndex() { + public String getFlattenResultIndex() { return flattenResultIndex; } } diff --git a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java index 2c7cdcc59..06ea16149 100644 --- a/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java +++ b/src/main/java/org/opensearch/timeseries/ratelimit/ResultWriteWorker.java @@ -200,7 +200,7 @@ private ActionListener> onGetConfig( resultToRetry.isHighPriority() ? RequestPriority.HIGH : RequestPriority.MEDIUM, resultToRetry, config.getCustomResultIndexOrAlias(), - config.getFlattenResultIndexMapping() + config.getFlattenResultIndexAlias() ) ); @@ -218,6 +218,6 @@ protected abstract ResultWriteRequestType createResultWriteRequest( RequestPriority priority, ResultType result, String resultIndex, - boolean flattenResultIndex + String flattenResultIndex ); } diff --git a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java index a03491024..d5d6f76eb 100644 --- a/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java +++ b/src/main/java/org/opensearch/timeseries/rest/handler/AbstractTimeSeriesActionHandler.java @@ -468,14 +468,16 @@ private void handlePostRequest(boolean indexingDryRun, ActionListener listene createConfig(indexingDryRun, ActionListener.wrap(createConfigResponse -> { if (shouldHandleFlattening(indexingDryRun)) { String configId = RestHandlerUtils.getConfigIdFromIndexResponse(createConfigResponse); - String flattenedResultIndexAlias = timeSeriesIndices - .getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + String flattenedResultIndexAlias = config.getFlattenResultIndexAlias(); timeSeriesIndices .initFlattenedResultIndex( flattenedResultIndexAlias, ActionListener - .wrap(initResponse -> setupIngestPipeline(configId, listener, createConfigResponse), listener::onFailure) + .wrap( + initResponse -> setupIngestPipeline(flattenedResultIndexAlias, configId, listener, createConfigResponse), + listener::onFailure + ) ); } else { listener.onResponse(createConfigResponse); @@ -489,9 +491,13 @@ private boolean shouldHandleFlattening(boolean indexingDryRun) { return !indexingDryRun && config.getCustomResultIndexOrAlias() != null && Boolean.TRUE.equals(flattenResultIndexMapping); } - protected void setupIngestPipeline(String configId, ActionListener listener, T createConfigResponse) { - String flattenedResultIndexAlias = timeSeriesIndices.getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); - String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(configId); + protected void setupIngestPipeline( + String flattenedResultIndexAlias, + String configId, + ActionListener listener, + T createConfigResponse + ) { + String pipelineId = config.getFlattenResultIndexIngestPipelineName(); try { BytesReference pipelineSource = createPipelineDefinition(flattenedResultIndexAlias); @@ -538,10 +544,13 @@ private BytesReference createPipelineDefinition(String indexName) throws IOExcep return BytesReference.bytes(pipelineBuilder); } - private UpdateSettingsRequest buildUpdateSettingsRequest(String defaultPipelineName, String configId) { - String flattenedResultIndex = timeSeriesIndices.getFlattenedResultIndexAlias(config.getCustomResultIndexOrAlias(), configId); + private UpdateSettingsRequest buildUpdateSettingsRequest( + String flattenedResultIndexAlias, + String defaultPipelineName, + String configId + ) { UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(); - updateSettingsRequest.indices(flattenedResultIndex); + updateSettingsRequest.indices(flattenedResultIndexAlias); Settings.Builder settingsBuilder = Settings.builder(); settingsBuilder.put("index.default_pipeline", defaultPipelineName); @@ -558,7 +567,7 @@ protected void bindIngestPipelineWithFlattenedResultIndex( ActionListener listener, T createConfigResponse ) { - UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest(pipelineId, configId); + UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest(flattenedResultIndexAlias, pipelineId, configId); client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(updateSettingsResponse -> { logger.info("Successfully updated settings for index: {} with pipeline: {}", flattenedResultIndexAlias, pipelineId); @@ -664,7 +673,11 @@ private void unbindIngestPipelineWithFlattenedResultIndex( boolean indexingDryRun ) { // The pipeline name _none specifies that the index does not have an ingest pipeline. - UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest("_none", existingConfig.getId()); + UpdateSettingsRequest updateSettingsRequest = buildUpdateSettingsRequest( + existingConfig.getFlattenResultIndexAlias(), + "_none", + existingConfig.getId() + ); client .admin() .indices() @@ -679,7 +692,7 @@ private void unbindIngestPipelineWithFlattenedResultIndex( } private void deleteIngestPipeline(Config existingConfig, ActionListener listener, String id, boolean indexingDryRun) { - String pipelineId = timeSeriesIndices.getFlattenResultIndexIngestPipelineId(existingConfig.getId()); + String pipelineId = existingConfig.getFlattenResultIndexIngestPipelineName(); client .admin() diff --git a/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java b/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java index 843c40bed..d1ff61702 100644 --- a/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java +++ b/src/test/java/org/opensearch/ad/ratelimit/ResultWriteWorkerTests.java @@ -121,7 +121,7 @@ public void testRegular() { RequestPriority.MEDIUM, detectResult, null, - false + null ); request.add(resultWriteRequest); @@ -131,7 +131,7 @@ public void testRegular() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, null)); // the request results one flush verify(resultHandler, times(1)).flush(any(), any()); @@ -154,7 +154,7 @@ public void testSingleRetryRequest() throws IOException { RequestPriority.MEDIUM, detectResult, null, - false + null ); request.add(resultWriteRequest); @@ -170,7 +170,7 @@ public void testSingleRetryRequest() throws IOException { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, null)); // one flush from the original request; and one due to retry verify(resultHandler, times(2)).flush(any(), any()); @@ -190,7 +190,7 @@ public void testRetryException() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, null)); // one flush from the original request; and one due to retry verify(resultHandler, times(2)).flush(any(), any()); verify(nodeStateManager, times(1)).setException(eq(detectorId), any(OpenSearchStatusException.class)); @@ -204,7 +204,7 @@ public void testOverloaded() { return null; }).when(resultHandler).flush(any(), any()); - resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, false)); + resultWriteQueue.put(new ADResultWriteRequest(Long.MAX_VALUE, detectorId, RequestPriority.MEDIUM, detectResult, null, null)); // one flush from the original request; and one due to retry verify(resultHandler, times(1)).flush(any(), any()); verify(nodeStateManager, times(1)).setException(eq(detectorId), any(OpenSearchRejectedExecutionException.class)); diff --git a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java index 6cc8dc2f8..473152fc8 100644 --- a/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java +++ b/src/test/java/org/opensearch/ad/rest/AnomalyDetectorRestApiIT.java @@ -231,8 +231,7 @@ public void testCreateAnomalyDetector_withFlattenedResultIndex() throws Exceptio assertTrue("Incorrect version", version > 0); // Ensure the flattened result index was created - String expectedFlattenedIndex = String - .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); + String expectedFlattenedIndex = "opensearch-ad-plugin-result-test_flattened_detectorwithflattenresultindex"; assertTrue("Alias for flattened result index does not exist", aliasExists(expectedFlattenedIndex)); // Start detector @@ -339,9 +338,8 @@ public void testUpdateAnomalyDetector_disableFlattenResultIndex_shouldDeletePipe assertEquals("Create anomaly detector with flattened result index failed", RestStatus.CREATED, TestHelpers.restStatus(response)); Map responseMap = entityAsMap(response); String id = (String) responseMap.get("_id"); - String expectedFlattenedIndex = String - .format(Locale.ROOT, "opensearch-ad-plugin-result-test_flattened_%s", id.toLowerCase(Locale.ROOT)); - String expectedPipelineId = String.format(Locale.ROOT, "flatten_result_index_ingest_pipeline%s", id.toLowerCase(Locale.ROOT)); + String expectedFlattenedIndex = "opensearch-ad-plugin-result-test_flattened_detectorwithflattenresultindex"; + String expectedPipelineId = "flatten_result_index_ingest_pipeline_detectorwithflattenresultindex"; String getIngestPipelineEndpoint = String.format(Locale.ROOT, "_ingest/pipeline/%s", expectedPipelineId); Response getPipelineResponse = TestHelpers.makeRequest(client(), "GET", getIngestPipelineEndpoint, ImmutableMap.of(), "", null); assertEquals( diff --git a/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java b/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java index a19442112..ee920c5a1 100644 --- a/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java +++ b/src/test/java/org/opensearch/ad/transport/handler/MultiEntityResultHandlerTests.java @@ -51,7 +51,7 @@ public void setUp() throws Exception { RequestPriority.MEDIUM, TestHelpers.randomAnomalyDetectResult(), null, - false + null ); request.add(resultWriteRequest); diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index 1dd709e12..b4bd580a0 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -517,7 +517,7 @@ public static AnomalyDetector randomAnomalyDetectorWithFlattenResultIndex(String return new AnomalyDetector( randomAlphaOfLength(10), randomLong(), - randomAlphaOfLength(20), + "detectorWithFlattenResultIndex", randomAlphaOfLength(30), timefield, ImmutableList.of(indexName.toLowerCase(Locale.ROOT)), @@ -1054,7 +1054,7 @@ public static ADResultWriteRequest randomADResultWriteRequest(String detectorId, RequestPriority.MEDIUM, randomHCADAnomalyDetectResult(score, grade), null, - false + null ); return resultWriteRequest; } @@ -2280,7 +2280,7 @@ public static ForecastResultWriteRequest randomForecastResultWriteRequest() { ForecastResult result = randomForecastResult(forecasterId); String resultIndex = random.nextBoolean() ? randomAlphaOfLength(10) : null; // Randomly decide to set resultIndex or not - return new ForecastResultWriteRequest(expirationEpochMs, forecasterId, priority, result, resultIndex, false); + return new ForecastResultWriteRequest(expirationEpochMs, forecasterId, priority, result, resultIndex, null); } public static ForecastResult randomForecastResult(String forecasterId) {