diff --git a/build.gradle b/build.gradle index 69d8e9d61..f4a002bac 100644 --- a/build.gradle +++ b/build.gradle @@ -34,7 +34,7 @@ buildscript { js_resource_folder = "src/test/resources/job-scheduler" common_utils_version = System.getProperty("common_utils.version", opensearch_build) job_scheduler_version = System.getProperty("job_scheduler.version", opensearch_build) - bwcVersionShort = "2.8.0" + bwcVersionShort = "2.9.0" bwcVersion = bwcVersionShort + ".0" bwcOpenSearchADDownload = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + bwcVersionShort + '/latest/linux/x64/tar/builds/' + 'opensearch/plugins/opensearch-anomaly-detection-' + bwcVersion + '.zip' diff --git a/src/main/java/org/opensearch/ad/AnomalyDetectorJobRunner.java b/src/main/java/org/opensearch/ad/AnomalyDetectorJobRunner.java index 32639f216..fa7682b5e 100644 --- a/src/main/java/org/opensearch/ad/AnomalyDetectorJobRunner.java +++ b/src/main/java/org/opensearch/ad/AnomalyDetectorJobRunner.java @@ -31,11 +31,10 @@ import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.WriteRequest; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.ADTaskState; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.ad.transport.AnomalyResultAction; @@ -63,6 +62,7 @@ import org.opensearch.timeseries.common.exception.InternalFailure; import org.opensearch.timeseries.common.exception.TimeSeriesException; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import com.google.common.base.Throwables; @@ -77,7 +77,7 @@ public class AnomalyDetectorJobRunner implements ScheduledJobRunner { private Client client; private ThreadPool threadPool; private ConcurrentHashMap detectorEndRunExceptionCount; - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; private ADTaskManager adTaskManager; private NodeStateManager nodeStateManager; private ExecuteADResultResponseRecorder recorder; @@ -117,7 +117,7 @@ public void setAdTaskManager(ADTaskManager adTaskManager) { this.adTaskManager = adTaskManager; } - public void setAnomalyDetectionIndices(AnomalyDetectionIndices anomalyDetectionIndices) { + public void setAnomalyDetectionIndices(ADIndexManagement anomalyDetectionIndices) { this.anomalyDetectionIndices = anomalyDetectionIndices; } @@ -514,7 +514,7 @@ private void stopAdJobForEndRunException( ); } - private void stopAdJob(String detectorId, AnomalyDetectorFunction function) { + private void stopAdJob(String detectorId, ExecutorFunction function) { GetRequest getRequest = new GetRequest(CommonName.JOB_INDEX).id(detectorId); ActionListener listener = ActionListener.wrap(response -> { if (response.isExists()) { diff --git a/src/main/java/org/opensearch/ad/AnomalyDetectorPlugin.java b/src/main/java/org/opensearch/ad/AnomalyDetectorPlugin.java index 7bbf4347b..774e79fa8 100644 --- a/src/main/java/org/opensearch/ad/AnomalyDetectorPlugin.java +++ b/src/main/java/org/opensearch/ad/AnomalyDetectorPlugin.java @@ -45,7 +45,7 @@ import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.feature.FeatureManager; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.CheckpointDao; import org.opensearch.ad.ml.EntityColdStarter; import org.opensearch.ad.ml.HybridThresholdingModel; @@ -154,7 +154,6 @@ import org.opensearch.ad.transport.handler.AnomalyResultBulkIndexHandler; import org.opensearch.ad.transport.handler.MultiEntityResultHandler; import org.opensearch.ad.util.ClientUtil; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.ad.util.IndexUtils; import org.opensearch.ad.util.SecurityClientUtil; import org.opensearch.ad.util.Throttler; @@ -176,6 +175,7 @@ import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.forecast.model.Forecaster; +import org.opensearch.forecast.settings.ForecastSettings; import org.opensearch.jobscheduler.spi.JobSchedulerExtension; import org.opensearch.jobscheduler.spi.ScheduledJobParser; import org.opensearch.jobscheduler.spi.ScheduledJobRunner; @@ -194,7 +194,10 @@ import org.opensearch.timeseries.constant.CommonName; import org.opensearch.timeseries.dataprocessor.Imputer; import org.opensearch.timeseries.dataprocessor.LinearUniformImputer; +import org.opensearch.timeseries.function.ThrowingSupplierWrapper; +import org.opensearch.timeseries.settings.TimeSeriesSettings; import org.opensearch.timeseries.stats.StatNames; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.watcher.ResourceWatcherService; import com.amazon.randomcutforest.parkservices.state.ThresholdedRandomCutForestMapper; @@ -226,7 +229,7 @@ public class AnomalyDetectorPlugin extends Plugin implements ActionPlugin, Scrip public static final String AD_BATCH_TASK_THREAD_POOL_NAME = "ad-batch-task-threadpool"; public static final String AD_JOB_TYPE = "opendistro_anomaly_detector"; private static Gson gson; - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; private AnomalyDetectorRunner anomalyDetectorRunner; private Client client; private ClusterService clusterService; @@ -333,14 +336,19 @@ public Collection createComponents( this.clientUtil = new ClientUtil(settings, client, throttler, threadPool); this.indexUtils = new IndexUtils(client, clientUtil, clusterService, indexNameExpressionResolver); this.nodeFilter = new DiscoveryNodeFilterer(clusterService); - this.anomalyDetectionIndices = new AnomalyDetectionIndices( - client, - clusterService, - threadPool, - settings, - nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES - ); + // convert from checked IOException to unchecked RuntimeException + this.anomalyDetectionIndices = ThrowingSupplierWrapper + .throwingSupplierWrapper( + () -> new ADIndexManagement( + client, + clusterService, + threadPool, + settings, + nodeFilter, + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES + ) + ) + .get(); this.clusterService = clusterService; Imputer imputer = new LinearUniformImputer(true); @@ -853,6 +861,9 @@ public List> getSettings() { List> systemSetting = ImmutableList .of( + // ====================================== + // AD settings + // ====================================== // HCAD cache LegacyOpenDistroAnomalyDetectorSettings.MAX_CACHE_MISS_HANDLING_PER_SECOND, AnomalyDetectorSettings.DEDICATED_CACHE_SIZE, @@ -894,7 +905,7 @@ public List> getSettings() { AnomalyDetectorSettings.MAX_MULTI_ENTITY_ANOMALY_DETECTORS, AnomalyDetectorSettings.AD_INDEX_PRESSURE_SOFT_LIMIT, AnomalyDetectorSettings.AD_INDEX_PRESSURE_HARD_LIMIT, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS, + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS, // Security LegacyOpenDistroAnomalyDetectorSettings.FILTER_BY_BACKEND_ROLES, AnomalyDetectorSettings.FILTER_BY_BACKEND_ROLES, @@ -938,7 +949,22 @@ public List> getSettings() { // clean resource AnomalyDetectorSettings.DELETE_AD_RESULT_WHEN_DELETE_DETECTOR, // stats/profile API - AnomalyDetectorSettings.MAX_MODEL_SIZE_PER_NODE + AnomalyDetectorSettings.MAX_MODEL_SIZE_PER_NODE, + // ====================================== + // Forecast settings + // ====================================== + // result index rollover + ForecastSettings.FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD, + ForecastSettings.FORECAST_RESULT_HISTORY_RETENTION_PERIOD, + ForecastSettings.FORECAST_RESULT_HISTORY_ROLLOVER_PERIOD, + // resource usage control + ForecastSettings.FORECAST_MODEL_MAX_SIZE_PERCENTAGE, + // TODO: add validation code + // ForecastSettings.FORECAST_MAX_SINGLE_STREAM_FORECASTERS, + // ForecastSettings.FORECAST_MAX_HC_FORECASTERS, + ForecastSettings.FORECAST_INDEX_PRESSURE_SOFT_LIMIT, + ForecastSettings.FORECAST_INDEX_PRESSURE_HARD_LIMIT, + ForecastSettings.FORECAST_MAX_PRIMARY_SHARDS ); return unmodifiableList( Stream diff --git a/src/main/java/org/opensearch/ad/AnomalyDetectorProfileRunner.java b/src/main/java/org/opensearch/ad/AnomalyDetectorProfileRunner.java index b5e56ba51..22ad7b369 100644 --- a/src/main/java/org/opensearch/ad/AnomalyDetectorProfileRunner.java +++ b/src/main/java/org/opensearch/ad/AnomalyDetectorProfileRunner.java @@ -49,7 +49,6 @@ import org.opensearch.ad.transport.RCFPollingAction; import org.opensearch.ad.transport.RCFPollingRequest; import org.opensearch.ad.transport.RCFPollingResponse; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.ad.util.ExceptionUtil; import org.opensearch.ad.util.MultiResponsesDelegateActionListener; import org.opensearch.ad.util.SecurityClientUtil; @@ -73,6 +72,7 @@ import org.opensearch.timeseries.common.exception.ResourceNotFoundException; import org.opensearch.timeseries.constant.CommonName; import org.opensearch.timeseries.model.IntervalTimeConfiguration; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; public class AnomalyDetectorProfileRunner extends AbstractProfileRunner { diff --git a/src/main/java/org/opensearch/ad/ExecuteADResultResponseRecorder.java b/src/main/java/org/opensearch/ad/ExecuteADResultResponseRecorder.java index eb56a885d..d63343bdb 100644 --- a/src/main/java/org/opensearch/ad/ExecuteADResultResponseRecorder.java +++ b/src/main/java/org/opensearch/ad/ExecuteADResultResponseRecorder.java @@ -25,7 +25,7 @@ import org.opensearch.action.update.UpdateResponse; import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.indices.ADIndex; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyResult; import org.opensearch.ad.model.DetectorProfileName; @@ -37,7 +37,6 @@ import org.opensearch.ad.transport.RCFPollingAction; import org.opensearch.ad.transport.RCFPollingRequest; import org.opensearch.ad.transport.handler.AnomalyIndexHandler; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.ad.util.ExceptionUtil; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; @@ -50,11 +49,12 @@ import org.opensearch.timeseries.common.exception.TimeSeriesException; import org.opensearch.timeseries.model.FeatureData; import org.opensearch.timeseries.model.IntervalTimeConfiguration; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class ExecuteADResultResponseRecorder { private static final Logger log = LogManager.getLogger(ExecuteADResultResponseRecorder.class); - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; private AnomalyIndexHandler anomalyResultHandler; private ADTaskManager adTaskManager; private DiscoveryNodeFilterer nodeFilter; @@ -65,7 +65,7 @@ public class ExecuteADResultResponseRecorder { private int rcfMinSamples; public ExecuteADResultResponseRecorder( - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, AnomalyIndexHandler anomalyResultHandler, ADTaskManager adTaskManager, DiscoveryNodeFilterer nodeFilter, diff --git a/src/main/java/org/opensearch/ad/NodeStateManager.java b/src/main/java/org/opensearch/ad/NodeStateManager.java index 0f5d32e35..e99cfdbe8 100644 --- a/src/main/java/org/opensearch/ad/NodeStateManager.java +++ b/src/main/java/org/opensearch/ad/NodeStateManager.java @@ -39,11 +39,11 @@ import org.opensearch.ad.util.ExceptionUtil; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.timeseries.common.exception.EndRunException; diff --git a/src/main/java/org/opensearch/ad/cluster/ADDataMigrator.java b/src/main/java/org/opensearch/ad/cluster/ADDataMigrator.java index 181788f1f..d5e0ace05 100644 --- a/src/main/java/org/opensearch/ad/cluster/ADDataMigrator.java +++ b/src/main/java/org/opensearch/ad/cluster/ADDataMigrator.java @@ -37,14 +37,13 @@ import org.opensearch.action.search.SearchRequest; import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.ADTaskState; import org.opensearch.ad.model.ADTaskType; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; import org.opensearch.ad.model.DetectorInternalState; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.util.ExceptionUtil; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; @@ -60,6 +59,7 @@ import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.timeseries.common.exception.ResourceNotFoundException; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; /** * Migrate AD data to support backward compatibility. @@ -71,14 +71,14 @@ public class ADDataMigrator { private final Client client; private final ClusterService clusterService; private final NamedXContentRegistry xContentRegistry; - private final AnomalyDetectionIndices detectionIndices; + private final ADIndexManagement detectionIndices; private final AtomicBoolean dataMigrated; public ADDataMigrator( Client client, ClusterService clusterService, NamedXContentRegistry xContentRegistry, - AnomalyDetectionIndices detectionIndices + ADIndexManagement detectionIndices ) { this.client = client; this.clusterService = clusterService; @@ -94,12 +94,12 @@ public void migrateData() { if (!dataMigrated.getAndSet(true)) { logger.info("Start migrating AD data"); - if (!detectionIndices.doesAnomalyDetectorJobIndexExist()) { + if (!detectionIndices.doesJobIndexExist()) { logger.info("AD job index doesn't exist, no need to migrate"); return; } - if (detectionIndices.doesDetectorStateIndexExist()) { + if (detectionIndices.doesStateIndexExist()) { migrateDetectorInternalStateToRealtimeTask(); } else { // If detection index doesn't exist, create index and backfill realtime task. @@ -179,7 +179,7 @@ public void backfillRealtimeTask(ConcurrentLinkedQueue detec } String jobId = job.getName(); - AnomalyDetectorFunction createRealtimeTaskFunction = () -> { + ExecutorFunction createRealtimeTaskFunction = () -> { GetRequest getRequest = new GetRequest(DETECTION_STATE_INDEX, jobId); client.get(getRequest, ActionListener.wrap(r -> { if (r != null && r.isExists()) { @@ -204,7 +204,7 @@ public void backfillRealtimeTask(ConcurrentLinkedQueue detec private void checkIfRealtimeTaskExistsAndBackfill( AnomalyDetectorJob job, - AnomalyDetectorFunction createRealtimeTaskFunction, + ExecutorFunction createRealtimeTaskFunction, ConcurrentLinkedQueue detectorJobs, boolean migrateAll ) { diff --git a/src/main/java/org/opensearch/ad/cluster/ClusterManagerEventListener.java b/src/main/java/org/opensearch/ad/cluster/ClusterManagerEventListener.java index 99e175193..9cf1dd905 100644 --- a/src/main/java/org/opensearch/ad/cluster/ClusterManagerEventListener.java +++ b/src/main/java/org/opensearch/ad/cluster/ClusterManagerEventListener.java @@ -18,7 +18,6 @@ import org.opensearch.ad.cluster.diskcleanup.ModelCheckpointIndexRetention; import org.opensearch.ad.util.ClientUtil; import org.opensearch.ad.util.DateUtils; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.LocalNodeClusterManagerListener; import org.opensearch.cluster.service.ClusterService; @@ -28,6 +27,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.threadpool.Scheduler.Cancellable; import org.opensearch.threadpool.ThreadPool; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import com.google.common.annotations.VisibleForTesting; diff --git a/src/main/java/org/opensearch/ad/cluster/HashRing.java b/src/main/java/org/opensearch/ad/cluster/HashRing.java index 3344332ae..edad3299b 100644 --- a/src/main/java/org/opensearch/ad/cluster/HashRing.java +++ b/src/main/java/org/opensearch/ad/cluster/HashRing.java @@ -40,7 +40,6 @@ import org.opensearch.action.admin.cluster.node.info.PluginsAndModules; import org.opensearch.ad.ml.ModelManager; import org.opensearch.ad.ml.SingleStreamModelIdMapper; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.client.ClusterAdminClient; @@ -54,6 +53,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.plugins.PluginInfo; import org.opensearch.timeseries.common.exception.TimeSeriesException; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import com.google.common.collect.Sets; diff --git a/src/main/java/org/opensearch/ad/cluster/HourlyCron.java b/src/main/java/org/opensearch/ad/cluster/HourlyCron.java index 4a574ecfb..a81156bb0 100644 --- a/src/main/java/org/opensearch/ad/cluster/HourlyCron.java +++ b/src/main/java/org/opensearch/ad/cluster/HourlyCron.java @@ -17,9 +17,9 @@ import org.opensearch.action.FailedNodeException; import org.opensearch.ad.transport.CronAction; import org.opensearch.ad.transport.CronRequest; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class HourlyCron implements Runnable { private static final Logger LOG = LogManager.getLogger(HourlyCron.class); diff --git a/src/main/java/org/opensearch/ad/indices/ADIndex.java b/src/main/java/org/opensearch/ad/indices/ADIndex.java index c936947b0..b345ef33e 100644 --- a/src/main/java/org/opensearch/ad/indices/ADIndex.java +++ b/src/main/java/org/opensearch/ad/indices/ADIndex.java @@ -14,41 +14,30 @@ import java.util.function.Supplier; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.util.ThrowingSupplierWrapper; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ThrowingSupplierWrapper; +import org.opensearch.timeseries.indices.TimeSeriesIndex; /** * Represent an AD index * */ -public enum ADIndex { +public enum ADIndex implements TimeSeriesIndex { // throw RuntimeException since we don't know how to handle the case when the mapping reading throws IOException RESULT( ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, true, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getAnomalyResultMappings) - ), - CONFIG( - CommonName.CONFIG_INDEX, - false, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getAnomalyDetectorMappings) - ), - JOB( - CommonName.JOB_INDEX, - false, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getAnomalyDetectorJobMappings) + ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getResultMappings) ), + CONFIG(CommonName.CONFIG_INDEX, false, ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getConfigMappings)), + JOB(CommonName.JOB_INDEX, false, ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getJobMappings)), CHECKPOINT( ADCommonName.CHECKPOINT_INDEX_NAME, false, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getCheckpointMappings) + ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getCheckpointMappings) ), - STATE( - ADCommonName.DETECTION_STATE_INDEX, - false, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getDetectionStateMappings) - ); + STATE(ADCommonName.DETECTION_STATE_INDEX, false, ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getStateMappings)); private final String indexName; // whether we use an alias for the index @@ -61,16 +50,24 @@ public enum ADIndex { this.mapping = mappingSupplier.get(); } + @Override public String getIndexName() { return indexName; } + @Override public boolean isAlias() { return alias; } + @Override public String getMapping() { return mapping; } + @Override + public boolean isJobIndex() { + return CommonName.JOB_INDEX.equals(indexName); + } + } diff --git a/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java new file mode 100644 index 000000000..fb0d9d4d5 --- /dev/null +++ b/src/main/java/org/opensearch/ad/indices/ADIndexManagement.java @@ -0,0 +1,274 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.ad.indices; + +import static org.opensearch.ad.constant.ADCommonName.DUMMY_AD_RESULT_ID; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD; +import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD; +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 java.io.IOException; +import java.util.EnumMap; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.ad.constant.ADCommonName; +import org.opensearch.ad.model.AnomalyResult; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.timeseries.common.exception.EndRunException; +import org.opensearch.timeseries.indices.IndexManagement; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; + +/** + * This class provides utility methods for various anomaly detection indices. + */ +public class ADIndexManagement extends IndexManagement { + private static final Logger logger = LogManager.getLogger(ADIndexManagement.class); + + // The index name pattern to query all the AD result history indices + public static final String AD_RESULT_HISTORY_INDEX_PATTERN = "<.opendistro-anomaly-results-history-{now/d}-1>"; + + // 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*"; + + /** + * Constructor function + * + * @param client OS client supports administrative actions + * @param clusterService OS cluster service + * @param threadPool OS thread pool + * @param settings OS cluster setting + * @param nodeFilter Used to filter eligible nodes to host AD indices + * @param maxUpdateRunningTimes max number of retries to update index mapping and setting + * @throws IOException + */ + public ADIndexManagement( + Client client, + ClusterService clusterService, + ThreadPool threadPool, + Settings settings, + DiscoveryNodeFilterer nodeFilter, + int maxUpdateRunningTimes + ) + throws IOException { + super( + client, + clusterService, + threadPool, + settings, + nodeFilter, + maxUpdateRunningTimes, + ADIndex.class, + AD_MAX_PRIMARY_SHARDS.get(settings), + AD_RESULT_HISTORY_ROLLOVER_PERIOD.get(settings), + AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD.get(settings), + AD_RESULT_HISTORY_RETENTION_PERIOD.get(settings), + ADIndex.RESULT.getMapping() + ); + this.clusterService.addLocalNodeClusterManagerListener(this); + + this.indexStates = new EnumMap(ADIndex.class); + + this.clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, it -> historyMaxDocs = it); + + this.clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_RESULT_HISTORY_ROLLOVER_PERIOD, it -> { + historyRolloverPeriod = it; + rescheduleRollover(); + }); + this.clusterService + .getClusterSettings() + .addSettingsUpdateConsumer(AD_RESULT_HISTORY_RETENTION_PERIOD, it -> { historyRetentionPeriod = it; }); + + this.clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_MAX_PRIMARY_SHARDS, it -> maxPrimaryShards = it); + } + + /** + * Get anomaly result index mapping json content. + * + * @return anomaly result index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getResultMappings() throws IOException { + return getMappings(ANOMALY_RESULTS_INDEX_MAPPING_FILE); + } + + /** + * Get anomaly detector state index mapping json content. + * + * @return anomaly detector state index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getStateMappings() throws IOException { + String detectionStateMappings = getMappings(ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE); + String detectorIndexMappings = getConfigMappings(); + detectorIndexMappings = detectorIndexMappings + .substring(detectorIndexMappings.indexOf("\"properties\""), detectorIndexMappings.lastIndexOf("}")); + return detectionStateMappings.replace("DETECTOR_INDEX_MAPPING_PLACE_HOLDER", detectorIndexMappings); + } + + /** + * Get checkpoint index mapping json content. + * + * @return checkpoint index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getCheckpointMappings() throws IOException { + return getMappings(CHECKPOINT_INDEX_MAPPING_FILE); + } + + /** + * anomaly result index exist or not. + * + * @return true if anomaly result index exists + */ + @Override + public boolean doesDefaultResultIndexExist() { + return doesAliasExist(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS); + } + + /** + * Anomaly state index exist or not. + * + * @return true if anomaly state index exists + */ + @Override + public boolean doesStateIndexExist() { + return doesIndexExist(ADCommonName.DETECTION_STATE_INDEX); + } + + /** + * Checkpoint index exist or not. + * + * @return true if checkpoint index exists + */ + @Override + public boolean doesCheckpointIndexExist() { + return doesIndexExist(ADCommonName.CHECKPOINT_INDEX_NAME); + } + + /** + * Create anomaly result index without checking exist or not. + * + * @param actionListener action called after create index + */ + @Override + public void initDefaultResultIndexDirectly(ActionListener actionListener) { + initResultIndexDirectly( + AD_RESULT_HISTORY_INDEX_PATTERN, + ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, + true, + AD_RESULT_HISTORY_INDEX_PATTERN, + ADIndex.RESULT, + actionListener + ); + } + + /** + * Create the state index. + * + * @param actionListener action called after create index + */ + public void initDetectionStateIndex(ActionListener actionListener) { + try { + CreateIndexRequest request = new CreateIndexRequest(ADCommonName.DETECTION_STATE_INDEX) + .mapping(getStateMappings(), XContentType.JSON) + .settings(settings); + adminClient.indices().create(request, markMappingUpToDate(ADIndex.STATE, actionListener)); + } catch (IOException e) { + logger.error("Fail to init AD detection state index", e); + actionListener.onFailure(e); + } + } + + /** + * Create the checkpoint index. + * + * @param actionListener action called after create index + * @throws EndRunException EndRunException due to failure to get mapping + */ + @Override + public void initCheckpointIndex(ActionListener actionListener) { + String mapping; + try { + mapping = getCheckpointMappings(); + } catch (IOException e) { + throw new EndRunException("", "Cannot find checkpoint mapping file", true); + } + CreateIndexRequest request = new CreateIndexRequest(ADCommonName.CHECKPOINT_INDEX_NAME).mapping(mapping, XContentType.JSON); + choosePrimaryShards(request, true); + adminClient.indices().create(request, markMappingUpToDate(ADIndex.CHECKPOINT, actionListener)); + } + + @Override + protected void rolloverAndDeleteHistoryIndex() { + rolloverAndDeleteHistoryIndex( + ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, + ALL_AD_RESULTS_INDEX_PATTERN, + AD_RESULT_HISTORY_INDEX_PATTERN, + ADIndex.RESULT + ); + } + + /** + * Create config index directly. + * + * @param actionListener action called after create index + * @throws IOException IOException from {@link IndexManagement#getConfigMappings} + */ + @Override + public void initConfigIndex(ActionListener actionListener) throws IOException { + super.initConfigIndex(markMappingUpToDate(ADIndex.CONFIG, actionListener)); + } + + /** + * Create job index. + * + * @param actionListener action called after create index + */ + @Override + public void initJobIndex(ActionListener actionListener) { + super.initJobIndex(markMappingUpToDate(ADIndex.JOB, actionListener)); + } + + @Override + protected IndexRequest createDummyIndexRequest(String resultIndex) throws IOException { + AnomalyResult dummyResult = AnomalyResult.getDummyResult(); + return new IndexRequest(resultIndex) + .id(DUMMY_AD_RESULT_ID) + .source(dummyResult.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS)); + } + + @Override + protected DeleteRequest createDummyDeleteRequest(String resultIndex) throws IOException { + return new DeleteRequest(resultIndex).id(DUMMY_AD_RESULT_ID); + } + + @Override + public void initCustomResultIndexDirectly(String resultIndex, ActionListener actionListener) { + initResultIndexDirectly(resultIndex, null, false, AD_RESULT_HISTORY_INDEX_PATTERN, ADIndex.RESULT, actionListener); + } +} diff --git a/src/main/java/org/opensearch/ad/ml/CheckpointDao.java b/src/main/java/org/opensearch/ad/ml/CheckpointDao.java index b7718ec01..738acd197 100644 --- a/src/main/java/org/opensearch/ad/ml/CheckpointDao.java +++ b/src/main/java/org/opensearch/ad/ml/CheckpointDao.java @@ -56,7 +56,7 @@ import org.opensearch.action.update.UpdateResponse; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.indices.ADIndex; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.util.ClientUtil; import org.opensearch.client.Client; import org.opensearch.index.IndexNotFoundException; @@ -128,7 +128,7 @@ public class CheckpointDao { private final Class thresholdingModelClass; - private final AnomalyDetectionIndices indexUtil; + private final ADIndexManagement indexUtil; private final JsonParser parser = new JsonParser(); // we won't read/write a checkpoint larger than a threshold private final int maxCheckpointBytes; @@ -166,7 +166,7 @@ public CheckpointDao( ThresholdedRandomCutForestMapper trcfMapper, Schema trcfSchema, Class thresholdingModelClass, - AnomalyDetectionIndices indexUtil, + ADIndexManagement indexUtil, int maxCheckpointBytes, GenericObjectPool serializeRCFBufferPool, int serializeRCFBufferSize, diff --git a/src/main/java/org/opensearch/ad/ratelimit/CheckpointReadWorker.java b/src/main/java/org/opensearch/ad/ratelimit/CheckpointReadWorker.java index 997be3fff..dca44e173 100644 --- a/src/main/java/org/opensearch/ad/ratelimit/CheckpointReadWorker.java +++ b/src/main/java/org/opensearch/ad/ratelimit/CheckpointReadWorker.java @@ -38,7 +38,7 @@ import org.opensearch.ad.caching.CacheProvider; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.indices.ADIndex; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.CheckpointDao; import org.opensearch.ad.ml.EntityModel; import org.opensearch.ad.ml.ModelManager; @@ -78,7 +78,7 @@ public class CheckpointReadWorker extends BatchWorker listener, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, String detectorId, Long seqNo, Long primaryTerm, @@ -284,10 +284,10 @@ public void start() { // index won't be created, only validation checks will be executed throughout the class private void createOrUpdateDetector() { try (ThreadContext.StoredContext context = client.threadPool().getThreadContext().stashContext()) { - if (!anomalyDetectionIndices.doesAnomalyDetectorIndexExist() && !this.isDryRun) { + if (!anomalyDetectionIndices.doesConfigIndexExist() && !this.isDryRun) { logger.info("AnomalyDetector Indices do not exist"); anomalyDetectionIndices - .initAnomalyDetectorIndex( + .initConfigIndex( ActionListener .wrap(response -> onCreateMappingsResponse(response, false), exception -> listener.onFailure(exception)) ); @@ -466,7 +466,7 @@ protected boolean hasCategoryField(AnomalyDetector detector) { } protected void validateAgainstExistingMultiEntityAnomalyDetector(String detectorId, boolean indexingDryRun) { - if (anomalyDetectionIndices.doesAnomalyDetectorIndexExist()) { + if (anomalyDetectionIndices.doesConfigIndexExist()) { QueryBuilder query = QueryBuilders.boolQuery().filter(QueryBuilders.existsQuery(AnomalyDetector.CATEGORY_FIELD)); SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder().query(query).size(0).timeout(requestTimeout); @@ -493,7 +493,7 @@ protected void createAnomalyDetector(boolean indexingDryRun) { if (categoricalFields != null && categoricalFields.size() > 0) { validateAgainstExistingMultiEntityAnomalyDetector(null, indexingDryRun); } else { - if (anomalyDetectionIndices.doesAnomalyDetectorIndexExist()) { + if (anomalyDetectionIndices.doesConfigIndexExist()) { QueryBuilder query = QueryBuilders.matchAllQuery(); SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder().query(query).size(0).timeout(requestTimeout); @@ -685,7 +685,7 @@ protected void onSearchAdInputIndicesResponse(SearchResponse response, String de } protected void checkADNameExists(String detectorId, boolean indexingDryRun) throws IOException { - if (anomalyDetectionIndices.doesAnomalyDetectorIndexExist()) { + if (anomalyDetectionIndices.doesConfigIndexExist()) { BoolQueryBuilder boolQueryBuilder = new BoolQueryBuilder(); // src/main/resources/mappings/anomaly-detectors.json#L14 boolQueryBuilder.must(QueryBuilders.termQuery("name.keyword", anomalyDetector.getName())); diff --git a/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorActionHandler.java b/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorActionHandler.java index f970c6da2..ed5376be9 100644 --- a/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorActionHandler.java +++ b/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorActionHandler.java @@ -28,6 +28,7 @@ import org.opensearch.core.xcontent.XContentParser; import org.opensearch.rest.RestStatus; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.util.RestHandlerUtils; /** @@ -53,7 +54,7 @@ public void getDetectorJob( Client client, String detectorId, ActionListener listener, - AnomalyDetectorFunction function, + ExecutorFunction function, NamedXContentRegistry xContentRegistry ) { if (clusterService.state().metadata().indices().containsKey(CommonName.JOB_INDEX)) { @@ -75,7 +76,7 @@ public void getDetectorJob( private void onGetAdJobResponseForWrite( GetResponse response, ActionListener listener, - AnomalyDetectorFunction function, + ExecutorFunction function, NamedXContentRegistry xContentRegistry ) { if (response.isExists()) { diff --git a/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java b/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java index 155d2c562..b401ce007 100644 --- a/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java +++ b/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java @@ -14,7 +14,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.ad.transport.IndexAnomalyDetectorResponse; @@ -66,7 +66,7 @@ public IndexAnomalyDetectorActionHandler( SecurityClientUtil clientUtil, TransportService transportService, ActionListener listener, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, String detectorId, Long seqNo, Long primaryTerm, diff --git a/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java b/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java index 07960b998..a81e00a35 100644 --- a/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java +++ b/src/main/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java @@ -31,7 +31,7 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.ExecuteADResultResponseRecorder; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.ADTaskState; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; @@ -51,6 +51,7 @@ import org.opensearch.jobscheduler.spi.schedule.Schedule; import org.opensearch.rest.RestStatus; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.IntervalTimeConfiguration; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportService; @@ -62,7 +63,7 @@ */ public class IndexAnomalyDetectorJobActionHandler { - private final AnomalyDetectionIndices anomalyDetectionIndices; + private final ADIndexManagement anomalyDetectionIndices; private final String detectorId; private final Long seqNo; private final Long primaryTerm; @@ -91,7 +92,7 @@ public class IndexAnomalyDetectorJobActionHandler { */ public IndexAnomalyDetectorJobActionHandler( Client client, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, String detectorId, Long seqNo, Long primaryTerm, @@ -160,8 +161,8 @@ public void startAnomalyDetectorJob(AnomalyDetector detector, ActionListener { + if (!anomalyDetectionIndices.doesJobIndexExist()) { + anomalyDetectionIndices.initJobIndex(ActionListener.wrap(response -> { if (response.isAcknowledged()) { logger.info("Created {} with mappings.", CommonName.CONFIG_INDEX); createJob(detector, startListener); @@ -290,7 +291,7 @@ private void onGetAnomalyDetectorJobForWrite( private void indexAnomalyDetectorJob( AnomalyDetectorJob job, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) throws IOException { IndexRequest indexRequest = new IndexRequest(CommonName.JOB_INDEX) @@ -313,7 +314,7 @@ private void indexAnomalyDetectorJob( private void onIndexAnomalyDetectorJobResponse( IndexResponse response, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { if (response == null || (response.getResult() != CREATED && response.getResult() != UPDATED)) { diff --git a/src/main/java/org/opensearch/ad/rest/handler/ValidateAnomalyDetectorActionHandler.java b/src/main/java/org/opensearch/ad/rest/handler/ValidateAnomalyDetectorActionHandler.java index a51eccd03..163d1df63 100644 --- a/src/main/java/org/opensearch/ad/rest/handler/ValidateAnomalyDetectorActionHandler.java +++ b/src/main/java/org/opensearch/ad/rest/handler/ValidateAnomalyDetectorActionHandler.java @@ -15,7 +15,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.transport.ValidateAnomalyDetectorResponse; import org.opensearch.ad.util.SecurityClientUtil; @@ -59,7 +59,7 @@ public ValidateAnomalyDetectorActionHandler( Client client, SecurityClientUtil clientUtil, ActionListener listener, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, AnomalyDetector anomalyDetector, TimeValue requestTimeout, Integer maxSingleEntityAnomalyDetectors, diff --git a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java index 4e609d8e3..22e72eba0 100644 --- a/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java +++ b/src/main/java/org/opensearch/ad/settings/AnomalyDetectorSettings.java @@ -351,7 +351,7 @@ private AnomalyDetectorSettings() {} ); // max number of primary shards of an AD index - public static final Setting MAX_PRIMARY_SHARDS = Setting + public static final Setting AD_MAX_PRIMARY_SHARDS = Setting .intSetting( "plugins.anomaly_detection.max_primary_shards", LegacyOpenDistroAnomalyDetectorSettings.MAX_PRIMARY_SHARDS, @@ -807,11 +807,6 @@ private AnomalyDetectorSettings() {} // total entities up to 10,000. public static final int MAX_TOTAL_ENTITIES_TO_TRACK = 10_000; - // ====================================== - // AD Index setting - // ====================================== - public static int MAX_UPDATE_RETRY_TIMES = 10_000; - // ====================================== // Cold start setting // ====================================== diff --git a/src/main/java/org/opensearch/ad/task/ADBatchTaskRunner.java b/src/main/java/org/opensearch/ad/task/ADBatchTaskRunner.java index e79e55c73..d944f67ab 100644 --- a/src/main/java/org/opensearch/ad/task/ADBatchTaskRunner.java +++ b/src/main/java/org/opensearch/ad/task/ADBatchTaskRunner.java @@ -58,14 +58,13 @@ import org.opensearch.ad.feature.SearchFeatureDao; import org.opensearch.ad.feature.SinglePointFeatures; import org.opensearch.ad.indices.ADIndex; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.ModelManager; import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.ADTaskState; import org.opensearch.ad.model.ADTaskType; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyResult; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.settings.ADEnabledSetting; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.stats.ADStats; @@ -102,6 +101,7 @@ import org.opensearch.timeseries.common.exception.TaskCancelledException; import org.opensearch.timeseries.common.exception.TimeSeriesException; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.DateRange; import org.opensearch.timeseries.model.Entity; import org.opensearch.timeseries.model.FeatureData; @@ -131,7 +131,7 @@ public class ADBatchTaskRunner { private final ADCircuitBreakerService adCircuitBreakerService; private final ADTaskManager adTaskManager; private final AnomalyResultBulkIndexHandler anomalyResultBulkIndexHandler; - private final AnomalyDetectionIndices anomalyDetectionIndices; + private final ADIndexManagement anomalyDetectionIndices; private final SearchFeatureDao searchFeatureDao; private final ADTaskCacheManager adTaskCacheManager; @@ -157,7 +157,7 @@ public ADBatchTaskRunner( ADCircuitBreakerService adCircuitBreakerService, FeatureManager featureManager, ADTaskManager adTaskManager, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, ADStats adStats, AnomalyResultBulkIndexHandler anomalyResultBulkIndexHandler, ADTaskCacheManager adTaskCacheManager, @@ -1325,7 +1325,7 @@ private void runNextPiece( } private void updateDetectorLevelTaskState(String detectorId, String detectorTaskId, String newState) { - AnomalyDetectorFunction function = () -> adTaskManager + ExecutorFunction function = () -> adTaskManager .updateADTask(detectorTaskId, ImmutableMap.of(STATE_FIELD, newState), ActionListener.wrap(r -> { logger.info("Updated HC detector task: {} state as: {} for detector: {}", detectorTaskId, newState, detectorId); adTaskCacheManager.updateDetectorTaskState(detectorId, detectorTaskId, newState); diff --git a/src/main/java/org/opensearch/ad/task/ADTaskManager.java b/src/main/java/org/opensearch/ad/task/ADTaskManager.java index c97f87e59..b79388148 100644 --- a/src/main/java/org/opensearch/ad/task/ADTaskManager.java +++ b/src/main/java/org/opensearch/ad/task/ADTaskManager.java @@ -19,7 +19,7 @@ import static org.opensearch.ad.constant.ADCommonMessages.HC_DETECTOR_TASK_IS_UPDATING; import static org.opensearch.ad.constant.ADCommonMessages.NO_ELIGIBLE_NODE_TO_RUN_DETECTOR; import static org.opensearch.ad.constant.ADCommonName.DETECTION_STATE_INDEX; -import static org.opensearch.ad.indices.AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN; +import static org.opensearch.ad.indices.ADIndexManagement.ALL_AD_RESULTS_INDEX_PATTERN; import static org.opensearch.ad.model.ADTask.COORDINATING_NODE_FIELD; import static org.opensearch.ad.model.ADTask.DETECTOR_ID_FIELD; import static org.opensearch.ad.model.ADTask.ERROR_FIELD; @@ -101,7 +101,7 @@ import org.opensearch.action.update.UpdateRequest; import org.opensearch.action.update.UpdateResponse; import org.opensearch.ad.cluster.HashRing; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.ADEntityTaskProfile; import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.ADTaskAction; @@ -111,7 +111,6 @@ import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; import org.opensearch.ad.model.DetectorProfile; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; import org.opensearch.ad.transport.ADBatchAnomalyResultAction; import org.opensearch.ad.transport.ADBatchAnomalyResultRequest; @@ -126,7 +125,6 @@ import org.opensearch.ad.transport.AnomalyDetectorJobResponse; import org.opensearch.ad.transport.ForwardADTaskAction; import org.opensearch.ad.transport.ForwardADTaskRequest; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; @@ -165,8 +163,10 @@ import org.opensearch.timeseries.common.exception.TaskCancelledException; import org.opensearch.timeseries.common.exception.TimeSeriesException; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.DateRange; import org.opensearch.timeseries.model.Entity; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportRequestOptions; import org.opensearch.transport.TransportService; @@ -189,7 +189,7 @@ public class ADTaskManager { private final Client client; private final ClusterService clusterService; private final NamedXContentRegistry xContentRegistry; - private final AnomalyDetectionIndices detectionIndices; + private final ADIndexManagement detectionIndices; private final DiscoveryNodeFilterer nodeFilter; private final ADTaskCacheManager adTaskCacheManager; @@ -213,7 +213,7 @@ public ADTaskManager( ClusterService clusterService, Client client, NamedXContentRegistry xContentRegistry, - AnomalyDetectionIndices detectionIndices, + ADIndexManagement detectionIndices, DiscoveryNodeFilterer nodeFilter, HashRing hashRing, ADTaskCacheManager adTaskCacheManager, @@ -737,7 +737,7 @@ public void startDetector( ActionListener listener ) { try { - if (detectionIndices.doesDetectorStateIndexExist()) { + if (detectionIndices.doesStateIndexExist()) { // If detection index exist, check if latest AD task is running getAndExecuteOnLatestDetectorLevelTask(detector.getId(), getADTaskTypes(detectionDateRange), (adTask) -> { if (!adTask.isPresent() || adTask.get().isDone()) { @@ -1077,7 +1077,7 @@ private void resetLatestDetectorTaskState( private void resetRealtimeDetectorTaskState( List runningRealtimeTasks, - AnomalyDetectorFunction function, + ExecutorFunction function, TransportService transportService, ActionListener listener ) { @@ -1115,7 +1115,7 @@ private void resetRealtimeDetectorTaskState( private void resetHistoricalDetectorTaskState( List runningHistoricalTasks, - AnomalyDetectorFunction function, + ExecutorFunction function, TransportService transportService, ActionListener listener ) { @@ -1261,7 +1261,7 @@ private boolean lastUpdateTimeOfHistoricalTaskExpired(ADTask adTask) { private void resetTaskStateAsStopped( ADTask adTask, - AnomalyDetectorFunction function, + ExecutorFunction function, TransportService transportService, ActionListener listener ) { @@ -1325,7 +1325,7 @@ private void resetEntityTasksAsStopped(String detectorTaskId) { public void cleanDetectorCache( ADTask adTask, TransportService transportService, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { String coordinatingNode = adTask.getCoordinatingNode(); @@ -1356,7 +1356,7 @@ public void cleanDetectorCache( } } - protected void cleanDetectorCache(ADTask adTask, TransportService transportService, AnomalyDetectorFunction function) { + protected void cleanDetectorCache(ADTask adTask, TransportService transportService, ExecutorFunction function) { String detectorId = adTask.getId(); String taskId = adTask.getTaskId(); cleanDetectorCache( @@ -1660,7 +1660,7 @@ private void cleanOldAdTaskDocs(IndexResponse response, ADTask adTask, ActionLis protected void deleteTaskDocs( String detectorId, SearchRequest searchRequest, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { ActionListener searchListener = ActionListener.wrap(r -> { @@ -1891,7 +1891,7 @@ public ADTaskCancellationState cancelLocalTaskByDetectorId(String detectorId, St * @param function AD function * @param listener action listener */ - public void deleteADTasks(String detectorId, AnomalyDetectorFunction function, ActionListener listener) { + public void deleteADTasks(String detectorId, ExecutorFunction function, ActionListener listener) { DeleteByQueryRequest request = new DeleteByQueryRequest(DETECTION_STATE_INDEX); BoolQueryBuilder query = new BoolQueryBuilder(); @@ -1993,7 +1993,7 @@ public void stopLatestRealtimeTask( if (error != null) { updatedFields.put(ADTask.ERROR_FIELD, error.getMessage()); } - AnomalyDetectorFunction function = () -> updateADTask(adTask.get().getTaskId(), updatedFields, ActionListener.wrap(r -> { + ExecutorFunction function = () -> updateADTask(adTask.get().getTaskId(), updatedFields, ActionListener.wrap(r -> { if (error == null) { listener.onResponse(new AnomalyDetectorJobResponse(detectorId, 0, 0, 0, RestStatus.OK)); } else { @@ -2112,7 +2112,7 @@ public void initRealtimeTaskCacheAndCleanupStaleCache( getAndExecuteOnLatestDetectorLevelTask(detectorId, REALTIME_TASK_TYPES, (adTaskOptional) -> { if (!adTaskOptional.isPresent()) { logger.debug("Can't find realtime task for detector {}, init realtime task cache directly", detectorId); - AnomalyDetectorFunction function = () -> createNewADTask( + ExecutorFunction function = () -> createNewADTask( detector, null, detector.getUser(), @@ -2163,8 +2163,8 @@ public void initRealtimeTaskCacheAndCleanupStaleCache( } } - private void recreateRealtimeTask(AnomalyDetectorFunction function, ActionListener listener) { - if (detectionIndices.doesDetectorStateIndexExist()) { + private void recreateRealtimeTask(ExecutorFunction function, ActionListener listener) { + if (detectionIndices.doesStateIndexExist()) { function.execute(); } else { // If detection index doesn't exist, create index and execute function. diff --git a/src/main/java/org/opensearch/ad/transport/AnomalyDetectorJobTransportAction.java b/src/main/java/org/opensearch/ad/transport/AnomalyDetectorJobTransportAction.java index 50b31a2d8..1f86cefbb 100644 --- a/src/main/java/org/opensearch/ad/transport/AnomalyDetectorJobTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/AnomalyDetectorJobTransportAction.java @@ -25,7 +25,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; import org.opensearch.ad.ExecuteADResultResponseRecorder; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.client.Client; @@ -47,7 +47,7 @@ public class AnomalyDetectorJobTransportAction extends HandledTransportAction listener, AnomalyDetectorFunction function) { + private void getDetectorJob(String detectorId, ActionListener listener, ExecutorFunction function) { if (clusterService.state().metadata().indices().containsKey(CommonName.JOB_INDEX)) { GetRequest request = new GetRequest(CommonName.JOB_INDEX).id(detectorId); client.get(request, ActionListener.wrap(response -> onGetAdJobResponseForWrite(response, listener, function), exception -> { @@ -203,7 +203,7 @@ private void getDetectorJob(String detectorId, ActionListener li } } - private void onGetAdJobResponseForWrite(GetResponse response, ActionListener listener, AnomalyDetectorFunction function) + private void onGetAdJobResponseForWrite(GetResponse response, ActionListener listener, ExecutorFunction function) throws IOException { if (response.isExists()) { String adJobId = response.getId(); diff --git a/src/main/java/org/opensearch/ad/transport/EntityResultTransportAction.java b/src/main/java/org/opensearch/ad/transport/EntityResultTransportAction.java index f48fddecf..021dec3b1 100644 --- a/src/main/java/org/opensearch/ad/transport/EntityResultTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/EntityResultTransportAction.java @@ -33,7 +33,7 @@ import org.opensearch.ad.caching.CacheProvider; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.indices.ADIndex; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.EntityModel; import org.opensearch.ad.ml.ModelManager; import org.opensearch.ad.ml.ModelState; @@ -86,7 +86,7 @@ public class EntityResultTransportAction extends HandledTransportAction { @@ -56,7 +56,7 @@ public class IndexAnomalyDetectorTransportAction extends HandledTransportAction< private final Client client; private final SecurityClientUtil clientUtil; private final TransportService transportService; - private final AnomalyDetectionIndices anomalyDetectionIndices; + private final ADIndexManagement anomalyDetectionIndices; private final ClusterService clusterService; private final NamedXContentRegistry xContentRegistry; private final ADTaskManager adTaskManager; @@ -72,7 +72,7 @@ public IndexAnomalyDetectorTransportAction( SecurityClientUtil clientUtil, ClusterService clusterService, Settings settings, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, NamedXContentRegistry xContentRegistry, ADTaskManager adTaskManager, SearchFeatureDao searchFeatureDao @@ -189,7 +189,7 @@ protected void adExecute( private void checkIndicesAndExecute( List indices, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { SearchRequest searchRequest = new SearchRequest() diff --git a/src/main/java/org/opensearch/ad/transport/SearchAnomalyResultTransportAction.java b/src/main/java/org/opensearch/ad/transport/SearchAnomalyResultTransportAction.java index 2f76bc7db..1043754c2 100644 --- a/src/main/java/org/opensearch/ad/transport/SearchAnomalyResultTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/SearchAnomalyResultTransportAction.java @@ -12,7 +12,7 @@ package org.opensearch.ad.transport; import static org.opensearch.ad.constant.ADCommonName.CUSTOM_RESULT_INDEX_PREFIX; -import static org.opensearch.ad.indices.AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN; +import static org.opensearch.ad.indices.ADIndexManagement.ALL_AD_RESULTS_INDEX_PATTERN; import static org.opensearch.ad.settings.AnomalyDetectorSettings.MAX_DETECTOR_UPPER_LIMIT; import java.util.ArrayList; diff --git a/src/main/java/org/opensearch/ad/transport/SearchTopAnomalyResultTransportAction.java b/src/main/java/org/opensearch/ad/transport/SearchTopAnomalyResultTransportAction.java index 4cc997391..f42c114e9 100644 --- a/src/main/java/org/opensearch/ad/transport/SearchTopAnomalyResultTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/SearchTopAnomalyResultTransportAction.java @@ -11,7 +11,7 @@ package org.opensearch.ad.transport; -import static org.opensearch.ad.indices.AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN; +import static org.opensearch.ad.indices.ADIndexManagement.ALL_AD_RESULTS_INDEX_PATTERN; import static org.opensearch.ad.settings.AnomalyDetectorSettings.TOP_ANOMALY_RESULT_TIMEOUT_IN_MILLIS; import java.time.Clock; diff --git a/src/main/java/org/opensearch/ad/transport/StopDetectorTransportAction.java b/src/main/java/org/opensearch/ad/transport/StopDetectorTransportAction.java index 7957b0364..f84d4114e 100644 --- a/src/main/java/org/opensearch/ad/transport/StopDetectorTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/StopDetectorTransportAction.java @@ -22,12 +22,12 @@ import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.inject.Inject; import org.opensearch.tasks.Task; import org.opensearch.timeseries.common.exception.InternalFailure; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; public class StopDetectorTransportAction extends HandledTransportAction { diff --git a/src/main/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportAction.java b/src/main/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportAction.java index 029af977b..ecd0ca07c 100644 --- a/src/main/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportAction.java +++ b/src/main/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportAction.java @@ -29,10 +29,9 @@ import org.opensearch.action.support.HandledTransportAction; import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.DetectorValidationIssue; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.rest.handler.ValidateAnomalyDetectorActionHandler; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.util.SecurityClientUtil; @@ -49,6 +48,7 @@ import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.tasks.Task; import org.opensearch.timeseries.common.exception.ValidationException; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.IntervalTimeConfiguration; import org.opensearch.timeseries.model.ValidationAspect; import org.opensearch.timeseries.model.ValidationIssueType; @@ -62,7 +62,7 @@ public class ValidateAnomalyDetectorTransportAction extends private final SecurityClientUtil clientUtil; private final ClusterService clusterService; private final NamedXContentRegistry xContentRegistry; - private final AnomalyDetectionIndices anomalyDetectionIndices; + private final ADIndexManagement anomalyDetectionIndices; private final SearchFeatureDao searchFeatureDao; private volatile Boolean filterByEnabled; private Clock clock; @@ -75,7 +75,7 @@ public ValidateAnomalyDetectorTransportAction( ClusterService clusterService, NamedXContentRegistry xContentRegistry, Settings settings, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, ActionFilters actionFilters, TransportService transportService, SearchFeatureDao searchFeatureDao @@ -108,7 +108,7 @@ protected void doExecute(Task task, ValidateAnomalyDetectorRequest request, Acti private void resolveUserAndExecute( User requestedUser, ActionListener listener, - AnomalyDetectorFunction function + ExecutorFunction function ) { try { // Check if user has backend roles @@ -231,7 +231,7 @@ private Map getFeatureSubIssuesFromErrorMessage(String errorMess private void checkIndicesAndExecute( List indices, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { SearchRequest searchRequest = new SearchRequest() diff --git a/src/main/java/org/opensearch/ad/transport/handler/AnomalyIndexHandler.java b/src/main/java/org/opensearch/ad/transport/handler/AnomalyIndexHandler.java index b3a6b8318..371640ad2 100644 --- a/src/main/java/org/opensearch/ad/transport/handler/AnomalyIndexHandler.java +++ b/src/main/java/org/opensearch/ad/transport/handler/AnomalyIndexHandler.java @@ -26,7 +26,7 @@ import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.util.BulkUtil; import org.opensearch.ad.util.ClientUtil; @@ -56,7 +56,7 @@ public class AnomalyIndexHandler { protected final ThreadPool threadPool; protected final BackoffPolicy savingBackoffPolicy; protected final String indexName; - protected final AnomalyDetectionIndices anomalyDetectionIndices; + protected final ADIndexManagement anomalyDetectionIndices; // whether save to a specific doc id or not. False by default. protected boolean fixedDoc; protected final ClientUtil clientUtil; @@ -80,7 +80,7 @@ public AnomalyIndexHandler( Settings settings, ThreadPool threadPool, String indexName, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, ClientUtil clientUtil, IndexUtils indexUtils, ClusterService clusterService @@ -131,9 +131,9 @@ public void index(T toSave, String detectorId, String customIndexName) { save(toSave, detectorId, customIndexName); return; } - if (!anomalyDetectionIndices.doesDefaultAnomalyResultIndexExist()) { + if (!anomalyDetectionIndices.doesDefaultResultIndexExist()) { anomalyDetectionIndices - .initDefaultAnomalyResultIndexDirectly( + .initDefaultResultIndexDirectly( ActionListener.wrap(initResponse -> onCreateIndexResponse(initResponse, toSave, detectorId), exception -> { if (ExceptionsHelper.unwrapCause(exception) instanceof ResourceAlreadyExistsException) { // It is possible the index has been created while we sending the create request diff --git a/src/main/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java b/src/main/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java index 3992c0ded..1e3aeb4da 100644 --- a/src/main/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java +++ b/src/main/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java @@ -25,7 +25,7 @@ import org.opensearch.action.bulk.BulkRequestBuilder; import org.opensearch.action.bulk.BulkResponse; import org.opensearch.action.index.IndexRequest; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyResult; import org.opensearch.ad.util.ClientUtil; import org.opensearch.ad.util.IndexUtils; @@ -41,7 +41,7 @@ public class AnomalyResultBulkIndexHandler extends AnomalyIndexHandler { private static final Logger LOG = LogManager.getLogger(AnomalyResultBulkIndexHandler.class); - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; public AnomalyResultBulkIndexHandler( Client client, @@ -50,7 +50,7 @@ public AnomalyResultBulkIndexHandler( ClientUtil clientUtil, IndexUtils indexUtils, ClusterService clusterService, - AnomalyDetectionIndices anomalyDetectionIndices + ADIndexManagement anomalyDetectionIndices ) { super(client, settings, threadPool, ANOMALY_RESULT_INDEX_ALIAS, anomalyDetectionIndices, clientUtil, indexUtils, clusterService); this.anomalyDetectionIndices = anomalyDetectionIndices; @@ -83,8 +83,8 @@ public void bulkIndexAnomalyResult(String resultIndex, List anoma bulkSaveDetectorResult(resultIndex, anomalyResults, listener); return; } - if (!anomalyDetectionIndices.doesDefaultAnomalyResultIndexExist()) { - anomalyDetectionIndices.initDefaultAnomalyResultIndexDirectly(ActionListener.wrap(response -> { + if (!anomalyDetectionIndices.doesDefaultResultIndexExist()) { + anomalyDetectionIndices.initDefaultResultIndexDirectly(ActionListener.wrap(response -> { if (response.isAcknowledged()) { bulkSaveDetectorResult(anomalyResults, listener); } else { diff --git a/src/main/java/org/opensearch/ad/transport/handler/MultiEntityResultHandler.java b/src/main/java/org/opensearch/ad/transport/handler/MultiEntityResultHandler.java index 731852763..d9d98b74a 100644 --- a/src/main/java/org/opensearch/ad/transport/handler/MultiEntityResultHandler.java +++ b/src/main/java/org/opensearch/ad/transport/handler/MultiEntityResultHandler.java @@ -17,7 +17,7 @@ import org.opensearch.ResourceAlreadyExistsException; import org.opensearch.action.ActionListener; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyResult; import org.opensearch.ad.transport.ADResultBulkAction; import org.opensearch.ad.transport.ADResultBulkRequest; @@ -52,7 +52,7 @@ public MultiEntityResultHandler( Client client, Settings settings, ThreadPool threadPool, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, ClientUtil clientUtil, IndexUtils indexUtils, ClusterService clusterService @@ -81,8 +81,8 @@ public void flush(ADResultBulkRequest currentBulkRequest, ActionListener { + if (!anomalyDetectionIndices.doesDefaultResultIndexExist()) { + anomalyDetectionIndices.initDefaultResultIndexDirectly(ActionListener.wrap(initResponse -> { if (initResponse.isAcknowledged()) { bulk(currentBulkRequest, listener); } else { diff --git a/src/main/java/org/opensearch/forecast/constant/ForecastCommonMessages.java b/src/main/java/org/opensearch/forecast/constant/ForecastCommonMessages.java index aabd4dc2e..46de0c762 100644 --- a/src/main/java/org/opensearch/forecast/constant/ForecastCommonMessages.java +++ b/src/main/java/org/opensearch/forecast/constant/ForecastCommonMessages.java @@ -46,16 +46,9 @@ public class ForecastCommonMessages { // Used for custom forecast result index // ====================================== public static String INVALID_RESULT_INDEX_PREFIX = "Result index must start with " + CUSTOM_RESULT_INDEX_PREFIX; - public static String INVALID_CHAR_IN_RESULT_INDEX_NAME = - "Result index name has invalid character. Valid characters are a-z, 0-9, -(hyphen) and _(underscore)"; // ====================================== // Task // ====================================== public static String FORECASTER_IS_RUNNING = "Forecaster is already running"; - - // ====================================== - // transport - // ====================================== - public static final String FORECAST_ID_MISSING_MSG = "forecaster ID is missing"; } diff --git a/src/main/java/org/opensearch/forecast/indices/ForecastIndex.java b/src/main/java/org/opensearch/forecast/indices/ForecastIndex.java new file mode 100644 index 000000000..8e514dd6e --- /dev/null +++ b/src/main/java/org/opensearch/forecast/indices/ForecastIndex.java @@ -0,0 +1,72 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.forecast.indices; + +import java.util.function.Supplier; + +import org.opensearch.ad.indices.ADIndexManagement; +import org.opensearch.forecast.constant.ForecastCommonName; +import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ThrowingSupplierWrapper; +import org.opensearch.timeseries.indices.TimeSeriesIndex; + +public enum ForecastIndex implements TimeSeriesIndex { + // throw RuntimeException since we don't know how to handle the case when the mapping reading throws IOException + RESULT( + ForecastCommonName.FORECAST_RESULT_INDEX_ALIAS, + true, + ThrowingSupplierWrapper.throwingSupplierWrapper(ForecastIndexManagement::getResultMappings) + ), + CONFIG(CommonName.CONFIG_INDEX, false, ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getConfigMappings)), + JOB(CommonName.JOB_INDEX, false, ThrowingSupplierWrapper.throwingSupplierWrapper(ADIndexManagement::getJobMappings)), + CHECKPOINT( + ForecastCommonName.FORECAST_CHECKPOINT_INDEX_NAME, + false, + ThrowingSupplierWrapper.throwingSupplierWrapper(ForecastIndexManagement::getCheckpointMappings) + ), + STATE( + ForecastCommonName.FORECAST_STATE_INDEX, + false, + ThrowingSupplierWrapper.throwingSupplierWrapper(ForecastIndexManagement::getStateMappings) + ); + + private final String indexName; + // whether we use an alias for the index + private final boolean alias; + private final String mapping; + + ForecastIndex(String name, boolean alias, Supplier mappingSupplier) { + this.indexName = name; + this.alias = alias; + this.mapping = mappingSupplier.get(); + } + + @Override + public String getIndexName() { + return indexName; + } + + @Override + public boolean isAlias() { + return alias; + } + + @Override + public String getMapping() { + return mapping; + } + + @Override + public boolean isJobIndex() { + return CommonName.JOB_INDEX.equals(indexName); + } +} diff --git a/src/main/java/org/opensearch/forecast/indices/ForecastIndexManagement.java b/src/main/java/org/opensearch/forecast/indices/ForecastIndexManagement.java new file mode 100644 index 000000000..adddadb00 --- /dev/null +++ b/src/main/java/org/opensearch/forecast/indices/ForecastIndexManagement.java @@ -0,0 +1,265 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.forecast.indices; + +import static org.opensearch.forecast.constant.ForecastCommonName.DUMMY_FORECAST_RESULT_ID; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_CHECKPOINT_INDEX_MAPPING_FILE; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_MAX_PRIMARY_SHARDS; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_RESULTS_INDEX_MAPPING_FILE; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_RESULT_HISTORY_RETENTION_PERIOD; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_RESULT_HISTORY_ROLLOVER_PERIOD; +import static org.opensearch.forecast.settings.ForecastSettings.FORECAST_STATE_INDEX_MAPPING_FILE; + +import java.io.IOException; +import java.util.EnumMap; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.forecast.constant.ForecastCommonName; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.timeseries.common.exception.EndRunException; +import org.opensearch.timeseries.indices.IndexManagement; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; + +public class ForecastIndexManagement extends IndexManagement { + private static final Logger logger = LogManager.getLogger(ForecastIndexManagement.class); + + // The index name pattern to query all the forecast result history indices + public static final String FORECAST_RESULT_HISTORY_INDEX_PATTERN = ""; + + // The index name pattern to query all forecast results, history and current forecast results + public static final String ALL_FORECAST_RESULTS_INDEX_PATTERN = "opensearch-forecast-results*"; + + /** + * Constructor function + * + * @param client OS client supports administrative actions + * @param clusterService OS cluster service + * @param threadPool OS thread pool + * @param settings OS cluster setting + * @param nodeFilter Used to filter eligible nodes to host forecast indices + * @param maxUpdateRunningTimes max number of retries to update index mapping and setting + * @throws IOException + */ + public ForecastIndexManagement( + Client client, + ClusterService clusterService, + ThreadPool threadPool, + Settings settings, + DiscoveryNodeFilterer nodeFilter, + int maxUpdateRunningTimes + ) + throws IOException { + super( + client, + clusterService, + threadPool, + settings, + nodeFilter, + maxUpdateRunningTimes, + ForecastIndex.class, + FORECAST_MAX_PRIMARY_SHARDS.get(settings), + FORECAST_RESULT_HISTORY_ROLLOVER_PERIOD.get(settings), + FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD.get(settings), + FORECAST_RESULT_HISTORY_RETENTION_PERIOD.get(settings), + ForecastIndex.RESULT.getMapping() + ); + this.indexStates = new EnumMap(ForecastIndex.class); + + this.clusterService + .getClusterSettings() + .addSettingsUpdateConsumer(FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD, it -> historyMaxDocs = it); + + this.clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_RESULT_HISTORY_ROLLOVER_PERIOD, it -> { + historyRolloverPeriod = it; + rescheduleRollover(); + }); + this.clusterService + .getClusterSettings() + .addSettingsUpdateConsumer(FORECAST_RESULT_HISTORY_RETENTION_PERIOD, it -> { historyRetentionPeriod = it; }); + + this.clusterService.getClusterSettings().addSettingsUpdateConsumer(FORECAST_MAX_PRIMARY_SHARDS, it -> maxPrimaryShards = it); + + this.updateRunningTimes = 0; + } + + /** + * Get forecast result index mapping json content. + * + * @return forecast result index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getResultMappings() throws IOException { + return getMappings(FORECAST_RESULTS_INDEX_MAPPING_FILE); + } + + /** + * Get forecaster state index mapping json content. + * + * @return forecaster state index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getStateMappings() throws IOException { + String forecastStateMappings = getMappings(FORECAST_STATE_INDEX_MAPPING_FILE); + String forecasterIndexMappings = getConfigMappings(); + forecasterIndexMappings = forecasterIndexMappings + .substring(forecasterIndexMappings.indexOf("\"properties\""), forecasterIndexMappings.lastIndexOf("}")); + return forecastStateMappings.replace("FORECASTER_INDEX_MAPPING_PLACE_HOLDER", forecasterIndexMappings); + } + + /** + * Get checkpoint index mapping json content. + * + * @return checkpoint index mapping + * @throws IOException IOException if mapping file can't be read correctly + */ + public static String getCheckpointMappings() throws IOException { + return getMappings(FORECAST_CHECKPOINT_INDEX_MAPPING_FILE); + } + + /** + * default forecaster result index exist or not. + * + * @return true if default forecaster result index exists + */ + @Override + public boolean doesDefaultResultIndexExist() { + return doesAliasExist(ForecastCommonName.FORECAST_RESULT_INDEX_ALIAS); + } + + /** + * Forecast state index exist or not. + * + * @return true if forecast state index exists + */ + @Override + public boolean doesStateIndexExist() { + return doesIndexExist(ForecastCommonName.FORECAST_STATE_INDEX); + } + + /** + * Checkpoint index exist or not. + * + * @return true if checkpoint index exists + */ + @Override + public boolean doesCheckpointIndexExist() { + return doesIndexExist(ForecastCommonName.FORECAST_CHECKPOINT_INDEX_NAME); + } + + /** + * Create the state index. + * + * @param actionListener action called after create index + */ + public void initStateIndex(ActionListener actionListener) { + try { + CreateIndexRequest request = new CreateIndexRequest(ForecastCommonName.FORECAST_STATE_INDEX) + .mapping(getStateMappings(), XContentType.JSON) + .settings(settings); + adminClient.indices().create(request, markMappingUpToDate(ForecastIndex.STATE, actionListener)); + } catch (IOException e) { + logger.error("Fail to init AD detection state index", e); + actionListener.onFailure(e); + } + } + + /** + * Create the checkpoint index. + * + * @param actionListener action called after create index + * @throws EndRunException EndRunException due to failure to get mapping + */ + @Override + public void initCheckpointIndex(ActionListener actionListener) { + String mapping; + try { + mapping = getCheckpointMappings(); + } catch (IOException e) { + throw new EndRunException("", "Cannot find checkpoint mapping file", true); + } + CreateIndexRequest request = new CreateIndexRequest(ForecastCommonName.FORECAST_CHECKPOINT_INDEX_NAME) + .mapping(mapping, XContentType.JSON); + choosePrimaryShards(request, true); + adminClient.indices().create(request, markMappingUpToDate(ForecastIndex.CHECKPOINT, actionListener)); + } + + @Override + protected void rolloverAndDeleteHistoryIndex() { + rolloverAndDeleteHistoryIndex( + ForecastCommonName.FORECAST_RESULT_INDEX_ALIAS, + ALL_FORECAST_RESULTS_INDEX_PATTERN, + FORECAST_RESULT_HISTORY_INDEX_PATTERN, + ForecastIndex.RESULT + ); + } + + /** + * Create config index directly. + * + * @param actionListener action called after create index + * @throws IOException IOException from {@link IndexManagement#getConfigMappings} + */ + @Override + public void initConfigIndex(ActionListener actionListener) throws IOException { + super.initConfigIndex(markMappingUpToDate(ForecastIndex.CONFIG, actionListener)); + } + + /** + * Create config index. + * + * @param actionListener action called after create index + */ + @Override + public void initJobIndex(ActionListener actionListener) { + super.initJobIndex(markMappingUpToDate(ForecastIndex.JOB, actionListener)); + } + + @Override + protected IndexRequest createDummyIndexRequest(String resultIndex) throws IOException { + // TODO: add real support when committing ForecastResult class + return new IndexRequest(resultIndex).id(DUMMY_FORECAST_RESULT_ID).source(XContentType.JSON, "field", "value"); + } + + @Override + protected DeleteRequest createDummyDeleteRequest(String resultIndex) throws IOException { + return new DeleteRequest(resultIndex).id(DUMMY_FORECAST_RESULT_ID); + } + + @Override + public void initDefaultResultIndexDirectly(ActionListener actionListener) { + initResultIndexDirectly( + FORECAST_RESULT_HISTORY_INDEX_PATTERN, + ForecastIndex.RESULT.getIndexName(), + false, + FORECAST_RESULT_HISTORY_INDEX_PATTERN, + ForecastIndex.RESULT, + actionListener + ); + } + + @Override + public void initCustomResultIndexDirectly(String resultIndex, ActionListener actionListener) { + // throws IOException { + initResultIndexDirectly(resultIndex, null, false, FORECAST_RESULT_HISTORY_INDEX_PATTERN, ForecastIndex.RESULT, actionListener); + } +} diff --git a/src/main/java/org/opensearch/timeseries/common/exception/NotSerializedExceptionName.java b/src/main/java/org/opensearch/timeseries/common/exception/NotSerializedExceptionName.java index 0cd2212be..70e1f5e4d 100644 --- a/src/main/java/org/opensearch/timeseries/common/exception/NotSerializedExceptionName.java +++ b/src/main/java/org/opensearch/timeseries/common/exception/NotSerializedExceptionName.java @@ -11,12 +11,13 @@ package org.opensearch.timeseries.common.exception; +import static org.opensearch.OpenSearchException.getExceptionName; + import java.util.Optional; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.BaseExceptionsHelper; import org.opensearch.common.io.stream.NotSerializableExceptionWrapper; /** @@ -29,16 +30,16 @@ */ public enum NotSerializedExceptionName { - RESOURCE_NOT_FOUND_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new ResourceNotFoundException("", ""))), - LIMIT_EXCEEDED_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new LimitExceededException("", "", false))), - END_RUN_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new EndRunException("", "", false))), - TIME_SERIES_DETECTION_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new TimeSeriesException("", ""))), - INTERNAL_FAILURE_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new InternalFailure("", ""))), - CLIENT_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new ClientException("", ""))), - CANCELLATION_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new TaskCancelledException("", ""))), - DUPLICATE_TASK_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new DuplicateTaskException(""))), - VERSION_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new VersionException(""))), - VALIDATION_EXCEPTION_NAME_UNDERSCORE(BaseExceptionsHelper.getExceptionName(new ValidationException("", null, null))); + RESOURCE_NOT_FOUND_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new ResourceNotFoundException("", ""))), + LIMIT_EXCEEDED_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new LimitExceededException("", "", false))), + END_RUN_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new EndRunException("", "", false))), + TIME_SERIES_DETECTION_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new TimeSeriesException("", ""))), + INTERNAL_FAILURE_NAME_UNDERSCORE(getExceptionName(new InternalFailure("", ""))), + CLIENT_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new ClientException("", ""))), + CANCELLATION_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new TaskCancelledException("", ""))), + DUPLICATE_TASK_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new DuplicateTaskException(""))), + VERSION_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new VersionException(""))), + VALIDATION_EXCEPTION_NAME_UNDERSCORE(getExceptionName(new ValidationException("", null, null))); private static final Logger LOG = LogManager.getLogger(NotSerializedExceptionName.class); private final String name; diff --git a/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java b/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java index 803c80474..393248237 100644 --- a/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java +++ b/src/main/java/org/opensearch/timeseries/constant/CommonMessages.java @@ -83,6 +83,6 @@ public static String getTooManyCategoricalFieldErr(int limit) { // rate limiting worker // ====================================== public static final String BUG_RESPONSE = "We might have bugs."; - public static final String MEMORY_LIMIT_EXCEEDED_ERR_MSG = "AD models memory usage exceeds our limit."; + public static final String MEMORY_LIMIT_EXCEEDED_ERR_MSG = "Models memory usage exceeds our limit."; } diff --git a/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorFunction.java b/src/main/java/org/opensearch/timeseries/function/ExecutorFunction.java similarity index 85% rename from src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorFunction.java rename to src/main/java/org/opensearch/timeseries/function/ExecutorFunction.java index 929120561..90cd93cfb 100644 --- a/src/main/java/org/opensearch/ad/rest/handler/AnomalyDetectorFunction.java +++ b/src/main/java/org/opensearch/timeseries/function/ExecutorFunction.java @@ -9,10 +9,10 @@ * GitHub history for details. */ -package org.opensearch.ad.rest.handler; +package org.opensearch.timeseries.function; @FunctionalInterface -public interface AnomalyDetectorFunction { +public interface ExecutorFunction { /** * Performs this operation. diff --git a/src/main/java/org/opensearch/ad/util/ThrowingConsumer.java b/src/main/java/org/opensearch/timeseries/function/ThrowingConsumer.java similarity index 92% rename from src/main/java/org/opensearch/ad/util/ThrowingConsumer.java rename to src/main/java/org/opensearch/timeseries/function/ThrowingConsumer.java index d3f981552..8a7210f01 100644 --- a/src/main/java/org/opensearch/ad/util/ThrowingConsumer.java +++ b/src/main/java/org/opensearch/timeseries/function/ThrowingConsumer.java @@ -9,7 +9,7 @@ * GitHub history for details. */ -package org.opensearch.ad.util; +package org.opensearch.timeseries.function; /** * A consumer that can throw checked exception diff --git a/src/main/java/org/opensearch/ad/util/ThrowingSupplier.java b/src/main/java/org/opensearch/timeseries/function/ThrowingSupplier.java similarity index 92% rename from src/main/java/org/opensearch/ad/util/ThrowingSupplier.java rename to src/main/java/org/opensearch/timeseries/function/ThrowingSupplier.java index 9810ffcaf..a56f513a8 100644 --- a/src/main/java/org/opensearch/ad/util/ThrowingSupplier.java +++ b/src/main/java/org/opensearch/timeseries/function/ThrowingSupplier.java @@ -9,7 +9,7 @@ * GitHub history for details. */ -package org.opensearch.ad.util; +package org.opensearch.timeseries.function; /** * A supplier that can throw checked exception diff --git a/src/main/java/org/opensearch/ad/util/ThrowingSupplierWrapper.java b/src/main/java/org/opensearch/timeseries/function/ThrowingSupplierWrapper.java similarity index 96% rename from src/main/java/org/opensearch/ad/util/ThrowingSupplierWrapper.java rename to src/main/java/org/opensearch/timeseries/function/ThrowingSupplierWrapper.java index 42ceb1526..c57b11d33 100644 --- a/src/main/java/org/opensearch/ad/util/ThrowingSupplierWrapper.java +++ b/src/main/java/org/opensearch/timeseries/function/ThrowingSupplierWrapper.java @@ -9,7 +9,7 @@ * GitHub history for details. */ -package org.opensearch.ad.util; +package org.opensearch.timeseries.function; import java.util.function.Supplier; diff --git a/src/main/java/org/opensearch/ad/indices/AnomalyDetectionIndices.java b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java similarity index 59% rename from src/main/java/org/opensearch/ad/indices/AnomalyDetectionIndices.java rename to src/main/java/org/opensearch/timeseries/indices/IndexManagement.java index 4f27f1b20..7040f2ff8 100644 --- a/src/main/java/org/opensearch/ad/indices/AnomalyDetectionIndices.java +++ b/src/main/java/org/opensearch/timeseries/indices/IndexManagement.java @@ -9,19 +9,9 @@ * GitHub history for details. */ -package org.opensearch.ad.indices; - -import static org.opensearch.ad.constant.ADCommonName.DUMMY_AD_RESULT_ID; -import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD; -import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD; -import static org.opensearch.ad.settings.AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD; -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.MAX_PRIMARY_SHARDS; +package org.opensearch.timeseries.indices; + import static org.opensearch.timeseries.constant.CommonMessages.CAN_NOT_FIND_RESULT_INDEX; -import static org.opensearch.timeseries.settings.TimeSeriesSettings.INDEX_MAPPING_FILE; -import static org.opensearch.timeseries.settings.TimeSeriesSettings.JOBS_INDEX_MAPPING_FILE; import java.io.IOException; import java.net.URL; @@ -58,11 +48,7 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.GroupedActionListener; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.constant.CommonValue; -import org.opensearch.ad.model.AnomalyResult; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; -import org.opensearch.ad.util.DiscoveryNodeFilterer; +import org.opensearch.ad.indices.ADIndex; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.cluster.LocalNodeClusterManagerListener; @@ -78,8 +64,6 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.commons.InjectSecurity; import org.opensearch.core.xcontent.NamedXContentRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.xcontent.XContentParser.Token; import org.opensearch.index.IndexNotFoundException; @@ -88,270 +72,437 @@ import org.opensearch.timeseries.common.exception.EndRunException; import org.opensearch.timeseries.constant.CommonMessages; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.constant.CommonValue; +import org.opensearch.timeseries.function.ExecutorFunction; +import org.opensearch.timeseries.indices.IndexManagement.IndexState; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import com.google.common.base.Charsets; import com.google.common.io.Resources; -/** - * This class provides utility methods for various anomaly detection indices. - */ -public class AnomalyDetectionIndices implements LocalNodeClusterManagerListener { - private static final Logger logger = LogManager.getLogger(AnomalyDetectionIndices.class); - - // The index name pattern to query all the AD result history indices - public static final String AD_RESULT_HISTORY_INDEX_PATTERN = "<.opendistro-anomaly-results-history-{now/d}-1>"; - - // 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*"; +public abstract class IndexManagement & TimeSeriesIndex> implements LocalNodeClusterManagerListener { + private static final Logger logger = LogManager.getLogger(IndexManagement.class); // minimum shards of the job index public static int minJobIndexReplicas = 1; // maximum shards of the job index public static int maxJobIndexReplicas = 20; - // package private for testing - static final String META = "_meta"; - private static final String SCHEMA_VERSION = "schema_version"; - - private ClusterService clusterService; - private final Client client; - private final AdminClient adminClient; - private final ThreadPool threadPool; - - private volatile TimeValue historyRolloverPeriod; - private volatile Long historyMaxDocs; - private volatile TimeValue historyRetentionPeriod; - - private Scheduler.Cancellable scheduledRollover = null; + public static final String META = "_meta"; + public static final String SCHEMA_VERSION = "schema_version"; + + protected ClusterService clusterService; + protected final Client client; + protected final AdminClient adminClient; + protected final ThreadPool threadPool; + protected DiscoveryNodeFilterer nodeFilter; + // index settings + protected final Settings settings; + // don't retry updating endlessly. Can be annoying if there are too many exception logs. + protected final int maxUpdateRunningTimes; - private DiscoveryNodeFilterer nodeFilter; - private int maxPrimaryShards; - // keep track of whether the mapping version is up-to-date - private EnumMap indexStates; // whether all index have the correct mappings - private boolean allMappingUpdated; + protected boolean allMappingUpdated; // whether all index settings are updated - private boolean allSettingUpdated; + protected boolean allSettingUpdated; // we only want one update at a time - private final AtomicBoolean updateRunning; - // don't retry updating endlessly. Can be annoying if there are too many exception logs. - private final int maxUpdateRunningTimes; + protected final AtomicBoolean updateRunning; // the number of times updates run - private int updateRunningTimes; - // AD index settings - private final Settings settings; - + protected int updateRunningTimes; + private final Class indexType; + // keep track of whether the mapping version is up-to-date + protected EnumMap indexStates; + protected int maxPrimaryShards; + private Scheduler.Cancellable scheduledRollover = null; + protected volatile TimeValue historyRolloverPeriod; + protected volatile Long historyMaxDocs; + protected volatile TimeValue historyRetentionPeriod; // result index mapping to valida custom index - private Map AD_RESULT_FIELD_CONFIGS; + private Map RESULT_FIELD_CONFIGS; + private String resultMapping; - class IndexState { + protected class IndexState { // keep track of whether the mapping version is up-to-date - private Boolean mappingUpToDate; + public Boolean mappingUpToDate; // keep track of whether the setting needs to change - private Boolean settingUpToDate; + public Boolean settingUpToDate; // record schema version reading from the mapping file - private Integer schemaVersion; + public Integer schemaVersion; - IndexState(ADIndex index) { + public IndexState(String mappingFile) { this.mappingUpToDate = false; - settingUpToDate = false; - this.schemaVersion = parseSchemaVersion(index.getMapping()); + this.settingUpToDate = false; + this.schemaVersion = IndexManagement.parseSchemaVersion(mappingFile); } } - /** - * Constructor function - * - * @param client ES client supports administrative actions - * @param clusterService ES cluster service - * @param threadPool ES thread pool - * @param settings ES cluster setting - * @param nodeFilter Used to filter eligible nodes to host AD indices - * @param maxUpdateRunningTimes max number of retries to update index mapping and setting - */ - public AnomalyDetectionIndices( + protected IndexManagement( Client client, ClusterService clusterService, ThreadPool threadPool, Settings settings, DiscoveryNodeFilterer nodeFilter, - int maxUpdateRunningTimes - ) { + int maxUpdateRunningTimes, + Class indexType, + int maxPrimaryShards, + TimeValue historyRolloverPeriod, + Long historyMaxDocs, + TimeValue historyRetentionPeriod, + String resultMapping + ) + throws IOException { this.client = client; this.adminClient = client.admin(); this.clusterService = clusterService; this.threadPool = threadPool; this.clusterService.addLocalNodeClusterManagerListener(this); - this.historyRolloverPeriod = AD_RESULT_HISTORY_ROLLOVER_PERIOD.get(settings); - this.historyMaxDocs = AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD.get(settings); - this.historyRetentionPeriod = AD_RESULT_HISTORY_RETENTION_PERIOD.get(settings); - this.maxPrimaryShards = MAX_PRIMARY_SHARDS.get(settings); - this.nodeFilter = nodeFilter; - - this.indexStates = new EnumMap(ADIndex.class); + this.settings = Settings.builder().put("index.hidden", true).build(); + this.maxUpdateRunningTimes = maxUpdateRunningTimes; + this.indexType = indexType; + this.maxPrimaryShards = maxPrimaryShards; + this.historyRolloverPeriod = historyRolloverPeriod; + this.historyMaxDocs = historyMaxDocs; + this.historyRetentionPeriod = historyRetentionPeriod; this.allMappingUpdated = false; this.allSettingUpdated = false; this.updateRunning = new AtomicBoolean(false); + this.updateRunningTimes = 0; + this.resultMapping = resultMapping; + } - this.clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, it -> historyMaxDocs = it); + /** + * Alias exists or not + * @param alias Alias name + * @return true if the alias exists + */ + public boolean doesAliasExist(String alias) { + return clusterService.state().metadata().hasAlias(alias); + } - this.clusterService.getClusterSettings().addSettingsUpdateConsumer(AD_RESULT_HISTORY_ROLLOVER_PERIOD, it -> { - historyRolloverPeriod = it; - rescheduleRollover(); - }); - this.clusterService - .getClusterSettings() - .addSettingsUpdateConsumer(AD_RESULT_HISTORY_RETENTION_PERIOD, it -> { historyRetentionPeriod = it; }); + public static Integer parseSchemaVersion(String mapping) { + try { + XContentParser xcp = XContentType.JSON + .xContent() + .createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, mapping); - this.clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_PRIMARY_SHARDS, it -> maxPrimaryShards = it); + while (!xcp.isClosed()) { + Token token = xcp.currentToken(); + if (token != null && token != XContentParser.Token.END_OBJECT && token != XContentParser.Token.START_OBJECT) { + if (xcp.currentName() != IndexManagement.META) { + xcp.nextToken(); + xcp.skipChildren(); + } else { + while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { + if (xcp.currentName().equals(IndexManagement.SCHEMA_VERSION)) { - this.settings = Settings.builder().put("index.hidden", true).build(); + Integer version = xcp.intValue(); + if (version < 0) { + version = CommonValue.NO_SCHEMA_VERSION; + } + return version; + } else { + xcp.nextToken(); + } + } - this.maxUpdateRunningTimes = maxUpdateRunningTimes; - this.updateRunningTimes = 0; + } + } + xcp.nextToken(); + } + return CommonValue.NO_SCHEMA_VERSION; + } catch (Exception e) { + // since this method is called in the constructor that is called by TimeSeriesAnalyticsPlugin.createComponents, + // we cannot throw checked exception + throw new RuntimeException(e); + } + } - this.AD_RESULT_FIELD_CONFIGS = null; + protected static Integer getIntegerSetting(GetSettingsResponse settingsResponse, String settingKey) { + Integer value = null; + for (Settings settings : settingsResponse.getIndexToSettings().values()) { + value = settings.getAsInt(settingKey, null); + if (value != null) { + break; + } + } + return value; } - private void initResultMapping() throws IOException { - if (AD_RESULT_FIELD_CONFIGS != null) { - // we have already initiated the field + protected static String getStringSetting(GetSettingsResponse settingsResponse, String settingKey) { + String value = null; + for (Settings settings : settingsResponse.getIndexToSettings().values()) { + value = settings.get(settingKey, null); + if (value != null) { + break; + } + } + return value; + } + + public boolean doesIndexExist(String indexName) { + return clusterService.state().metadata().hasIndex(indexName); + } + + protected static String getMappings(String mappingFileRelativePath) throws IOException { + URL url = IndexManagement.class.getClassLoader().getResource(mappingFileRelativePath); + return Resources.toString(url, Charsets.UTF_8); + } + + protected void choosePrimaryShards(CreateIndexRequest request, boolean hiddenIndex) { + request + .settings( + Settings + .builder() + // put 1 primary shards per hot node if possible + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, getNumberOfPrimaryShards()) + // 1 replica for better search performance and fail-over + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put("index.hidden", hiddenIndex) + ); + } + + protected void deleteOldHistoryIndices(String indexPattern, TimeValue historyRetentionPeriod) { + Set candidates = new HashSet(); + + ClusterStateRequest clusterStateRequest = new ClusterStateRequest() + .clear() + .indices(indexPattern) + .metadata(true) + .local(true) + .indicesOptions(IndicesOptions.strictExpand()); + + adminClient.cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { + String latestToDelete = null; + long latest = Long.MIN_VALUE; + for (IndexMetadata indexMetaData : clusterStateResponse.getState().metadata().indices().values()) { + long creationTime = indexMetaData.getCreationDate(); + if ((Instant.now().toEpochMilli() - creationTime) > historyRetentionPeriod.millis()) { + String indexName = indexMetaData.getIndex().getName(); + candidates.add(indexName); + if (latest < creationTime) { + latest = creationTime; + latestToDelete = indexName; + } + } + } + if (candidates.size() > 1) { + // delete all indices except the last one because the last one may contain docs newer than the retention period + candidates.remove(latestToDelete); + String[] toDelete = candidates.toArray(Strings.EMPTY_ARRAY); + DeleteIndexRequest deleteIndexRequest = new DeleteIndexRequest(toDelete); + adminClient.indices().delete(deleteIndexRequest, ActionListener.wrap(deleteIndexResponse -> { + if (!deleteIndexResponse.isAcknowledged()) { + logger.error("Could not delete one or more result indices: {}. Retrying one by one.", Arrays.toString(toDelete)); + deleteIndexIteration(toDelete); + } else { + logger.info("Succeeded in deleting expired result indices: {}.", Arrays.toString(toDelete)); + } + }, exception -> { + logger.error("Failed to delete expired result indices: {}.", Arrays.toString(toDelete)); + deleteIndexIteration(toDelete); + })); + } + }, exception -> { logger.error("Fail to delete result indices", exception); })); + } + + protected void deleteIndexIteration(String[] toDelete) { + for (String index : toDelete) { + DeleteIndexRequest singleDeleteRequest = new DeleteIndexRequest(index); + adminClient.indices().delete(singleDeleteRequest, ActionListener.wrap(singleDeleteResponse -> { + if (!singleDeleteResponse.isAcknowledged()) { + logger.error("Retrying deleting {} does not succeed.", index); + } + }, exception -> { + if (exception instanceof IndexNotFoundException) { + logger.info("{} was already deleted.", index); + } else { + logger.error(new ParameterizedMessage("Retrying deleting {} does not succeed.", index), exception); + } + })); + } + } + + @SuppressWarnings("unchecked") + protected void shouldUpdateConcreteIndex(String concreteIndex, Integer newVersion, ActionListener thenDo) { + IndexMetadata indexMeataData = clusterService.state().getMetadata().indices().get(concreteIndex); + if (indexMeataData == null) { + thenDo.onResponse(Boolean.FALSE); return; } - String resultMapping = getAnomalyResultMappings(); + Integer oldVersion = CommonValue.NO_SCHEMA_VERSION; - Map asMap = XContentHelper.convertToMap(new BytesArray(resultMapping), false, XContentType.JSON).v2(); - Object properties = asMap.get(CommonName.PROPERTIES); - if (properties instanceof Map) { - AD_RESULT_FIELD_CONFIGS = (Map) properties; - } else { - logger.error("Fail to read result mapping file."); + Map indexMapping = indexMeataData.mapping().getSourceAsMap(); + Object meta = indexMapping.get(IndexManagement.META); + if (meta != null && meta instanceof Map) { + Map metaMapping = (Map) meta; + Object schemaVersion = metaMapping.get(org.opensearch.timeseries.constant.CommonName.SCHEMA_VERSION_FIELD); + if (schemaVersion instanceof Integer) { + oldVersion = (Integer) schemaVersion; + } } + thenDo.onResponse(newVersion > oldVersion); } - /** - * Get anomaly detector index mapping json content. - * - * @return anomaly detector index mapping - * @throws IOException IOException if mapping file can't be read correctly - */ - public static String getAnomalyDetectorMappings() throws IOException { - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(INDEX_MAPPING_FILE); - return Resources.toString(url, Charsets.UTF_8); + protected void updateJobIndexSettingIfNecessary(String indexName, IndexState jobIndexState, ActionListener listener) { + GetSettingsRequest getSettingsRequest = new GetSettingsRequest() + .indices(indexName) + .names( + new String[] { + IndexMetadata.SETTING_NUMBER_OF_SHARDS, + IndexMetadata.SETTING_NUMBER_OF_REPLICAS, + IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS } + ); + client.execute(GetSettingsAction.INSTANCE, getSettingsRequest, ActionListener.wrap(settingResponse -> { + // auto expand setting is a range string like "1-all" + String autoExpandReplica = getStringSetting(settingResponse, IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS); + // if the auto expand setting is already there, return immediately + if (autoExpandReplica != null) { + jobIndexState.settingUpToDate = true; + logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", indexName)); + listener.onResponse(null); + return; + } + Integer primaryShardsNumber = getIntegerSetting(settingResponse, IndexMetadata.SETTING_NUMBER_OF_SHARDS); + Integer replicaNumber = getIntegerSetting(settingResponse, IndexMetadata.SETTING_NUMBER_OF_REPLICAS); + if (primaryShardsNumber == null || replicaNumber == null) { + logger + .error( + new ParameterizedMessage( + "Fail to find job index's primary or replica shard number: primary [{}], replica [{}]", + primaryShardsNumber, + replicaNumber + ) + ); + // don't throw exception as we don't know how to handle it and retry next time + listener.onResponse(null); + return; + } + // at least minJobIndexReplicas + // at most maxJobIndexReplicas / primaryShardsNumber replicas. + // For example, if we have 2 primary shards, since the max number of shards are maxJobIndexReplicas (20), + // we will use 20 / 2 = 10 replicas as the upper bound of replica. + int maxExpectedReplicas = Math + .max(IndexManagement.maxJobIndexReplicas / primaryShardsNumber, IndexManagement.minJobIndexReplicas); + Settings updatedSettings = Settings + .builder() + .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, IndexManagement.minJobIndexReplicas + "-" + maxExpectedReplicas) + .build(); + final UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(indexName).settings(updatedSettings); + client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> { + jobIndexState.settingUpToDate = true; + logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", indexName)); + listener.onResponse(null); + }, listener::onFailure)); + }, e -> { + if (e instanceof IndexNotFoundException) { + // new index will be created with auto expand replica setting + jobIndexState.settingUpToDate = true; + logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", indexName)); + listener.onResponse(null); + } else { + listener.onFailure(e); + } + })); } /** - * Get anomaly result index mapping json content. + * Create config index if not exist. * - * @return anomaly result index mapping - * @throws IOException IOException if mapping file can't be read correctly + * @param actionListener action called after create index + * @throws IOException IOException from {@link IndexManagement#getConfigMappings} */ - public static String getAnomalyResultMappings() throws IOException { - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(ANOMALY_RESULTS_INDEX_MAPPING_FILE); - return Resources.toString(url, Charsets.UTF_8); + public void initConfigIndexIfAbsent(ActionListener actionListener) throws IOException { + if (!doesConfigIndexExist()) { + initConfigIndex(actionListener); + } } /** - * Get anomaly detector job index mapping json content. + * Create config index directly. * - * @return anomaly detector job index mapping - * @throws IOException IOException if mapping file can't be read correctly + * @param actionListener action called after create index + * @throws IOException IOException from {@link IndexManagement#getConfigMappings} */ - public static String getAnomalyDetectorJobMappings() throws IOException { - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(JOBS_INDEX_MAPPING_FILE); - return Resources.toString(url, Charsets.UTF_8); + public void initConfigIndex(ActionListener actionListener) throws IOException { + CreateIndexRequest request = new CreateIndexRequest(CommonName.CONFIG_INDEX) + .mapping(getConfigMappings(), XContentType.JSON) + .settings(settings); + adminClient.indices().create(request, actionListener); } /** - * Get anomaly detector state index mapping json content. + * Config index exist or not. * - * @return anomaly detector state index mapping - * @throws IOException IOException if mapping file can't be read correctly + * @return true if config index exists */ - public static String getDetectionStateMappings() throws IOException { - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE); - String detectionStateMappings = Resources.toString(url, Charsets.UTF_8); - String detectorIndexMappings = AnomalyDetectionIndices.getAnomalyDetectorMappings(); - detectorIndexMappings = detectorIndexMappings - .substring(detectorIndexMappings.indexOf("\"properties\""), detectorIndexMappings.lastIndexOf("}")); - return detectionStateMappings.replace("DETECTOR_INDEX_MAPPING_PLACE_HOLDER", detectorIndexMappings); + public boolean doesConfigIndexExist() { + return doesIndexExist(CommonName.CONFIG_INDEX); } /** - * Get checkpoint index mapping json content. + * Job index exist or not. * - * @return checkpoint index mapping - * @throws IOException IOException if mapping file can't be read correctly + * @return true if anomaly detector job index exists */ - public static String getCheckpointMappings() throws IOException { - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(CHECKPOINT_INDEX_MAPPING_FILE); - return Resources.toString(url, Charsets.UTF_8); + public boolean doesJobIndexExist() { + return doesIndexExist(CommonName.JOB_INDEX); } /** - * Anomaly detector index exist or not. + * Get config index mapping in json format. * - * @return true if anomaly detector index exists + * @return config index mapping + * @throws IOException IOException if mapping file can't be read correctly */ - public boolean doesAnomalyDetectorIndexExist() { - return clusterService.state().getRoutingTable().hasIndex(CommonName.CONFIG_INDEX); + public static String getConfigMappings() throws IOException { + return getMappings(TimeSeriesSettings.CONFIG_INDEX_MAPPING_FILE); } /** - * Anomaly detector job index exist or not. + * Get job index mapping in json format. * - * @return true if anomaly detector job index exists + * @return job index mapping + * @throws IOException IOException if mapping file can't be read correctly */ - public boolean doesAnomalyDetectorJobIndexExist() { - return clusterService.state().getRoutingTable().hasIndex(CommonName.JOB_INDEX); + public static String getJobMappings() throws IOException { + return getMappings(TimeSeriesSettings.JOBS_INDEX_MAPPING_FILE); } /** - * anomaly result index exist or not. + * Createjob index. * - * @return true if anomaly result index exists + * @param actionListener action called after create index */ - public boolean doesDefaultAnomalyResultIndexExist() { - return clusterService.state().metadata().hasAlias(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS); - } - - public boolean doesIndexExist(String indexName) { - return clusterService.state().metadata().hasIndex(indexName); - } - - public void initCustomResultIndexAndExecute(String resultIndex, AnomalyDetectorFunction function, ActionListener listener) { + public void initJobIndex(ActionListener actionListener) { try { - if (!doesIndexExist(resultIndex)) { - initCustomAnomalyResultIndexDirectly(resultIndex, ActionListener.wrap(response -> { - if (response.isAcknowledged()) { - logger.info("Successfully created anomaly detector result index {}", resultIndex); - validateCustomResultIndexAndExecute(resultIndex, function, listener); - } else { - String error = "Creating anomaly detector result index with mappings call not acknowledged: " + resultIndex; - logger.error(error); - listener.onFailure(new EndRunException(error, true)); - } - }, exception -> { - if (ExceptionsHelper.unwrapCause(exception) instanceof ResourceAlreadyExistsException) { - // It is possible the index has been created while we sending the create request - validateCustomResultIndexAndExecute(resultIndex, function, listener); - } else { - logger.error("Failed to create anomaly detector result index " + resultIndex, exception); - listener.onFailure(exception); - } - })); - } else { - validateCustomResultIndexAndExecute(resultIndex, function, listener); - } - } catch (Exception e) { - logger.error("Failed to create custom result index " + resultIndex, e); - listener.onFailure(e); + CreateIndexRequest request = new CreateIndexRequest(CommonName.JOB_INDEX).mapping(getJobMappings(), XContentType.JSON); + request + .settings( + Settings + .builder() + // AD job index is small. 1 primary shard is enough + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + // Job scheduler puts both primary and replica shards in the + // hash ring. Auto-expand the number of replicas based on the + // number of data nodes (up to 20) in the cluster so that each node can + // become a coordinating node. This is useful when customers + // scale out their cluster so that we can do adaptive scaling + // accordingly. + // At least 1 replica for fail-over. + .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, minJobIndexReplicas + "-" + maxJobIndexReplicas) + .put("index.hidden", true) + ); + adminClient.indices().create(request, actionListener); + } catch (IOException e) { + logger.error("Fail to init AD job index", e); + actionListener.onFailure(e); } } - public void validateCustomResultIndexAndExecute(String resultIndex, AnomalyDetectorFunction function, ActionListener listener) { + public void validateCustomResultIndexAndExecute(String resultIndex, ExecutorFunction function, ActionListener listener) { try { if (!isValidResultIndexMapping(resultIndex)) { logger.warn("Can't create detector with custom result index {} as its mapping is invalid", resultIndex); @@ -359,37 +510,260 @@ public void validateCustomResultIndexAndExecute(String resultIndex, AnomalyD return; } - AnomalyResult dummyResult = AnomalyResult.getDummyResult(); - IndexRequest indexRequest = new IndexRequest(resultIndex) - .id(DUMMY_AD_RESULT_ID) - .source(dummyResult.toXContent(XContentBuilder.builder(XContentType.JSON.xContent()), ToXContent.EMPTY_PARAMS)); + IndexRequest indexRequest = createDummyIndexRequest(resultIndex); + // User may have no write permission on custom result index. Talked with security plugin team, seems no easy way to verify - // if user has write permission. So just tried to write and delete a dummy anomaly result to verify. + // if user has write permission. So just tried to write and delete a dummy forecast result to verify. client.index(indexRequest, ActionListener.wrap(response -> { - logger.debug("Successfully wrote dummy AD result to result index {}", resultIndex); - client.delete(new DeleteRequest(resultIndex).id(DUMMY_AD_RESULT_ID), ActionListener.wrap(deleteResponse -> { - logger.debug("Successfully deleted dummy AD result from result index {}", resultIndex); + logger.debug("Successfully wrote dummy result to result index {}", resultIndex); + client.delete(createDummyDeleteRequest(resultIndex), ActionListener.wrap(deleteResponse -> { + logger.debug("Successfully deleted dummy result from result index {}", resultIndex); function.execute(); }, ex -> { - logger.error("Failed to delete dummy AD result from result index " + resultIndex, ex); + logger.error("Failed to delete dummy result from result index " + resultIndex, ex); listener.onFailure(ex); })); }, exception -> { - logger.error("Failed to write dummy AD result to result index " + resultIndex, exception); + logger.error("Failed to write dummy result to result index " + resultIndex, exception); listener.onFailure(exception); })); } catch (Exception e) { - logger.error("Failed to create detector with custom result index " + resultIndex, e); + logger.error("Failed to validate custom result index " + resultIndex, e); listener.onFailure(e); } } + public void update() { + if ((allMappingUpdated && allSettingUpdated) || updateRunningTimes >= maxUpdateRunningTimes || updateRunning.get()) { + return; + } + updateRunning.set(true); + updateRunningTimes++; + + // set updateRunning to false when both updateMappingIfNecessary and updateSettingIfNecessary + // stop running + final GroupedActionListener groupListeneer = new GroupedActionListener<>( + ActionListener.wrap(r -> updateRunning.set(false), exception -> { + updateRunning.set(false); + logger.error("Fail to update time series indices", exception); + }), + // 2 since we need both updateMappingIfNecessary and updateSettingIfNecessary to return + // before setting updateRunning to false + 2 + ); + + updateMappingIfNecessary(groupListeneer); + updateSettingIfNecessary(groupListeneer); + } + + private void updateSettingIfNecessary(GroupedActionListener delegateListeneer) { + if (allSettingUpdated) { + delegateListeneer.onResponse(null); + return; + } + + List updates = new ArrayList<>(); + for (IndexType index : indexType.getEnumConstants()) { + Boolean updated = indexStates.computeIfAbsent(index, k -> new IndexState(k.getMapping())).settingUpToDate; + if (Boolean.FALSE.equals(updated)) { + updates.add(index); + } + } + if (updates.size() == 0) { + allSettingUpdated = true; + delegateListeneer.onResponse(null); + return; + } + + final GroupedActionListener conglomerateListeneer = new GroupedActionListener<>( + ActionListener.wrap(r -> delegateListeneer.onResponse(null), exception -> { + delegateListeneer.onResponse(null); + logger.error("Fail to update time series indices' mappings", exception); + }), + updates.size() + ); + for (IndexType timeseriesIndex : updates) { + logger.info(new ParameterizedMessage("Check [{}]'s setting", timeseriesIndex.getIndexName())); + if (timeseriesIndex.isJobIndex()) { + updateJobIndexSettingIfNecessary( + ADIndex.JOB.getIndexName(), + indexStates.computeIfAbsent(timeseriesIndex, k -> new IndexState(k.getMapping())), + conglomerateListeneer + ); + } else { + // we don't have settings to update for other indices + IndexState indexState = indexStates.computeIfAbsent(timeseriesIndex, k -> new IndexState(k.getMapping())); + indexState.settingUpToDate = true; + logger.info(new ParameterizedMessage("Mark [{}]'s setting up-to-date", timeseriesIndex.getIndexName())); + conglomerateListeneer.onResponse(null); + } + } + } + + /** + * Update mapping if schema version changes. + */ + private void updateMappingIfNecessary(GroupedActionListener delegateListeneer) { + if (allMappingUpdated) { + delegateListeneer.onResponse(null); + return; + } + + List updates = new ArrayList<>(); + for (IndexType index : indexType.getEnumConstants()) { + Boolean updated = indexStates.computeIfAbsent(index, k -> new IndexState(k.getMapping())).mappingUpToDate; + if (Boolean.FALSE.equals(updated)) { + updates.add(index); + } + } + if (updates.size() == 0) { + allMappingUpdated = true; + delegateListeneer.onResponse(null); + return; + } + + final GroupedActionListener conglomerateListeneer = new GroupedActionListener<>( + ActionListener.wrap(r -> delegateListeneer.onResponse(null), exception -> { + delegateListeneer.onResponse(null); + logger.error("Fail to update time series indices' mappings", exception); + }), + updates.size() + ); + + for (IndexType adIndex : updates) { + logger.info(new ParameterizedMessage("Check [{}]'s mapping", adIndex.getIndexName())); + shouldUpdateIndex(adIndex, ActionListener.wrap(shouldUpdate -> { + if (shouldUpdate) { + adminClient + .indices() + .putMapping( + new PutMappingRequest().indices(adIndex.getIndexName()).source(adIndex.getMapping(), XContentType.JSON), + ActionListener.wrap(putMappingResponse -> { + if (putMappingResponse.isAcknowledged()) { + logger.info(new ParameterizedMessage("Succeeded in updating [{}]'s mapping", adIndex.getIndexName())); + markMappingUpdated(adIndex); + } else { + logger.error(new ParameterizedMessage("Fail to update [{}]'s mapping", adIndex.getIndexName())); + } + conglomerateListeneer.onResponse(null); + }, exception -> { + logger + .error( + new ParameterizedMessage( + "Fail to update [{}]'s mapping due to [{}]", + adIndex.getIndexName(), + exception.getMessage() + ) + ); + conglomerateListeneer.onFailure(exception); + }) + ); + } else { + // index does not exist or the version is already up-to-date. + // When creating index, new mappings will be used. + // We don't need to update it. + logger.info(new ParameterizedMessage("We don't need to update [{}]'s mapping", adIndex.getIndexName())); + markMappingUpdated(adIndex); + conglomerateListeneer.onResponse(null); + } + }, exception -> { + logger + .error( + new ParameterizedMessage("Fail to check whether we should update [{}]'s mapping", adIndex.getIndexName()), + exception + ); + conglomerateListeneer.onFailure(exception); + })); + + } + } + + private void markMappingUpdated(IndexType adIndex) { + IndexState indexState = indexStates.computeIfAbsent(adIndex, k -> new IndexState(k.getMapping())); + if (Boolean.FALSE.equals(indexState.mappingUpToDate)) { + indexState.mappingUpToDate = Boolean.TRUE; + logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", adIndex.getIndexName())); + } + } + + private void shouldUpdateIndex(IndexType index, ActionListener thenDo) { + boolean exists = false; + if (index.isAlias()) { + exists = doesAliasExist(index.getIndexName()); + } else { + exists = doesIndexExist(index.getIndexName()); + } + if (false == exists) { + thenDo.onResponse(Boolean.FALSE); + return; + } + + Integer newVersion = indexStates.computeIfAbsent(index, k -> new IndexState(k.getMapping())).schemaVersion; + if (index.isAlias()) { + GetAliasesRequest getAliasRequest = new GetAliasesRequest() + .aliases(index.getIndexName()) + .indicesOptions(IndicesOptions.lenientExpandOpenHidden()); + adminClient.indices().getAliases(getAliasRequest, ActionListener.wrap(getAliasResponse -> { + String concreteIndex = null; + for (Map.Entry> entry : getAliasResponse.getAliases().entrySet()) { + if (false == entry.getValue().isEmpty()) { + // we assume the alias map to one concrete index, thus we can return after finding one + concreteIndex = entry.getKey(); + break; + } + } + if (concreteIndex == null) { + thenDo.onResponse(Boolean.FALSE); + return; + } + shouldUpdateConcreteIndex(concreteIndex, newVersion, thenDo); + }, exception -> logger.error(new ParameterizedMessage("Fail to get [{}]'s alias", index.getIndexName()), exception))); + } else { + shouldUpdateConcreteIndex(index.getIndexName(), newVersion, thenDo); + } + } + + /** + * + * @param index Index metadata + * @return The schema version of the given Index + */ + public int getSchemaVersion(IndexType index) { + IndexState indexState = this.indexStates.computeIfAbsent(index, k -> new IndexState(k.getMapping())); + return indexState.schemaVersion; + } + + public void initCustomResultIndexAndExecute(String resultIndex, ExecutorFunction function, ActionListener listener) { + if (!doesIndexExist(resultIndex)) { + initCustomResultIndexDirectly(resultIndex, ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Successfully created result index {}", resultIndex); + validateCustomResultIndexAndExecute(resultIndex, function, listener); + } else { + String error = "Creating result index with mappings call not acknowledged: " + resultIndex; + logger.error(error); + listener.onFailure(new EndRunException(error, false)); + } + }, exception -> { + if (ExceptionsHelper.unwrapCause(exception) instanceof ResourceAlreadyExistsException) { + // It is possible the index has been created while we sending the create request + validateCustomResultIndexAndExecute(resultIndex, function, listener); + } else { + logger.error("Failed to create result index " + resultIndex, exception); + listener.onFailure(exception); + } + })); + } else { + validateCustomResultIndexAndExecute(resultIndex, function, listener); + } + } + public void validateCustomIndexForBackendJob( String resultIndex, String securityLogId, String user, List roles, - AnomalyDetectorFunction function, + ExecutorFunction function, ActionListener listener ) { if (!doesIndexExist(resultIndex)) { @@ -416,743 +790,197 @@ public void validateCustomIndexForBackendJob( } } - /** - * Check if custom result index has correct index mapping. - * @param resultIndex result index - * @return true if result index mapping is valid - */ - public boolean isValidResultIndexMapping(String resultIndex) { - try { - initResultMapping(); - if (AD_RESULT_FIELD_CONFIGS == null) { - // failed to populate the field - return false; - } - IndexMetadata indexMetadata = clusterService.state().metadata().index(resultIndex); - Map indexMapping = indexMetadata.mapping().sourceAsMap(); - String propertyName = CommonName.PROPERTIES; - if (!indexMapping.containsKey(propertyName) || !(indexMapping.get(propertyName) instanceof LinkedHashMap)) { - return false; - } - LinkedHashMap mapping = (LinkedHashMap) indexMapping.get(propertyName); + protected int getNumberOfPrimaryShards() { + return Math.min(nodeFilter.getNumberOfEligibleDataNodes(), maxPrimaryShards); + } - boolean correctResultIndexMapping = true; + @Override + public void onClusterManager() { + try { + // try to rollover immediately as we might be restarting the cluster + rolloverAndDeleteHistoryIndex(); - for (String fieldName : AD_RESULT_FIELD_CONFIGS.keySet()) { - Object defaultSchema = AD_RESULT_FIELD_CONFIGS.get(fieldName); - // the field might be a map or map of map - // example: map: {type=date, format=strict_date_time||epoch_millis} - // map of map: {type=nested, properties={likelihood={type=double}, value_list={type=nested, properties={data={type=double}, - // feature_id={type=keyword}}}}} - // if it is a map of map, Object.equals can compare them regardless of order - if (!mapping.containsKey(fieldName) || !defaultSchema.equals(mapping.get(fieldName))) { - correctResultIndexMapping = false; - break; - } - } - return correctResultIndexMapping; + // schedule the next rollover for approx MAX_AGE later + scheduledRollover = threadPool + .scheduleWithFixedDelay(() -> rolloverAndDeleteHistoryIndex(), historyRolloverPeriod, executorName()); } catch (Exception e) { - logger.error("Failed to validate result index mapping for index " + resultIndex, e); - return false; + // This should be run on cluster startup + logger.error("Error rollover result indices. " + "Can't rollover result until clusterManager node is restarted.", e); } - } - /** - * Anomaly state index exist or not. - * - * @return true if anomaly state index exists - */ - public boolean doesDetectorStateIndexExist() { - return clusterService.state().getRoutingTable().hasIndex(ADCommonName.DETECTION_STATE_INDEX); - } - - /** - * Checkpoint index exist or not. - * - * @return true if checkpoint index exists - */ - public boolean doesCheckpointIndexExist() { - return clusterService.state().getRoutingTable().hasIndex(ADCommonName.CHECKPOINT_INDEX_NAME); - } - - /** - * Index exists or not - * @param clusterServiceAccessor Cluster service - * @param name Index name - * @return true if the index exists - */ - public static boolean doesIndexExists(ClusterService clusterServiceAccessor, String name) { - return clusterServiceAccessor.state().getRoutingTable().hasIndex(name); + @Override + public void offClusterManager() { + if (scheduledRollover != null) { + scheduledRollover.cancel(); + } } - /** - * Alias exists or not - * @param clusterServiceAccessor Cluster service - * @param alias Alias name - * @return true if the alias exists - */ - public static boolean doesAliasExists(ClusterService clusterServiceAccessor, String alias) { - return clusterServiceAccessor.state().metadata().hasAlias(alias); + private String executorName() { + return ThreadPool.Names.MANAGEMENT; } - private ActionListener markMappingUpToDate(ADIndex index, ActionListener followingListener) { - return ActionListener.wrap(createdResponse -> { - if (createdResponse.isAcknowledged()) { - IndexState indexStatetate = indexStates.computeIfAbsent(index, IndexState::new); - if (Boolean.FALSE.equals(indexStatetate.mappingUpToDate)) { - indexStatetate.mappingUpToDate = Boolean.TRUE; - logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", index.getIndexName())); - } + protected void rescheduleRollover() { + if (clusterService.state().getNodes().isLocalNodeElectedClusterManager()) { + if (scheduledRollover != null) { + scheduledRollover.cancel(); } - followingListener.onResponse(createdResponse); - }, exception -> followingListener.onFailure(exception)); - } - - /** - * Create anomaly detector index if not exist. - * - * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyDetectorMappings} - */ - public void initAnomalyDetectorIndexIfAbsent(ActionListener actionListener) throws IOException { - if (!doesAnomalyDetectorIndexExist()) { - initAnomalyDetectorIndex(actionListener); + scheduledRollover = threadPool + .scheduleWithFixedDelay(() -> rolloverAndDeleteHistoryIndex(), historyRolloverPeriod, executorName()); } } - /** - * Create anomaly detector index directly. - * - * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyDetectorMappings} - */ - public void initAnomalyDetectorIndex(ActionListener actionListener) throws IOException { - CreateIndexRequest request = new CreateIndexRequest(CommonName.CONFIG_INDEX) - .mapping(getAnomalyDetectorMappings(), XContentType.JSON) - .settings(settings); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.CONFIG, actionListener)); - } - - /** - * Create anomaly result index if not exist. - * - * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyResultMappings} - */ - public void initDefaultAnomalyResultIndexIfAbsent(ActionListener actionListener) throws IOException { - if (!doesDefaultAnomalyResultIndexExist()) { - initDefaultAnomalyResultIndexDirectly(actionListener); + private void initResultMapping() throws IOException { + if (RESULT_FIELD_CONFIGS != null) { + // we have already initiated the field + return; } - } - - /** - * choose the number of primary shards for checkpoint, multientity result, and job scheduler based on the number of hot nodes. Max 10. - * @param request The request to add the setting - */ - private void choosePrimaryShards(CreateIndexRequest request) { - choosePrimaryShards(request, true); - } - - private void choosePrimaryShards(CreateIndexRequest request, boolean hiddenIndex) { - request - .settings( - Settings - .builder() - // put 1 primary shards per hot node if possible - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, getNumberOfPrimaryShards()) - // 1 replica for better search performance and fail-over - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) - .put("index.hidden", hiddenIndex) - ); - } - - private int getNumberOfPrimaryShards() { - return Math.min(nodeFilter.getNumberOfEligibleDataNodes(), maxPrimaryShards); - } - - /** - * Create anomaly result index without checking exist or not. - * - * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyResultMappings} - */ - public void initDefaultAnomalyResultIndexDirectly(ActionListener actionListener) throws IOException { - initAnomalyResultIndexDirectly(AD_RESULT_HISTORY_INDEX_PATTERN, ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, true, actionListener); - } - - public void initCustomAnomalyResultIndexDirectly(String resultIndex, ActionListener actionListener) - throws IOException { - initAnomalyResultIndexDirectly(resultIndex, null, false, actionListener); - } - public void initAnomalyResultIndexDirectly( - String resultIndex, - String alias, - boolean hiddenIndex, - ActionListener actionListener - ) throws IOException { - String mapping = getAnomalyResultMappings(); - CreateIndexRequest request = new CreateIndexRequest(resultIndex).mapping(mapping, XContentType.JSON); - if (alias != null) { - request.alias(new Alias(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS)); - } - choosePrimaryShards(request, hiddenIndex); - if (AD_RESULT_HISTORY_INDEX_PATTERN.equals(resultIndex)) { - adminClient.indices().create(request, markMappingUpToDate(ADIndex.RESULT, actionListener)); + Map asMap = XContentHelper.convertToMap(new BytesArray(resultMapping), false, XContentType.JSON).v2(); + Object properties = asMap.get(CommonName.PROPERTIES); + if (properties instanceof Map) { + RESULT_FIELD_CONFIGS = (Map) properties; } else { - adminClient.indices().create(request, actionListener); + logger.error("Fail to read result mapping file."); } } /** - * Create anomaly detector job index. - * - * @param actionListener action called after create index + * Check if custom result index has correct index mapping. + * @param resultIndex result index + * @return true if result index mapping is valid */ - public void initAnomalyDetectorJobIndex(ActionListener actionListener) { + public boolean isValidResultIndexMapping(String resultIndex) { try { - CreateIndexRequest request = new CreateIndexRequest(CommonName.JOB_INDEX) - .mapping(getAnomalyDetectorJobMappings(), XContentType.JSON); - request - .settings( - Settings - .builder() - // AD job index is small. 1 primary shard is enough - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - // Job scheduler puts both primary and replica shards in the - // hash ring. Auto-expand the number of replicas based on the - // number of data nodes (up to 20) in the cluster so that each node can - // become a coordinating node. This is useful when customers - // scale out their cluster so that we can do adaptive scaling - // accordingly. - // At least 1 replica for fail-over. - .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, minJobIndexReplicas + "-" + maxJobIndexReplicas) - .put("index.hidden", true) - ); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.JOB, actionListener)); - } catch (IOException e) { - logger.error("Fail to init AD job index", e); - actionListener.onFailure(e); - } - } + initResultMapping(); + if (RESULT_FIELD_CONFIGS == null) { + // failed to populate the field + return false; + } + IndexMetadata indexMetadata = clusterService.state().metadata().index(resultIndex); + Map indexMapping = indexMetadata.mapping().sourceAsMap(); + String propertyName = CommonName.PROPERTIES; + if (!indexMapping.containsKey(propertyName) || !(indexMapping.get(propertyName) instanceof LinkedHashMap)) { + return false; + } + LinkedHashMap mapping = (LinkedHashMap) indexMapping.get(propertyName); - /** - * Create the state index. - * - * @param actionListener action called after create index - */ - public void initDetectionStateIndex(ActionListener actionListener) { - try { - CreateIndexRequest request = new CreateIndexRequest(ADCommonName.DETECTION_STATE_INDEX) - .mapping(getDetectionStateMappings(), XContentType.JSON) - .settings(settings); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.STATE, actionListener)); - } catch (IOException e) { - logger.error("Fail to init AD detection state index", e); - actionListener.onFailure(e); + boolean correctResultIndexMapping = true; + + for (String fieldName : RESULT_FIELD_CONFIGS.keySet()) { + Object defaultSchema = RESULT_FIELD_CONFIGS.get(fieldName); + // the field might be a map or map of map + // example: map: {type=date, format=strict_date_time||epoch_millis} + // map of map: {type=nested, properties={likelihood={type=double}, value_list={type=nested, properties={data={type=double}, + // feature_id={type=keyword}}}}} + // if it is a map of map, Object.equals can compare them regardless of order + if (!mapping.containsKey(fieldName) || !defaultSchema.equals(mapping.get(fieldName))) { + correctResultIndexMapping = false; + break; + } + } + return correctResultIndexMapping; + } catch (Exception e) { + logger.error("Failed to validate result index mapping for index " + resultIndex, e); + return false; } + } /** - * Create the checkpoint index. + * Create forecast result index if not exist. * * @param actionListener action called after create index - * @throws EndRunException EndRunException due to failure to get mapping */ - public void initCheckpointIndex(ActionListener actionListener) { - String mapping; - try { - mapping = getCheckpointMappings(); - } catch (IOException e) { - throw new EndRunException("", "Cannot find checkpoint mapping file", true); - } - CreateIndexRequest request = new CreateIndexRequest(ADCommonName.CHECKPOINT_INDEX_NAME).mapping(mapping, XContentType.JSON); - choosePrimaryShards(request); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.CHECKPOINT, actionListener)); - } - - @Override - public void onClusterManager() { - try { - // try to rollover immediately as we might be restarting the cluster - rolloverAndDeleteHistoryIndex(); - - // schedule the next rollover for approx MAX_AGE later - scheduledRollover = threadPool - .scheduleWithFixedDelay(() -> rolloverAndDeleteHistoryIndex(), historyRolloverPeriod, executorName()); - } catch (Exception e) { - // This should be run on cluster startup - logger.error("Error rollover AD result indices. " + "Can't rollover AD result until clusterManager node is restarted.", e); - } - } - - @Override - public void offClusterManager() { - if (scheduledRollover != null) { - scheduledRollover.cancel(); + public void initDefaultResultIndexIfAbsent(ActionListener actionListener) { + if (!doesDefaultResultIndexExist()) { + initDefaultResultIndexDirectly(actionListener); } } - private String executorName() { - return ThreadPool.Names.MANAGEMENT; - } - - private void rescheduleRollover() { - if (clusterService.state().getNodes().isLocalNodeElectedClusterManager()) { - if (scheduledRollover != null) { - scheduledRollover.cancel(); + protected ActionListener markMappingUpToDate( + IndexType index, + ActionListener followingListener + ) { + return ActionListener.wrap(createdResponse -> { + if (createdResponse.isAcknowledged()) { + IndexState indexStatetate = indexStates.computeIfAbsent(index, k -> new IndexState(k.getMapping())); + if (Boolean.FALSE.equals(indexStatetate.mappingUpToDate)) { + indexStatetate.mappingUpToDate = Boolean.TRUE; + logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", index.getIndexName())); + } } - scheduledRollover = threadPool - .scheduleWithFixedDelay(() -> rolloverAndDeleteHistoryIndex(), historyRolloverPeriod, executorName()); - } + followingListener.onResponse(createdResponse); + }, exception -> followingListener.onFailure(exception)); } - void rolloverAndDeleteHistoryIndex() { - if (!doesDefaultAnomalyResultIndexExist()) { + protected void rolloverAndDeleteHistoryIndex( + String resultIndexAlias, + String allResultIndicesPattern, + String rolloverIndexPattern, + IndexType resultIndex + ) { + if (!doesDefaultResultIndexExist()) { return; } // We have to pass null for newIndexName in order to get Elastic to increment the index count. - RolloverRequest rollOverRequest = new RolloverRequest(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, null); - String adResultMapping = null; - try { - adResultMapping = getAnomalyResultMappings(); - } catch (IOException e) { - logger.error("Fail to roll over AD result index, as can't get AD result index mapping"); - return; - } + RolloverRequest rollOverRequest = new RolloverRequest(resultIndexAlias, null); + CreateIndexRequest createRequest = rollOverRequest.getCreateIndexRequest(); - createRequest.index(AD_RESULT_HISTORY_INDEX_PATTERN).mapping(adResultMapping, XContentType.JSON); + createRequest.index(rolloverIndexPattern).mapping(resultMapping, XContentType.JSON); - choosePrimaryShards(createRequest); + choosePrimaryShards(createRequest, true); rollOverRequest.addMaxIndexDocsCondition(historyMaxDocs * getNumberOfPrimaryShards()); adminClient.indices().rolloverIndex(rollOverRequest, ActionListener.wrap(response -> { if (!response.isRolledOver()) { - logger - .warn( - "{} not rolled over. Conditions were: {}", - ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, - response.getConditionStatus() - ); + logger.warn("{} not rolled over. Conditions were: {}", resultIndexAlias, response.getConditionStatus()); } else { - IndexState indexStatetate = indexStates.computeIfAbsent(ADIndex.RESULT, IndexState::new); + IndexState indexStatetate = indexStates.computeIfAbsent(resultIndex, k -> new IndexState(k.getMapping())); indexStatetate.mappingUpToDate = true; - logger.info("{} rolled over. Conditions were: {}", ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, response.getConditionStatus()); - deleteOldHistoryIndices(); + logger.info("{} rolled over. Conditions were: {}", resultIndexAlias, response.getConditionStatus()); + deleteOldHistoryIndices(allResultIndicesPattern, historyRetentionPeriod); } }, exception -> { logger.error("Fail to roll over result index", exception); })); } - void deleteOldHistoryIndices() { - Set candidates = new HashSet(); + protected void initResultIndexDirectly( + String resultIndexName, + String alias, + boolean hiddenIndex, + String resultIndexPattern, + IndexType resultIndex, + ActionListener actionListener + ) { + CreateIndexRequest request = new CreateIndexRequest(resultIndexName).mapping(resultMapping, XContentType.JSON); + if (alias != null) { + request.alias(new Alias(alias)); + } + choosePrimaryShards(request, hiddenIndex); + if (resultIndexPattern.equals(resultIndexName)) { + adminClient.indices().create(request, markMappingUpToDate(resultIndex, actionListener)); + } else { + adminClient.indices().create(request, actionListener); + } + } - ClusterStateRequest clusterStateRequest = new ClusterStateRequest() - .clear() - .indices(AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN) - .metadata(true) - .local(true) - .indicesOptions(IndicesOptions.strictExpand()); + public abstract boolean doesCheckpointIndexExist(); - adminClient.cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { - String latestToDelete = null; - long latest = Long.MIN_VALUE; - for (IndexMetadata indexMetaData : clusterStateResponse.getState().metadata().indices().values()) { - long creationTime = indexMetaData.getCreationDate(); + public abstract void initCheckpointIndex(ActionListener actionListener); - if ((Instant.now().toEpochMilli() - creationTime) > historyRetentionPeriod.millis()) { - String indexName = indexMetaData.getIndex().getName(); - candidates.add(indexName); - if (latest < creationTime) { - latest = creationTime; - latestToDelete = indexName; - } - } - } + public abstract boolean doesDefaultResultIndexExist(); - if (candidates.size() > 1) { - // delete all indices except the last one because the last one may contain docs newer than the retention period - candidates.remove(latestToDelete); - String[] toDelete = candidates.toArray(Strings.EMPTY_ARRAY); - DeleteIndexRequest deleteIndexRequest = new DeleteIndexRequest(toDelete); - adminClient.indices().delete(deleteIndexRequest, ActionListener.wrap(deleteIndexResponse -> { - if (!deleteIndexResponse.isAcknowledged()) { - logger - .error( - "Could not delete one or more Anomaly result indices: {}. Retrying one by one.", - Arrays.toString(toDelete) - ); - deleteIndexIteration(toDelete); - } else { - logger.info("Succeeded in deleting expired anomaly result indices: {}.", Arrays.toString(toDelete)); - } - }, exception -> { - logger.error("Failed to delete expired anomaly result indices: {}.", Arrays.toString(toDelete)); - deleteIndexIteration(toDelete); - })); - } - }, exception -> { logger.error("Fail to delete result indices", exception); })); - } + public abstract boolean doesStateIndexExist(); - private void deleteIndexIteration(String[] toDelete) { - for (String index : toDelete) { - DeleteIndexRequest singleDeleteRequest = new DeleteIndexRequest(index); - adminClient.indices().delete(singleDeleteRequest, ActionListener.wrap(singleDeleteResponse -> { - if (!singleDeleteResponse.isAcknowledged()) { - logger.error("Retrying deleting {} does not succeed.", index); - } - }, exception -> { - if (exception instanceof IndexNotFoundException) { - logger.info("{} was already deleted.", index); - } else { - logger.error(new ParameterizedMessage("Retrying deleting {} does not succeed.", index), exception); - } - })); - } - } - - public void update() { - if ((allMappingUpdated && allSettingUpdated) || updateRunningTimes >= maxUpdateRunningTimes || updateRunning.get()) { - return; - } - updateRunning.set(true); - updateRunningTimes++; - - // set updateRunning to false when both updateMappingIfNecessary and updateSettingIfNecessary - // stop running - final GroupedActionListener groupListeneer = new GroupedActionListener<>( - ActionListener.wrap(r -> updateRunning.set(false), exception -> { - updateRunning.set(false); - logger.error("Fail to update AD indices", exception); - }), - // 2 since we need both updateMappingIfNecessary and updateSettingIfNecessary to return - // before setting updateRunning to false - 2 - ); - - updateMappingIfNecessary(groupListeneer); - updateSettingIfNecessary(groupListeneer); - } - - private void updateSettingIfNecessary(GroupedActionListener delegateListeneer) { - if (allSettingUpdated) { - delegateListeneer.onResponse(null); - return; - } - - List updates = new ArrayList<>(); - for (ADIndex index : ADIndex.values()) { - Boolean updated = indexStates.computeIfAbsent(index, IndexState::new).settingUpToDate; - if (Boolean.FALSE.equals(updated)) { - updates.add(index); - } - } - if (updates.size() == 0) { - allSettingUpdated = true; - delegateListeneer.onResponse(null); - return; - } - - final GroupedActionListener conglomerateListeneer = new GroupedActionListener<>( - ActionListener.wrap(r -> delegateListeneer.onResponse(null), exception -> { - delegateListeneer.onResponse(null); - logger.error("Fail to update AD indices' mappings", exception); - }), - updates.size() - ); - for (ADIndex adIndex : updates) { - logger.info(new ParameterizedMessage("Check [{}]'s setting", adIndex.getIndexName())); - switch (adIndex) { - case JOB: - updateJobIndexSettingIfNecessary(indexStates.computeIfAbsent(adIndex, IndexState::new), conglomerateListeneer); - break; - default: - // we don't have settings to update for other indices - IndexState indexState = indexStates.computeIfAbsent(adIndex, IndexState::new); - indexState.settingUpToDate = true; - logger.info(new ParameterizedMessage("Mark [{}]'s setting up-to-date", adIndex.getIndexName())); - conglomerateListeneer.onResponse(null); - break; - } - - } - } - - /** - * Update mapping if schema version changes. - */ - private void updateMappingIfNecessary(GroupedActionListener delegateListeneer) { - if (allMappingUpdated) { - delegateListeneer.onResponse(null); - return; - } - - List updates = new ArrayList<>(); - for (ADIndex index : ADIndex.values()) { - Boolean updated = indexStates.computeIfAbsent(index, IndexState::new).mappingUpToDate; - if (Boolean.FALSE.equals(updated)) { - updates.add(index); - } - } - if (updates.size() == 0) { - allMappingUpdated = true; - delegateListeneer.onResponse(null); - return; - } + public abstract void initDefaultResultIndexDirectly(ActionListener actionListener); - final GroupedActionListener conglomerateListeneer = new GroupedActionListener<>( - ActionListener.wrap(r -> delegateListeneer.onResponse(null), exception -> { - delegateListeneer.onResponse(null); - logger.error("Fail to update AD indices' mappings", exception); - }), - updates.size() - ); + protected abstract IndexRequest createDummyIndexRequest(String resultIndex) throws IOException; - for (ADIndex adIndex : updates) { - logger.info(new ParameterizedMessage("Check [{}]'s mapping", adIndex.getIndexName())); - shouldUpdateIndex(adIndex, ActionListener.wrap(shouldUpdate -> { - if (shouldUpdate) { - adminClient - .indices() - .putMapping( - new PutMappingRequest().indices(adIndex.getIndexName()).source(adIndex.getMapping(), XContentType.JSON), - ActionListener.wrap(putMappingResponse -> { - if (putMappingResponse.isAcknowledged()) { - logger.info(new ParameterizedMessage("Succeeded in updating [{}]'s mapping", adIndex.getIndexName())); - markMappingUpdated(adIndex); - } else { - logger.error(new ParameterizedMessage("Fail to update [{}]'s mapping", adIndex.getIndexName())); - } - conglomerateListeneer.onResponse(null); - }, exception -> { - logger - .error( - new ParameterizedMessage( - "Fail to update [{}]'s mapping due to [{}]", - adIndex.getIndexName(), - exception.getMessage() - ) - ); - conglomerateListeneer.onFailure(exception); - }) - ); - } else { - // index does not exist or the version is already up-to-date. - // When creating index, new mappings will be used. - // We don't need to update it. - logger.info(new ParameterizedMessage("We don't need to update [{}]'s mapping", adIndex.getIndexName())); - markMappingUpdated(adIndex); - conglomerateListeneer.onResponse(null); - } - }, exception -> { - logger - .error( - new ParameterizedMessage("Fail to check whether we should update [{}]'s mapping", adIndex.getIndexName()), - exception - ); - conglomerateListeneer.onFailure(exception); - })); + protected abstract DeleteRequest createDummyDeleteRequest(String resultIndex) throws IOException; - } - } - - private void markMappingUpdated(ADIndex adIndex) { - IndexState indexState = indexStates.computeIfAbsent(adIndex, IndexState::new); - if (Boolean.FALSE.equals(indexState.mappingUpToDate)) { - indexState.mappingUpToDate = Boolean.TRUE; - logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", adIndex.getIndexName())); - } - } + protected abstract void rolloverAndDeleteHistoryIndex(); - private void shouldUpdateIndex(ADIndex index, ActionListener thenDo) { - boolean exists = false; - if (index.isAlias()) { - exists = AnomalyDetectionIndices.doesAliasExists(clusterService, index.getIndexName()); - } else { - exists = AnomalyDetectionIndices.doesIndexExists(clusterService, index.getIndexName()); - } - if (false == exists) { - thenDo.onResponse(Boolean.FALSE); - return; - } - - Integer newVersion = indexStates.computeIfAbsent(index, IndexState::new).schemaVersion; - if (index.isAlias()) { - GetAliasesRequest getAliasRequest = new GetAliasesRequest() - .aliases(index.getIndexName()) - .indicesOptions(IndicesOptions.lenientExpandOpenHidden()); - adminClient.indices().getAliases(getAliasRequest, ActionListener.wrap(getAliasResponse -> { - String concreteIndex = null; - for (Map.Entry> entry : getAliasResponse.getAliases().entrySet()) { - if (false == entry.getValue().isEmpty()) { - // we assume the alias map to one concrete index, thus we can return after finding one - concreteIndex = entry.getKey(); - break; - } - } - if (concreteIndex == null) { - thenDo.onResponse(Boolean.FALSE); - return; - } - shouldUpdateConcreteIndex(concreteIndex, newVersion, thenDo); - }, exception -> logger.error(new ParameterizedMessage("Fail to get [{}]'s alias", index.getIndexName()), exception))); - } else { - shouldUpdateConcreteIndex(index.getIndexName(), newVersion, thenDo); - } - } - - @SuppressWarnings("unchecked") - private void shouldUpdateConcreteIndex(String concreteIndex, Integer newVersion, ActionListener thenDo) { - IndexMetadata indexMeataData = clusterService.state().getMetadata().indices().get(concreteIndex); - if (indexMeataData == null) { - thenDo.onResponse(Boolean.FALSE); - return; - } - Integer oldVersion = CommonValue.NO_SCHEMA_VERSION; - - Map indexMapping = indexMeataData.mapping().getSourceAsMap(); - Object meta = indexMapping.get(META); - if (meta != null && meta instanceof Map) { - Map metaMapping = (Map) meta; - Object schemaVersion = metaMapping.get(CommonName.SCHEMA_VERSION_FIELD); - if (schemaVersion instanceof Integer) { - oldVersion = (Integer) schemaVersion; - } - } - thenDo.onResponse(newVersion > oldVersion); - } - - private static Integer parseSchemaVersion(String mapping) { - try { - XContentParser xcp = XContentType.JSON - .xContent() - .createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, mapping); - - while (!xcp.isClosed()) { - Token token = xcp.currentToken(); - if (token != null && token != XContentParser.Token.END_OBJECT && token != XContentParser.Token.START_OBJECT) { - if (xcp.currentName() != META) { - xcp.nextToken(); - xcp.skipChildren(); - } else { - while (xcp.nextToken() != XContentParser.Token.END_OBJECT) { - if (xcp.currentName().equals(SCHEMA_VERSION)) { - - Integer version = xcp.intValue(); - if (version < 0) { - version = CommonValue.NO_SCHEMA_VERSION; - } - return version; - } else { - xcp.nextToken(); - } - } - - } - } - xcp.nextToken(); - } - return CommonValue.NO_SCHEMA_VERSION; - } catch (Exception e) { - // since this method is called in the constructor that is called by AnomalyDetectorPlugin.createComponents, - // we cannot throw checked exception - throw new RuntimeException(e); - } - } - - /** - * - * @param index Index metadata - * @return The schema version of the given Index - */ - public int getSchemaVersion(ADIndex index) { - IndexState indexState = this.indexStates.computeIfAbsent(index, IndexState::new); - return indexState.schemaVersion; - } - - private void updateJobIndexSettingIfNecessary(IndexState jobIndexState, ActionListener listener) { - GetSettingsRequest getSettingsRequest = new GetSettingsRequest() - .indices(ADIndex.JOB.getIndexName()) - .names( - new String[] { - IndexMetadata.SETTING_NUMBER_OF_SHARDS, - IndexMetadata.SETTING_NUMBER_OF_REPLICAS, - IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS } - ); - client.execute(GetSettingsAction.INSTANCE, getSettingsRequest, ActionListener.wrap(settingResponse -> { - // auto expand setting is a range string like "1-all" - String autoExpandReplica = getStringSetting(settingResponse, IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS); - // if the auto expand setting is already there, return immediately - if (autoExpandReplica != null) { - jobIndexState.settingUpToDate = true; - logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", ADIndex.JOB.getIndexName())); - listener.onResponse(null); - return; - } - Integer primaryShardsNumber = getIntegerSetting(settingResponse, IndexMetadata.SETTING_NUMBER_OF_SHARDS); - Integer replicaNumber = getIntegerSetting(settingResponse, IndexMetadata.SETTING_NUMBER_OF_REPLICAS); - if (primaryShardsNumber == null || replicaNumber == null) { - logger - .error( - new ParameterizedMessage( - "Fail to find AD job index's primary or replica shard number: primary [{}], replica [{}]", - primaryShardsNumber, - replicaNumber - ) - ); - // don't throw exception as we don't know how to handle it and retry next time - listener.onResponse(null); - return; - } - // at least minJobIndexReplicas - // at most maxJobIndexReplicas / primaryShardsNumber replicas. - // For example, if we have 2 primary shards, since the max number of shards are maxJobIndexReplicas (20), - // we will use 20 / 2 = 10 replicas as the upper bound of replica. - int maxExpectedReplicas = Math.max(maxJobIndexReplicas / primaryShardsNumber, minJobIndexReplicas); - Settings updatedSettings = Settings - .builder() - .put(IndexMetadata.SETTING_AUTO_EXPAND_REPLICAS, minJobIndexReplicas + "-" + maxExpectedReplicas) - .build(); - final UpdateSettingsRequest updateSettingsRequest = new UpdateSettingsRequest(ADIndex.JOB.getIndexName()) - .settings(updatedSettings); - client.admin().indices().updateSettings(updateSettingsRequest, ActionListener.wrap(response -> { - jobIndexState.settingUpToDate = true; - logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", ADIndex.JOB.getIndexName())); - listener.onResponse(null); - }, listener::onFailure)); - }, e -> { - if (e instanceof IndexNotFoundException) { - // new index will be created with auto expand replica setting - jobIndexState.settingUpToDate = true; - logger.info(new ParameterizedMessage("Mark [{}]'s mapping up-to-date", ADIndex.JOB.getIndexName())); - listener.onResponse(null); - } else { - listener.onFailure(e); - } - })); - } - - private static Integer getIntegerSetting(GetSettingsResponse settingsResponse, String settingKey) { - Integer value = null; - for (Settings settings : settingsResponse.getIndexToSettings().values()) { - value = settings.getAsInt(settingKey, null); - if (value != null) { - break; - } - } - return value; - } - - private static String getStringSetting(GetSettingsResponse settingsResponse, String settingKey) { - String value = null; - for (Settings settings : settingsResponse.getIndexToSettings().values()) { - value = settings.get(settingKey, null); - if (value != null) { - break; - } - } - return value; - } + public abstract void initCustomResultIndexDirectly(String resultIndex, ActionListener actionListener); } diff --git a/src/main/java/org/opensearch/timeseries/indices/TimeSeriesIndex.java b/src/main/java/org/opensearch/timeseries/indices/TimeSeriesIndex.java new file mode 100644 index 000000000..e7364ed32 --- /dev/null +++ b/src/main/java/org/opensearch/timeseries/indices/TimeSeriesIndex.java @@ -0,0 +1,22 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.timeseries.indices; + +public interface TimeSeriesIndex { + public String getIndexName(); + + public boolean isAlias(); + + public String getMapping(); + + public boolean isJobIndex(); +} diff --git a/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java b/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java index 054e241aa..a9aebff53 100644 --- a/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java +++ b/src/main/java/org/opensearch/timeseries/settings/TimeSeriesSettings.java @@ -5,6 +5,11 @@ package org.opensearch.timeseries.settings; +import java.time.Duration; + +import org.opensearch.common.settings.Setting; +import org.opensearch.common.unit.TimeValue; + public class TimeSeriesSettings { // ====================================== @@ -16,7 +21,7 @@ public class TimeSeriesSettings { // the larger shingle size, the harder to fill in a complete shingle public static final int MAX_SHINGLE_SIZE = 60; - public static final String INDEX_MAPPING_FILE = "mappings/anomaly-detectors.json"; + public static final String CONFIG_INDEX_MAPPING_FILE = "mappings/anomaly-detectors.json"; public static final String JOBS_INDEX_MAPPING_FILE = "mappings/anomaly-detector-jobs.json"; @@ -40,4 +45,147 @@ public class TimeSeriesSettings { // ====================================== public static final int MAX_BATCH_TASK_PIECE_SIZE = 10_000; + // within an interval, how many percents are used to process requests. + // 1.0 means we use all of the detection interval to process requests. + // to ensure we don't block next interval, it is better to set it less than 1.0. + public static final float INTERVAL_RATIO_FOR_REQUESTS = 0.9f; + + public static final Duration HOURLY_MAINTENANCE = Duration.ofHours(1); + + // ====================================== + // Checkpoint setting + // ====================================== + // we won't accept a checkpoint larger than 30MB. Or we risk OOM. + // For reference, in RCF 1.0, the checkpoint of a RCF with 50 trees, 10 dimensions, + // 256 samples is of 3.2MB. + // In compact rcf, the same RCF is of 163KB. + // Since we allow at most 5 features, and the default shingle size is 8 and default + // tree number size is 100, we can have at most 25.6 MB in RCF 1.0. + // It is possible that cx increases the max features or shingle size, but we don't want + // to risk OOM for the flexibility. + public static final int MAX_CHECKPOINT_BYTES = 30_000_000; + + // Sets the cap on the number of buffer that can be allocated by the rcf deserialization + // buffer pool. Each buffer is of 512 bytes. Memory occupied by 20 buffers is 10.24 KB. + public static final int MAX_TOTAL_RCF_SERIALIZATION_BUFFERS = 20; + + // the size of the buffer used for rcf deserialization + public static final int SERIALIZATION_BUFFER_BYTES = 512; + + // ====================================== + // rate-limiting queue parameters + // ====================================== + /** + * CheckpointWriteRequest consists of IndexRequest (200 KB), and QueuedRequest + * fields (148 bytes, read comments of ENTITY_REQUEST_SIZE_CONSTANT). + * The total is roughly 200 KB per request. + * + * We don't want the total size exceeds 1% of the heap. + * We should have at most 1% heap / 200KB = heap / 20,000,000 + * For t3.small, 1% heap is of 10MB. The queue's size is up to + * 10^ 7 / 2.0 * 10^5 = 50 + */ + public static int CHECKPOINT_WRITE_QUEUE_SIZE_IN_BYTES = 200_000; + + /** + * ResultWriteRequest consists of index request (roughly 1KB), and QueuedRequest + * fields (148 bytes, read comments of ENTITY_REQUEST_SIZE_CONSTANT). + * Plus Java object size (12 bytes), we have roughly 1160 bytes per request + * + * We don't want the total size exceeds 1% of the heap. + * We should have at most 1% heap / 1148 = heap / 116,000 + * For t3.small, 1% heap is of 10MB. The queue's size is up to + * 10^ 7 / 1160 = 8621 + */ + public static int RESULT_WRITE_QUEUE_SIZE_IN_BYTES = 1160; + + /** + * FeatureRequest has entityName (# category fields * 256, the recommended limit + * of a keyword field length), model Id (roughly 256 bytes), and QueuedRequest + * fields including config Id(roughly 128 bytes), dataStartTimeMillis (long, + * 8 bytes), and currentFeature (16 bytes, assume two features on average). + * Plus Java object size (12 bytes), we have roughly 932 bytes per request + * assuming we have 2 categorical fields (plan to support 2 categorical fields now). + * We don't want the total size exceeds 0.1% of the heap. + * We can have at most 0.1% heap / 932 = heap / 932,000. + * For t3.small, 0.1% heap is of 1MB. The queue's size is up to + * 10^ 6 / 932 = 1072 + */ + public static int FEATURE_REQUEST_SIZE_IN_BYTES = 932; + + /** + * CheckpointMaintainRequest has model Id (roughly 256 bytes), and QueuedRequest + * fields including detector Id(roughly 128 bytes), expirationEpochMs (long, + * 8 bytes), and priority (12 bytes). + * Plus Java object size (12 bytes), we have roughly 416 bytes per request. + * We don't want the total size exceeds 0.1% of the heap. + * We can have at most 0.1% heap / 416 = heap / 416,000. + * For t3.small, 0.1% heap is of 1MB. The queue's size is up to + * 10^ 6 / 416 = 2403 + */ + public static int CHECKPOINT_MAINTAIN_REQUEST_SIZE_IN_BYTES = 416; + + public static final float MAX_QUEUED_TASKS_RATIO = 0.5f; + + public static final float MEDIUM_SEGMENT_PRUNE_RATIO = 0.1f; + + public static final float LOW_SEGMENT_PRUNE_RATIO = 0.3f; + + // expensive maintenance (e.g., queue maintenance) with 1/10000 probability + public static final int MAINTENANCE_FREQ_CONSTANT = 10000; + + public static final Duration QUEUE_MAINTENANCE = Duration.ofMinutes(10); + + // ====================================== + // ML parameters + // ====================================== + // RCF + public static final int NUM_SAMPLES_PER_TREE = 256; + + public static final int NUM_TREES = 30; + + public static final double TIME_DECAY = 0.0001; + + // If we have 32 + shingleSize (hopefully recent) values, RCF can get up and running. It will be noisy — + // there is a reason that default size is 256 (+ shingle size), but it may be more useful for people to + /// start seeing some results. + public static final int NUM_MIN_SAMPLES = 32; + + // for a batch operation, we want all of the bounding box in-place for speed + public static final double BATCH_BOUNDING_BOX_CACHE_RATIO = 1; + + // ====================================== + // Cold start setting + // ====================================== + public static int MAX_COLD_START_ROUNDS = 2; + + // Thresholding + public static final double THRESHOLD_MIN_PVALUE = 0.995; + + // ====================================== + // Cold start setting + // ====================================== + public static final Setting MAX_RETRY_FOR_UNRESPONSIVE_NODE = Setting + .intSetting("plugins.timeseries.max_retry_for_unresponsive_node", 5, 0, Setting.Property.NodeScope, Setting.Property.Dynamic); + + public static final Setting BACKOFF_MINUTES = Setting + .positiveTimeSetting( + "plugins.timeseries.backoff_minutes", + TimeValue.timeValueMinutes(15), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting COOLDOWN_MINUTES = Setting + .positiveTimeSetting( + "plugins.timeseries.cooldown_minutes", + TimeValue.timeValueMinutes(5), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + // ====================================== + // AD Index setting + // ====================================== + public static int MAX_UPDATE_RETRY_TIMES = 10_000; } diff --git a/src/main/java/org/opensearch/ad/util/DiscoveryNodeFilterer.java b/src/main/java/org/opensearch/timeseries/util/DiscoveryNodeFilterer.java similarity index 98% rename from src/main/java/org/opensearch/ad/util/DiscoveryNodeFilterer.java rename to src/main/java/org/opensearch/timeseries/util/DiscoveryNodeFilterer.java index 69407a501..ca3ba4eba 100644 --- a/src/main/java/org/opensearch/ad/util/DiscoveryNodeFilterer.java +++ b/src/main/java/org/opensearch/timeseries/util/DiscoveryNodeFilterer.java @@ -9,7 +9,7 @@ * GitHub history for details. */ -package org.opensearch.ad.util; +package org.opensearch.timeseries.util; import java.util.ArrayList; import java.util.List; diff --git a/src/main/resources/mappings/forecast-checkpoint.json b/src/main/resources/mappings/forecast-checkpoint.json new file mode 100644 index 000000000..e3337c643 --- /dev/null +++ b/src/main/resources/mappings/forecast-checkpoint.json @@ -0,0 +1,64 @@ +{ + "dynamic": true, + "_meta": { + "schema_version": 1 + }, + "properties": { + "forecaster_id": { + "type": "keyword" + }, + "timestamp": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "schema_version": { + "type": "integer" + }, + "entity": { + "type": "nested", + "properties": { + "name": { + "type": "keyword" + }, + "value": { + "type": "keyword" + } + } + }, + "model": { + "type": "binary" + }, + "samples": { + "type": "nested", + "properties": { + "value_list": { + "type": "double" + }, + "data_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "data_end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + }, + "last_processed_sample": { + "type": "nested", + "properties": { + "value_list": { + "type": "double" + }, + "data_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "data_end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + } + } +} diff --git a/src/main/resources/mappings/forecast-results.json b/src/main/resources/mappings/forecast-results.json new file mode 100644 index 000000000..124fded11 --- /dev/null +++ b/src/main/resources/mappings/forecast-results.json @@ -0,0 +1,124 @@ +{ + "dynamic": true, + "_meta": { + "schema_version": 1 + }, + "properties": { + "forecaster_id": { + "type": "keyword" + }, + "feature_data": { + "type": "nested", + "properties": { + "feature_id": { + "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" + }, + "forecast_series": { + "type": "nested", + "properties": { + "lower_bound": { + "type": "double" + }, + "upper_bound": { + "type": "double" + }, + "feature_id": { + "type": "keyword" + }, + "value": { + "type": "double" + }, + "data_start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "data_end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + } + } +} diff --git a/src/main/resources/mappings/forecast-state.json b/src/main/resources/mappings/forecast-state.json new file mode 100644 index 000000000..59c95a76d --- /dev/null +++ b/src/main/resources/mappings/forecast-state.json @@ -0,0 +1,133 @@ +{ + "dynamic": false, + "_meta": { + "schema_version": 1 + }, + "properties": { + "schema_version": { + "type": "integer" + }, + "last_update_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "error": { + "type": "text" + }, + "started_by": { + "type": "keyword" + }, + "stopped_by": { + "type": "keyword" + }, + "forecaster_id": { + "type": "keyword" + }, + "state": { + "type": "keyword" + }, + "task_progress": { + "type": "float" + }, + "init_progress": { + "type": "float" + }, + "current_piece": { + "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" + }, + "is_latest": { + "type": "boolean" + }, + "task_type": { + "type": "keyword" + }, + "checkpoint_id": { + "type": "keyword" + }, + "coordinating_node": { + "type": "keyword" + }, + "worker_node": { + "type": "keyword" + }, + "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" + } + } + } + } + }, + "forecaster": { + FORECASTER_INDEX_MAPPING_PLACE_HOLDER + }, + "date_range": { + "properties": { + "start_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + }, + "end_time": { + "type": "date", + "format": "strict_date_time||epoch_millis" + } + } + }, + "parent_task_id": { + "type": "keyword" + }, + "entity": { + "type": "nested", + "properties": { + "name": { + "type": "keyword" + }, + "value": { + "type": "keyword" + } + } + }, + "estimated_minutes_left": { + "type": "integer" + } + } +} diff --git a/src/test/java/org/opensearch/action/admin/indices/mapping/get/IndexAnomalyDetectorActionHandlerTests.java b/src/test/java/org/opensearch/action/admin/indices/mapping/get/IndexAnomalyDetectorActionHandlerTests.java index e92f56766..e2904c319 100644 --- a/src/test/java/org/opensearch/action/admin/indices/mapping/get/IndexAnomalyDetectorActionHandlerTests.java +++ b/src/test/java/org/opensearch/action/admin/indices/mapping/get/IndexAnomalyDetectorActionHandlerTests.java @@ -46,7 +46,7 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.NodeStateManager; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorActionHandler; import org.opensearch.ad.task.ADTaskManager; @@ -86,7 +86,7 @@ public class IndexAnomalyDetectorActionHandlerTests extends AbstractTimeSeriesTe private SecurityClientUtil clientUtil; private TransportService transportService; private ActionListener channel; - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; private String detectorId; private Long seqNo; private Long primaryTerm; @@ -129,8 +129,8 @@ public void setUp() throws Exception { channel = mock(ActionListener.class); - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); - when(anomalyDetectionIndices.doesAnomalyDetectorIndexExist()).thenReturn(true); + anomalyDetectionIndices = mock(ADIndexManagement.class); + when(anomalyDetectionIndices.doesConfigIndexExist()).thenReturn(true); detectorId = "123"; seqNo = 0L; diff --git a/src/test/java/org/opensearch/action/admin/indices/mapping/get/ValidateAnomalyDetectorActionHandlerTests.java b/src/test/java/org/opensearch/action/admin/indices/mapping/get/ValidateAnomalyDetectorActionHandlerTests.java index f79f38d3c..2869943b6 100644 --- a/src/test/java/org/opensearch/action/admin/indices/mapping/get/ValidateAnomalyDetectorActionHandlerTests.java +++ b/src/test/java/org/opensearch/action/admin/indices/mapping/get/ValidateAnomalyDetectorActionHandlerTests.java @@ -34,7 +34,7 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.NodeStateManager; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.rest.handler.AbstractAnomalyDetectorActionHandler; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorActionHandler; @@ -64,7 +64,7 @@ public class ValidateAnomalyDetectorActionHandlerTests extends AbstractTimeSerie protected ClusterService clusterService; protected ActionListener channel; protected TransportService transportService; - protected AnomalyDetectionIndices anomalyDetectionIndices; + protected ADIndexManagement anomalyDetectionIndices; protected String detectorId; protected Long seqNo; protected Long primaryTerm; @@ -98,8 +98,8 @@ public void setUp() throws Exception { channel = mock(ActionListener.class); transportService = mock(TransportService.class); - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); - when(anomalyDetectionIndices.doesAnomalyDetectorIndexExist()).thenReturn(true); + anomalyDetectionIndices = mock(ADIndexManagement.class); + when(anomalyDetectionIndices.doesConfigIndexExist()).thenReturn(true); detectorId = "123"; seqNo = 0L; diff --git a/src/test/java/org/opensearch/ad/ADIntegTestCase.java b/src/test/java/org/opensearch/ad/ADIntegTestCase.java index db3295ce7..a1b8daacf 100644 --- a/src/test/java/org/opensearch/ad/ADIntegTestCase.java +++ b/src/test/java/org/opensearch/ad/ADIntegTestCase.java @@ -41,7 +41,7 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.mock.plugin.MockReindexPlugin; import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.AnomalyDetector; @@ -105,7 +105,7 @@ public void setUp() throws Exception { public void createDetectors(List detectors, boolean createIndexFirst) throws IOException { if (createIndexFirst) { - createIndex(CommonName.CONFIG_INDEX, AnomalyDetectionIndices.getAnomalyDetectorMappings()); + createIndex(CommonName.CONFIG_INDEX, ADIndexManagement.getConfigMappings()); } for (AnomalyDetector detector : detectors) { @@ -129,19 +129,19 @@ public String createADTask(ADTask adTask) throws IOException { } public void createDetectorIndex() throws IOException { - createIndex(CommonName.CONFIG_INDEX, AnomalyDetectionIndices.getAnomalyDetectorMappings()); + createIndex(CommonName.CONFIG_INDEX, ADIndexManagement.getConfigMappings()); } public void createADResultIndex() throws IOException { - createIndex(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, AnomalyDetectionIndices.getAnomalyResultMappings()); + createIndex(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, ADIndexManagement.getResultMappings()); } public void createCustomADResultIndex(String indexName) throws IOException { - createIndex(indexName, AnomalyDetectionIndices.getAnomalyResultMappings()); + createIndex(indexName, ADIndexManagement.getResultMappings()); } public void createDetectionStateIndex() throws IOException { - createIndex(ADCommonName.DETECTION_STATE_INDEX, AnomalyDetectionIndices.getDetectionStateMappings()); + createIndex(ADCommonName.DETECTION_STATE_INDEX, ADIndexManagement.getStateMappings()); } public void createTestDataIndex(String indexName) { diff --git a/src/test/java/org/opensearch/ad/AbstractProfileRunnerTests.java b/src/test/java/org/opensearch/ad/AbstractProfileRunnerTests.java index cd63cda8c..f28de4547 100644 --- a/src/test/java/org/opensearch/ad/AbstractProfileRunnerTests.java +++ b/src/test/java/org/opensearch/ad/AbstractProfileRunnerTests.java @@ -36,7 +36,6 @@ import org.opensearch.ad.model.DetectorProfileName; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.ad.transport.AnomalyResultTests; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.ad.util.SecurityClientUtil; import org.opensearch.client.Client; import org.opensearch.cluster.ClusterName; @@ -46,6 +45,7 @@ import org.opensearch.common.transport.TransportAddress; import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.TestHelpers; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; public class AbstractProfileRunnerTests extends AbstractTimeSeriesTest { diff --git a/src/test/java/org/opensearch/ad/AnomalyDetectorJobRunnerTests.java b/src/test/java/org/opensearch/ad/AnomalyDetectorJobRunnerTests.java index 8f43de44c..751c7992a 100644 --- a/src/test/java/org/opensearch/ad/AnomalyDetectorJobRunnerTests.java +++ b/src/test/java/org/opensearch/ad/AnomalyDetectorJobRunnerTests.java @@ -54,7 +54,7 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; import org.opensearch.ad.model.AnomalyResult; @@ -65,7 +65,6 @@ import org.opensearch.ad.transport.AnomalyResultResponse; import org.opensearch.ad.transport.handler.AnomalyIndexHandler; import org.opensearch.ad.util.ClientUtil; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.bytes.BytesReference; @@ -92,6 +91,7 @@ import org.opensearch.timeseries.constant.CommonName; import org.opensearch.timeseries.model.FeatureData; import org.opensearch.timeseries.model.IntervalTimeConfiguration; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import com.google.common.collect.ImmutableList; @@ -143,7 +143,7 @@ public class AnomalyDetectorJobRunnerTests extends AbstractTimeSeriesTest { @Mock private NodeStateManager nodeStateManager; - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; @BeforeClass public static void setUpBeforeClass() { @@ -181,7 +181,7 @@ public void setup() throws Exception { runner.setSettings(settings); - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + anomalyDetectionIndices = mock(ADIndexManagement.class); runner.setAnomalyDetectionIndices(anomalyDetectionIndices); diff --git a/src/test/java/org/opensearch/ad/MultiEntityProfileRunnerTests.java b/src/test/java/org/opensearch/ad/MultiEntityProfileRunnerTests.java index 3d8eb7617..80ef180ed 100644 --- a/src/test/java/org/opensearch/ad/MultiEntityProfileRunnerTests.java +++ b/src/test/java/org/opensearch/ad/MultiEntityProfileRunnerTests.java @@ -66,6 +66,7 @@ import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.TestHelpers; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; public class MultiEntityProfileRunnerTests extends AbstractTimeSeriesTest { diff --git a/src/test/java/org/opensearch/ad/cluster/ADDataMigratorTests.java b/src/test/java/org/opensearch/ad/cluster/ADDataMigratorTests.java index 64a786b32..dcfc14298 100644 --- a/src/test/java/org/opensearch/ad/cluster/ADDataMigratorTests.java +++ b/src/test/java/org/opensearch/ad/cluster/ADDataMigratorTests.java @@ -34,7 +34,7 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.ShardSearchFailure; import org.opensearch.ad.ADUnitTestCase; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; @@ -53,7 +53,7 @@ public class ADDataMigratorTests extends ADUnitTestCase { private Client client; private ClusterService clusterService; private NamedXContentRegistry namedXContentRegistry; - private AnomalyDetectionIndices detectionIndices; + private ADIndexManagement detectionIndices; private ADDataMigrator adDataMigrator; private String detectorId; private String taskId; @@ -69,7 +69,7 @@ public void setUp() throws Exception { client = mock(Client.class); clusterService = mock(ClusterService.class); namedXContentRegistry = TestHelpers.xContentRegistry(); - detectionIndices = mock(AnomalyDetectionIndices.class); + detectionIndices = mock(ADIndexManagement.class); detectorId = randomAlphaOfLength(10); taskId = randomAlphaOfLength(10); detectorContent = "{\"_index\":\".opendistro-anomaly-detectors\",\"_type\":\"_doc\",\"_id\":\"" @@ -104,8 +104,8 @@ public void setUp() throws Exception { } public void testMigrateDataWithNullJobResponse() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(1); @@ -118,8 +118,8 @@ public void testMigrateDataWithNullJobResponse() { } public void testMigrateDataWithInitingDetectionStateIndexFailure() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(false); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(false); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(0); @@ -138,8 +138,8 @@ public void testMigrateDataWithInitingDetectionStateIndexFailure() { } public void testMigrateDataWithInitingDetectionStateIndexAlreadyExists() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(false); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(false); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(0); @@ -158,8 +158,8 @@ public void testMigrateDataWithInitingDetectionStateIndexAlreadyExists() { } public void testMigrateDataWithInitingDetectionStateIndexNotAcknowledged() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(false); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(false); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(0); @@ -178,8 +178,8 @@ public void testMigrateDataWithInitingDetectionStateIndexNotAcknowledged() { } public void testMigrateDataWithInitingDetectionStateIndexAcknowledged() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(false); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(false); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(0); @@ -198,8 +198,8 @@ public void testMigrateDataWithInitingDetectionStateIndexAcknowledged() { } public void testMigrateDataWithEmptyJobResponse() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(1); @@ -232,8 +232,8 @@ public void testMigrateDataWithEmptyJobResponse() { } public void testMigrateDataWithNormalJobResponseButMissingDetector() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); doAnswer(invocation -> { // Return correct AD job when search job index @@ -282,8 +282,8 @@ public void testMigrateDataWithNormalJobResponseButMissingDetector() { } public void testMigrateDataWithNormalJobResponseAndExistingDetector() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); String detectorId = randomAlphaOfLength(10); doAnswer(invocation -> { @@ -349,8 +349,8 @@ public void testMigrateDataWithNormalJobResponseAndExistingDetector() { } public void testMigrateDataWithNormalJobResponse_ExistingDetector_ExistingInternalError() { - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); String detectorId = randomAlphaOfLength(10); doAnswer(invocation -> { @@ -420,7 +420,7 @@ public void testMigrateDataWithNormalJobResponse_ExistingDetector_ExistingIntern public void testMigrateDataTwice() { adDataMigrator.migrateData(); adDataMigrator.migrateData(); - verify(detectionIndices, times(1)).doesAnomalyDetectorJobIndexExist(); + verify(detectionIndices, times(1)).doesJobIndexExist(); } public void testMigrateDataWithNoAvailableShardsException() { @@ -432,8 +432,8 @@ public void testMigrateDataWithNoAvailableShardsException() { ); return null; }).when(client).search(any(), any()); - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); adDataMigrator.migrateData(); assertFalse(adDataMigrator.isMigrated()); @@ -445,8 +445,8 @@ public void testMigrateDataWithIndexNotFoundException() { listener.onFailure(new IndexNotFoundException(CommonName.JOB_INDEX)); return null; }).when(client).search(any(), any()); - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); adDataMigrator.migrateData(); verify(adDataMigrator, never()).backfillRealtimeTask(any(), anyBoolean()); @@ -459,8 +459,8 @@ public void testMigrateDataWithUnknownException() { listener.onFailure(new RuntimeException("test unknown exception")); return null; }).when(client).search(any(), any()); - when(detectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(true); + when(detectionIndices.doesJobIndexExist()).thenReturn(true); + when(detectionIndices.doesStateIndexExist()).thenReturn(true); adDataMigrator.migrateData(); verify(adDataMigrator, never()).backfillRealtimeTask(any(), anyBoolean()); diff --git a/src/test/java/org/opensearch/ad/cluster/ClusterManagerEventListenerTests.java b/src/test/java/org/opensearch/ad/cluster/ClusterManagerEventListenerTests.java index 6dcda085a..637c5e10e 100644 --- a/src/test/java/org/opensearch/ad/cluster/ClusterManagerEventListenerTests.java +++ b/src/test/java/org/opensearch/ad/cluster/ClusterManagerEventListenerTests.java @@ -31,7 +31,6 @@ import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.util.ClientUtil; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.component.LifecycleListener; @@ -41,6 +40,7 @@ import org.opensearch.threadpool.Scheduler.Cancellable; import org.opensearch.threadpool.ThreadPool; import org.opensearch.timeseries.AbstractTimeSeriesTest; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class ClusterManagerEventListenerTests extends AbstractTimeSeriesTest { private ClusterService clusterService; diff --git a/src/test/java/org/opensearch/ad/cluster/HashRingTests.java b/src/test/java/org/opensearch/ad/cluster/HashRingTests.java index cd94385c7..b6474678c 100644 --- a/src/test/java/org/opensearch/ad/cluster/HashRingTests.java +++ b/src/test/java/org/opensearch/ad/cluster/HashRingTests.java @@ -39,7 +39,6 @@ import org.opensearch.ad.ADUnitTestCase; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.ml.ModelManager; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.client.ClusterAdminClient; @@ -51,6 +50,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.plugins.PluginInfo; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; diff --git a/src/test/java/org/opensearch/ad/cluster/HourlyCronTests.java b/src/test/java/org/opensearch/ad/cluster/HourlyCronTests.java index 7e5f6c83e..a3d151c0f 100644 --- a/src/test/java/org/opensearch/ad/cluster/HourlyCronTests.java +++ b/src/test/java/org/opensearch/ad/cluster/HourlyCronTests.java @@ -32,7 +32,6 @@ import org.opensearch.ad.transport.CronAction; import org.opensearch.ad.transport.CronNodeResponse; import org.opensearch.ad.transport.CronResponse; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; @@ -40,6 +39,7 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.timeseries.AbstractTimeSeriesTest; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import test.org.opensearch.ad.util.ClusterCreation; diff --git a/src/test/java/org/opensearch/ad/feature/NoPowermockSearchFeatureDaoTests.java b/src/test/java/org/opensearch/ad/feature/NoPowermockSearchFeatureDaoTests.java index c5619eae0..1d0da6d19 100644 --- a/src/test/java/org/opensearch/ad/feature/NoPowermockSearchFeatureDaoTests.java +++ b/src/test/java/org/opensearch/ad/feature/NoPowermockSearchFeatureDaoTests.java @@ -61,12 +61,12 @@ import org.opensearch.ad.util.SecurityClientUtil; import org.opensearch.client.Client; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasables; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; -import org.opensearch.core.common.lease.Releasables; import org.opensearch.index.mapper.DateFieldMapper; import org.opensearch.index.query.QueryBuilders; import org.opensearch.indices.breaker.NoneCircuitBreakerService; diff --git a/src/test/java/org/opensearch/ad/indices/AnomalyDetectionIndicesTests.java b/src/test/java/org/opensearch/ad/indices/AnomalyDetectionIndicesTests.java index c94420c19..d67dd1b01 100644 --- a/src/test/java/org/opensearch/ad/indices/AnomalyDetectionIndicesTests.java +++ b/src/test/java/org/opensearch/ad/indices/AnomalyDetectionIndicesTests.java @@ -16,27 +16,20 @@ import java.util.Collections; import org.junit.Before; -import org.opensearch.action.index.IndexRequest; -import org.opensearch.action.index.IndexResponse; import org.opensearch.ad.AnomalyDetectorPlugin; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.model.AnomalyDetector; -import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.plugins.Plugin; -import org.opensearch.rest.RestStatus; -import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.timeseries.TestHelpers; import org.opensearch.timeseries.constant.CommonName; -import org.opensearch.timeseries.util.RestHandlerUtils; +import org.opensearch.timeseries.indices.IndexManagementIntegTestCase; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; -public class AnomalyDetectionIndicesTests extends OpenSearchIntegTestCase { +public class AnomalyDetectionIndicesTests extends IndexManagementIntegTestCase { - private AnomalyDetectionIndices indices; + private ADIndexManagement indices; private Settings settings; private DiscoveryNodeFilterer nodeFilter; @@ -48,34 +41,34 @@ protected Collection> nodePlugins() { } @Before - public void setup() { + public void setup() throws IOException { settings = Settings .builder() .put("plugins.anomaly_detection.ad_result_history_rollover_period", TimeValue.timeValueHours(12)) - .put("plugins.anomaly_detection.ad_result_history_max_age", TimeValue.timeValueHours(24)) + .put("plugins.anomaly_detection.ad_result_history_retention_period", TimeValue.timeValueHours(24)) .put("plugins.anomaly_detection.ad_result_history_max_docs", 10000L) .put("plugins.anomaly_detection.request_timeout", TimeValue.timeValueSeconds(10)) .build(); nodeFilter = new DiscoveryNodeFilterer(clusterService()); - indices = new AnomalyDetectionIndices( + indices = new ADIndexManagement( client(), clusterService(), client().threadPool(), settings, nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES ); } public void testAnomalyDetectorIndexNotExists() { - boolean exists = indices.doesAnomalyDetectorIndexExist(); + boolean exists = indices.doesConfigIndexExist(); assertFalse(exists); } public void testAnomalyDetectorIndexExists() throws IOException { - indices.initAnomalyDetectorIndexIfAbsent(TestHelpers.createActionListener(response -> { + indices.initConfigIndexIfAbsent(TestHelpers.createActionListener(response -> { boolean acknowledged = response.isAcknowledged(); assertTrue(acknowledged); }, failure -> { throw new RuntimeException("should not recreate index"); })); @@ -84,7 +77,7 @@ public void testAnomalyDetectorIndexExists() throws IOException { public void testAnomalyDetectorIndexExistsAndNotRecreate() throws IOException { indices - .initAnomalyDetectorIndexIfAbsent( + .initConfigIndexIfAbsent( TestHelpers .createActionListener( response -> response.isAcknowledged(), @@ -94,7 +87,7 @@ public void testAnomalyDetectorIndexExistsAndNotRecreate() throws IOException { TestHelpers.waitForIndexCreationToComplete(client(), CommonName.CONFIG_INDEX); if (client().admin().indices().prepareExists(CommonName.CONFIG_INDEX).get().isExists()) { indices - .initAnomalyDetectorIndexIfAbsent( + .initConfigIndexIfAbsent( TestHelpers .createActionListener( response -> { throw new RuntimeException("should not recreate index " + CommonName.CONFIG_INDEX); }, @@ -105,12 +98,12 @@ public void testAnomalyDetectorIndexExistsAndNotRecreate() throws IOException { } public void testAnomalyResultIndexNotExists() { - boolean exists = indices.doesDefaultAnomalyResultIndexExist(); + boolean exists = indices.doesDefaultResultIndexExist(); assertFalse(exists); } public void testAnomalyResultIndexExists() throws IOException { - indices.initDefaultAnomalyResultIndexIfAbsent(TestHelpers.createActionListener(response -> { + indices.initDefaultResultIndexIfAbsent(TestHelpers.createActionListener(response -> { boolean acknowledged = response.isAcknowledged(); assertTrue(acknowledged); }, failure -> { throw new RuntimeException("should not recreate index"); })); @@ -119,7 +112,7 @@ public void testAnomalyResultIndexExists() throws IOException { public void testAnomalyResultIndexExistsAndNotRecreate() throws IOException { indices - .initDefaultAnomalyResultIndexIfAbsent( + .initDefaultResultIndexIfAbsent( TestHelpers .createActionListener( response -> logger.info("Acknowledged: " + response.isAcknowledged()), @@ -129,7 +122,7 @@ public void testAnomalyResultIndexExistsAndNotRecreate() throws IOException { TestHelpers.waitForIndexCreationToComplete(client(), ADCommonName.ANOMALY_RESULT_INDEX_ALIAS); if (client().admin().indices().prepareExists(ADCommonName.ANOMALY_RESULT_INDEX_ALIAS).get().isExists()) { indices - .initDefaultAnomalyResultIndexIfAbsent( + .initDefaultResultIndexIfAbsent( TestHelpers .createActionListener( response -> { @@ -143,22 +136,25 @@ public void testAnomalyResultIndexExistsAndNotRecreate() throws IOException { } } - private void createRandomDetector(String indexName) throws IOException { - // creates a random anomaly detector and indexes it - AnomalyDetector detector = TestHelpers.randomAnomalyDetector(TestHelpers.randomUiMetadata(), null); - - XContentBuilder xContentBuilder = XContentFactory.jsonBuilder(); - detector.toXContent(xContentBuilder, RestHandlerUtils.XCONTENT_WITH_TYPE); - - IndexResponse indexResponse = client().index(new IndexRequest(indexName).source(xContentBuilder)).actionGet(); - assertEquals("Doc was not created", RestStatus.CREATED, indexResponse.status()); - } - public void testGetDetectionStateIndexMapping() throws IOException { - String detectorIndexMappings = AnomalyDetectionIndices.getAnomalyDetectorMappings(); + String detectorIndexMappings = ADIndexManagement.getConfigMappings(); detectorIndexMappings = detectorIndexMappings .substring(detectorIndexMappings.indexOf("\"properties\""), detectorIndexMappings.lastIndexOf("}")); - String detectionStateIndexMapping = AnomalyDetectionIndices.getDetectionStateMappings(); + String detectionStateIndexMapping = ADIndexManagement.getStateMappings(); assertTrue(detectionStateIndexMapping.contains(detectorIndexMappings)); } + + public void testValidateCustomIndexForBackendJob() throws IOException, InterruptedException { + String resultMapping = ADIndexManagement.getResultMappings(); + + validateCustomIndexForBackendJob(indices, resultMapping); + } + + public void testValidateCustomIndexForBackendJobInvalidMapping() { + validateCustomIndexForBackendJobInvalidMapping(indices); + } + + public void testValidateCustomIndexForBackendJobNoIndex() { + validateCustomIndexForBackendJobNoIndex(indices); + } } diff --git a/src/test/java/org/opensearch/ad/indices/CustomIndexTests.java b/src/test/java/org/opensearch/ad/indices/CustomIndexTests.java index 7959330f4..9e4741c1b 100644 --- a/src/test/java/org/opensearch/ad/indices/CustomIndexTests.java +++ b/src/test/java/org/opensearch/ad/indices/CustomIndexTests.java @@ -24,7 +24,6 @@ import org.opensearch.Version; import org.opensearch.ad.model.AnomalyResult; import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; @@ -36,9 +35,11 @@ import org.opensearch.common.settings.Settings; import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class CustomIndexTests extends AbstractTimeSeriesTest { - AnomalyDetectionIndices adIndices; + ADIndexManagement adIndices; Client client; ClusterService clusterService; DiscoveryNodeFilterer nodeFilter; @@ -71,7 +72,7 @@ public void setUp() throws Exception { AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD, AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS ) ) ) @@ -81,13 +82,13 @@ public void setUp() throws Exception { nodeFilter = mock(DiscoveryNodeFilterer.class); - adIndices = new AnomalyDetectionIndices( + adIndices = new ADIndexManagement( client, clusterService, threadPool, settings, nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES ); } diff --git a/src/test/java/org/opensearch/ad/indices/InitAnomalyDetectionIndicesTests.java b/src/test/java/org/opensearch/ad/indices/InitAnomalyDetectionIndicesTests.java index fd92b6573..686710e9d 100644 --- a/src/test/java/org/opensearch/ad/indices/InitAnomalyDetectionIndicesTests.java +++ b/src/test/java/org/opensearch/ad/indices/InitAnomalyDetectionIndicesTests.java @@ -31,20 +31,20 @@ import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.client.IndicesAdminClient; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.threadpool.ThreadPool; import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class InitAnomalyDetectionIndicesTests extends AbstractTimeSeriesTest { Client client; @@ -52,7 +52,7 @@ public class InitAnomalyDetectionIndicesTests extends AbstractTimeSeriesTest { ThreadPool threadPool; Settings settings; DiscoveryNodeFilterer nodeFilter; - AnomalyDetectionIndices adIndices; + ADIndexManagement adIndices; ClusterName clusterName; ClusterState clusterState; IndicesAdminClient indicesClient; @@ -86,7 +86,7 @@ public void setUp() throws Exception { AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD, AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS ) ) ) @@ -97,13 +97,13 @@ public void setUp() throws Exception { clusterState = ClusterState.builder(clusterName).metadata(Metadata.builder().build()).build(); when(clusterService.state()).thenReturn(clusterState); - adIndices = new AnomalyDetectionIndices( + adIndices = new ADIndexManagement( client, clusterService, threadPool, settings, nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES ); } @@ -121,7 +121,7 @@ private void fixedPrimaryShardsIndexCreationTemplate(String index) throws IOExce ActionListener listener = mock(ActionListener.class); if (index.equals(CommonName.CONFIG_INDEX)) { - adIndices.initAnomalyDetectorIndexIfAbsent(listener); + adIndices.initConfigIndexIfAbsent(listener); } else { adIndices.initDetectionStateIndex(listener); } @@ -133,22 +133,24 @@ private void fixedPrimaryShardsIndexCreationTemplate(String index) throws IOExce } @SuppressWarnings("unchecked") - private void fixedPrimaryShardsIndexNoCreationTemplate(String index, String alias) throws IOException { + private void fixedPrimaryShardsIndexNoCreationTemplate(String index, String... alias) throws IOException { clusterState = mock(ClusterState.class); when(clusterService.state()).thenReturn(clusterState); - RoutingTable.Builder rb = RoutingTable.builder(); - rb.addAsNew(indexMeta(index, 1L)); - when(clusterState.getRoutingTable()).thenReturn(rb.build()); + // RoutingTable.Builder rb = RoutingTable.builder(); + // rb.addAsNew(indexMeta(index, 1L)); + // when(clusterState.metadata()).thenReturn(rb.build()); Metadata.Builder mb = Metadata.builder(); - mb.put(indexMeta(".opendistro-anomaly-results-history-2020.06.24-000003", 1L, ADCommonName.ANOMALY_RESULT_INDEX_ALIAS), true); + // mb.put(indexMeta(".opendistro-anomaly-results-history-2020.06.24-000003", 1L, ADCommonName.ANOMALY_RESULT_INDEX_ALIAS), true); + mb.put(indexMeta(index, 1L, alias), true); + when(clusterState.metadata()).thenReturn(mb.build()); ActionListener listener = mock(ActionListener.class); if (index.equals(CommonName.CONFIG_INDEX)) { - adIndices.initAnomalyDetectorIndexIfAbsent(listener); + adIndices.initConfigIndexIfAbsent(listener); } else { - adIndices.initDefaultAnomalyResultIndexIfAbsent(listener); + adIndices.initDefaultResultIndexIfAbsent(listener); } verify(indicesClient, never()).create(any(), any()); @@ -180,15 +182,15 @@ private void adaptivePrimaryShardsIndexCreationTemplate(String index) throws IOE ActionListener listener = mock(ActionListener.class); if (index.equals(CommonName.CONFIG_INDEX)) { - adIndices.initAnomalyDetectorIndexIfAbsent(listener); + adIndices.initConfigIndexIfAbsent(listener); } else if (index.equals(ADCommonName.DETECTION_STATE_INDEX)) { adIndices.initDetectionStateIndex(listener); } else if (index.equals(ADCommonName.CHECKPOINT_INDEX_NAME)) { adIndices.initCheckpointIndex(listener); } else if (index.equals(CommonName.JOB_INDEX)) { - adIndices.initAnomalyDetectorJobIndex(listener); + adIndices.initJobIndex(listener); } else { - adIndices.initDefaultAnomalyResultIndexIfAbsent(listener); + adIndices.initDefaultResultIndexIfAbsent(listener); } ArgumentCaptor captor = ArgumentCaptor.forClass(CreateIndexResponse.class); @@ -198,11 +200,11 @@ private void adaptivePrimaryShardsIndexCreationTemplate(String index) throws IOE } public void testNotCreateDetector() throws IOException { - fixedPrimaryShardsIndexNoCreationTemplate(CommonName.CONFIG_INDEX, null); + fixedPrimaryShardsIndexNoCreationTemplate(CommonName.CONFIG_INDEX); } public void testNotCreateResult() throws IOException { - fixedPrimaryShardsIndexNoCreationTemplate(CommonName.CONFIG_INDEX, null); + fixedPrimaryShardsIndexNoCreationTemplate(CommonName.CONFIG_INDEX); } public void testCreateDetector() throws IOException { diff --git a/src/test/java/org/opensearch/ad/indices/RolloverTests.java b/src/test/java/org/opensearch/ad/indices/RolloverTests.java index 77d8eeaf4..abb853aef 100644 --- a/src/test/java/org/opensearch/ad/indices/RolloverTests.java +++ b/src/test/java/org/opensearch/ad/indices/RolloverTests.java @@ -36,7 +36,6 @@ import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.client.ClusterAdminClient; @@ -49,9 +48,11 @@ import org.opensearch.common.settings.Settings; import org.opensearch.threadpool.ThreadPool; import org.opensearch.timeseries.AbstractTimeSeriesTest; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class RolloverTests extends AbstractTimeSeriesTest { - private AnomalyDetectionIndices adIndices; + private ADIndexManagement adIndices; private IndicesAdminClient indicesClient; private ClusterAdminClient clusterAdminClient; private ClusterName clusterName; @@ -77,7 +78,7 @@ public void setUp() throws Exception { AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD, AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS ) ) ) @@ -96,13 +97,13 @@ public void setUp() throws Exception { numberOfNodes = 2; when(nodeFilter.getNumberOfEligibleDataNodes()).thenReturn(numberOfNodes); - adIndices = new AnomalyDetectionIndices( + adIndices = new ADIndexManagement( client, clusterService, threadPool, settings, nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES ); clusterAdminClient = mock(ClusterAdminClient.class); @@ -110,7 +111,7 @@ public void setUp() throws Exception { doAnswer(invocation -> { ClusterStateRequest clusterStateRequest = invocation.getArgument(0); - assertEquals(AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN, clusterStateRequest.indices()[0]); + assertEquals(ADIndexManagement.ALL_AD_RESULTS_INDEX_PATTERN, clusterStateRequest.indices()[0]); @SuppressWarnings("unchecked") ActionListener listener = (ActionListener) invocation.getArgument(1); listener.onResponse(new ClusterStateResponse(clusterName, clusterState, true)); @@ -128,7 +129,7 @@ private void assertRolloverRequest(RolloverRequest request) { assertEquals(new MaxDocsCondition(defaultMaxDocs * numberOfNodes), conditions.get(MaxDocsCondition.NAME)); CreateIndexRequest createIndexRequest = request.getCreateIndexRequest(); - assertEquals(AnomalyDetectionIndices.AD_RESULT_HISTORY_INDEX_PATTERN, createIndexRequest.index()); + assertEquals(ADIndexManagement.AD_RESULT_HISTORY_INDEX_PATTERN, createIndexRequest.index()); assertTrue(createIndexRequest.mappings().contains("data_start_time")); } @@ -168,7 +169,7 @@ private void setUpRolloverSuccess() { assertEquals(new MaxDocsCondition(defaultMaxDocs * numberOfNodes), conditions.get(MaxDocsCondition.NAME)); CreateIndexRequest createIndexRequest = request.getCreateIndexRequest(); - assertEquals(AnomalyDetectionIndices.AD_RESULT_HISTORY_INDEX_PATTERN, createIndexRequest.index()); + assertEquals(ADIndexManagement.AD_RESULT_HISTORY_INDEX_PATTERN, createIndexRequest.index()); assertTrue(createIndexRequest.mappings().contains("data_start_time")); listener.onResponse(new RolloverResponse(null, null, Collections.emptyMap(), request.isDryRun(), true, true, true)); return null; diff --git a/src/test/java/org/opensearch/ad/indices/UpdateMappingTests.java b/src/test/java/org/opensearch/ad/indices/UpdateMappingTests.java index 2c19432bf..f53393014 100644 --- a/src/test/java/org/opensearch/ad/indices/UpdateMappingTests.java +++ b/src/test/java/org/opensearch/ad/indices/UpdateMappingTests.java @@ -41,7 +41,6 @@ import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.AdminClient; import org.opensearch.client.Client; import org.opensearch.client.IndicesAdminClient; @@ -57,11 +56,13 @@ import org.opensearch.index.IndexNotFoundException; import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; public class UpdateMappingTests extends AbstractTimeSeriesTest { private static String resultIndexName; - private AnomalyDetectionIndices adIndices; + private ADIndexManagement adIndices; private ClusterService clusterService; private int numberOfNodes; private AdminClient adminClient; @@ -98,7 +99,7 @@ public void setUp() throws Exception { AnomalyDetectorSettings.AD_RESULT_HISTORY_MAX_DOCS_PER_SHARD, AnomalyDetectorSettings.AD_RESULT_HISTORY_ROLLOVER_PERIOD, AnomalyDetectorSettings.AD_RESULT_HISTORY_RETENTION_PERIOD, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS ) ) ) @@ -122,25 +123,24 @@ public void setUp() throws Exception { nodeFilter = mock(DiscoveryNodeFilterer.class); numberOfNodes = 2; when(nodeFilter.getNumberOfEligibleDataNodes()).thenReturn(numberOfNodes); - adIndices = new AnomalyDetectionIndices( + adIndices = new ADIndexManagement( client, clusterService, threadPool, settings, nodeFilter, - AnomalyDetectorSettings.MAX_UPDATE_RETRY_TIMES + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES ); } public void testNoIndexToUpdate() { adIndices.update(); verify(indicesAdminClient, never()).putMapping(any(), any()); - // for an index, we may check doesAliasExists/doesIndexExists and shouldUpdateConcreteIndex - // 1 time for result index since alias does not exist and 2 times for other indices - verify(clusterService, times(9)).state(); + // for an index, we may check doesAliasExists/doesIndexExists + verify(clusterService, times(5)).state(); adIndices.update(); // we will not trigger new check since we have checked all indices before - verify(clusterService, times(9)).state(); + verify(clusterService, times(5)).state(); } @SuppressWarnings({ "serial", "unchecked" }) @@ -165,7 +165,7 @@ public void testUpdateMapping() throws IOException { .numberOfReplicas(0) .putMapping(new MappingMetadata("type", new HashMap() { { - put(AnomalyDetectionIndices.META, new HashMap() { + put(ADIndexManagement.META, new HashMap() { { // version 1 will cause update put(CommonName.SCHEMA_VERSION_FIELD, 1); @@ -298,7 +298,7 @@ public void testFailtoUpdateJobSetting() { } @SuppressWarnings("unchecked") - public void testTooManyUpdate() { + public void testTooManyUpdate() throws IOException { setUpSuccessfulGetJobSetting(); doAnswer(invocation -> { ActionListener listener = (ActionListener) invocation.getArgument(2); @@ -307,7 +307,7 @@ public void testTooManyUpdate() { return null; }).when(indicesAdminClient).updateSettings(any(), any()); - adIndices = new AnomalyDetectionIndices(client, clusterService, threadPool, settings, nodeFilter, 1); + adIndices = new ADIndexManagement(client, clusterService, threadPool, settings, nodeFilter, 1); adIndices.update(); adIndices.update(); diff --git a/src/test/java/org/opensearch/ad/ml/CheckpointDaoTests.java b/src/test/java/org/opensearch/ad/ml/CheckpointDaoTests.java index 7d74b55d7..bd3ca3d5b 100644 --- a/src/test/java/org/opensearch/ad/ml/CheckpointDaoTests.java +++ b/src/test/java/org/opensearch/ad/ml/CheckpointDaoTests.java @@ -95,7 +95,7 @@ import org.opensearch.action.update.UpdateRequest; import org.opensearch.action.update.UpdateResponse; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.util.ClientUtil; import org.opensearch.client.Client; @@ -143,7 +143,7 @@ public class CheckpointDaoTests extends OpenSearchTestCase { private Clock clock; @Mock - private AnomalyDetectionIndices indexUtil; + private ADIndexManagement indexUtil; private Schema trcfSchema; diff --git a/src/test/java/org/opensearch/ad/ml/CheckpointDeleteTests.java b/src/test/java/org/opensearch/ad/ml/CheckpointDeleteTests.java index 8e29e93ac..c94c145cb 100644 --- a/src/test/java/org/opensearch/ad/ml/CheckpointDeleteTests.java +++ b/src/test/java/org/opensearch/ad/ml/CheckpointDeleteTests.java @@ -28,7 +28,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.util.ClientUtil; import org.opensearch.client.Client; import org.opensearch.index.IndexNotFoundException; @@ -64,7 +64,7 @@ private enum DeleteExecutionMode { private Client client; private ClientUtil clientUtil; private Gson gson; - private AnomalyDetectionIndices indexUtil; + private ADIndexManagement indexUtil; private String detectorId; private int maxCheckpointBytes; private GenericObjectPool objectPool; @@ -87,7 +87,7 @@ public void setUp() throws Exception { client = mock(Client.class); clientUtil = mock(ClientUtil.class); gson = null; - indexUtil = mock(AnomalyDetectionIndices.class); + indexUtil = mock(ADIndexManagement.class); detectorId = "123"; maxCheckpointBytes = 1_000_000; diff --git a/src/test/java/org/opensearch/ad/ml/ModelManagerTests.java b/src/test/java/org/opensearch/ad/ml/ModelManagerTests.java index 452354537..531f251ed 100644 --- a/src/test/java/org/opensearch/ad/ml/ModelManagerTests.java +++ b/src/test/java/org/opensearch/ad/ml/ModelManagerTests.java @@ -66,7 +66,6 @@ import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.ratelimit.CheckpointWriteWorker; import org.opensearch.ad.settings.AnomalyDetectorSettings; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; @@ -78,6 +77,7 @@ import org.opensearch.timeseries.common.exception.ResourceNotFoundException; import org.opensearch.timeseries.dataprocessor.LinearUniformImputer; import org.opensearch.timeseries.model.Entity; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.powermock.modules.junit4.PowerMockRunner; import org.powermock.modules.junit4.PowerMockRunnerDelegate; diff --git a/src/test/java/org/opensearch/ad/mock/transport/MockAnomalyDetectorJobTransportActionWithUser.java b/src/test/java/org/opensearch/ad/mock/transport/MockAnomalyDetectorJobTransportActionWithUser.java index 4a6f1eca1..15d37c89d 100644 --- a/src/test/java/org/opensearch/ad/mock/transport/MockAnomalyDetectorJobTransportActionWithUser.java +++ b/src/test/java/org/opensearch/ad/mock/transport/MockAnomalyDetectorJobTransportActionWithUser.java @@ -21,7 +21,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.HandledTransportAction; import org.opensearch.ad.ExecuteADResultResponseRecorder; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.ad.transport.AnomalyDetectorJobRequest; @@ -47,7 +47,7 @@ public class MockAnomalyDetectorJobTransportActionWithUser extends private final Client client; private final ClusterService clusterService; private final Settings settings; - private final AnomalyDetectionIndices anomalyDetectionIndices; + private final ADIndexManagement anomalyDetectionIndices; private final NamedXContentRegistry xContentRegistry; private volatile Boolean filterByEnabled; private ThreadContext.StoredContext context; @@ -62,7 +62,7 @@ public MockAnomalyDetectorJobTransportActionWithUser( Client client, ClusterService clusterService, Settings settings, - AnomalyDetectionIndices anomalyDetectionIndices, + ADIndexManagement anomalyDetectionIndices, NamedXContentRegistry xContentRegistry, ADTaskManager adTaskManager, ExecuteADResultResponseRecorder recorder diff --git a/src/test/java/org/opensearch/ad/ratelimit/CheckpointReadWorkerTests.java b/src/test/java/org/opensearch/ad/ratelimit/CheckpointReadWorkerTests.java index c274f7c01..f23dbd484 100644 --- a/src/test/java/org/opensearch/ad/ratelimit/CheckpointReadWorkerTests.java +++ b/src/test/java/org/opensearch/ad/ratelimit/CheckpointReadWorkerTests.java @@ -51,7 +51,7 @@ import org.opensearch.ad.caching.CacheProvider; import org.opensearch.ad.caching.EntityCache; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.CheckpointDao; import org.opensearch.ad.ml.EntityModel; import org.opensearch.ad.ml.ModelManager; @@ -94,7 +94,7 @@ public class CheckpointReadWorkerTests extends AbstractRateLimitingTest { ModelManager modelManager; EntityColdStartWorker coldstartQueue; ResultWriteWorker resultWriteQueue; - AnomalyDetectionIndices anomalyDetectionIndices; + ADIndexManagement anomalyDetectionIndices; CacheProvider cacheProvider; EntityCache entityCache; EntityFeatureRequest request, request2, request3; @@ -136,7 +136,7 @@ public void setUp() throws Exception { coldstartQueue = mock(EntityColdStartWorker.class); resultWriteQueue = mock(ResultWriteWorker.class); - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + anomalyDetectionIndices = mock(ADIndexManagement.class); cacheProvider = mock(CacheProvider.class); entityCache = mock(EntityCache.class); diff --git a/src/test/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandlerTests.java b/src/test/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandlerTests.java index 953e6cc20..59eba777c 100644 --- a/src/test/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandlerTests.java +++ b/src/test/java/org/opensearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandlerTests.java @@ -38,7 +38,7 @@ import org.opensearch.ad.NodeStateManager; import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.mock.model.MockSimpleLog; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyResult; @@ -50,7 +50,6 @@ import org.opensearch.ad.transport.ProfileAction; import org.opensearch.ad.transport.ProfileResponse; import org.opensearch.ad.transport.handler.AnomalyIndexHandler; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -61,13 +60,14 @@ import org.opensearch.timeseries.common.exception.InternalFailure; import org.opensearch.timeseries.common.exception.ResourceNotFoundException; import org.opensearch.timeseries.model.Feature; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; import com.google.common.collect.ImmutableList; public class IndexAnomalyDetectorJobActionHandlerTests extends OpenSearchTestCase { - private static AnomalyDetectionIndices anomalyDetectionIndices; + private static ADIndexManagement anomalyDetectionIndices; private static String detectorId; private static Long seqNo; private static Long primaryTerm; @@ -94,12 +94,12 @@ public static void setOnce() throws IOException { detectorId = "123"; seqNo = 1L; primaryTerm = 2L; - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + anomalyDetectionIndices = mock(ADIndexManagement.class); xContentRegistry = NamedXContentRegistry.EMPTY; transportService = mock(TransportService.class); requestTimeout = TimeValue.timeValueMinutes(60); - when(anomalyDetectionIndices.doesAnomalyDetectorJobIndexExist()).thenReturn(true); + when(anomalyDetectionIndices.doesJobIndexExist()).thenReturn(true); nodeFilter = mock(DiscoveryNodeFilterer.class); detector = TestHelpers.randomAnomalyDetectorUsingCategoryFields(detectorId, Arrays.asList("a")); diff --git a/src/test/java/org/opensearch/ad/settings/AnomalyDetectorSettingsTests.java b/src/test/java/org/opensearch/ad/settings/AnomalyDetectorSettingsTests.java index 333740322..9ee9a5d37 100644 --- a/src/test/java/org/opensearch/ad/settings/AnomalyDetectorSettingsTests.java +++ b/src/test/java/org/opensearch/ad/settings/AnomalyDetectorSettingsTests.java @@ -95,7 +95,7 @@ public void testAllOpenSearchSettingsReturned() { AnomalyDetectorSettings.MAX_ENTITIES_FOR_PREVIEW, AnomalyDetectorSettings.AD_INDEX_PRESSURE_SOFT_LIMIT, AnomalyDetectorSettings.AD_INDEX_PRESSURE_HARD_LIMIT, - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS, + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS, AnomalyDetectorSettings.FILTER_BY_BACKEND_ROLES, AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE, AnomalyDetectorSettings.BATCH_TASK_PIECE_INTERVAL_SECONDS, @@ -184,7 +184,7 @@ public void testAllLegacyOpenDistroSettingsFallback() { // INDEX_PRESSURE_SOFT_LIMIT does not use legacy setting assertEquals(Float.valueOf(0.6f), AnomalyDetectorSettings.AD_INDEX_PRESSURE_SOFT_LIMIT.get(Settings.EMPTY)); assertEquals( - AnomalyDetectorSettings.MAX_PRIMARY_SHARDS.get(Settings.EMPTY), + AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS.get(Settings.EMPTY), LegacyOpenDistroAnomalyDetectorSettings.MAX_PRIMARY_SHARDS.get(Settings.EMPTY) ); assertEquals( @@ -297,7 +297,7 @@ public void testSettingsGetValue() { assertEquals(LegacyOpenDistroAnomalyDetectorSettings.INDEX_PRESSURE_SOFT_LIMIT.get(settings), Float.valueOf(0.8f)); settings = Settings.builder().put("plugins.anomaly_detection.max_primary_shards", 80).build(); - assertEquals(AnomalyDetectorSettings.MAX_PRIMARY_SHARDS.get(settings), Integer.valueOf(80)); + assertEquals(AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS.get(settings), Integer.valueOf(80)); assertEquals(LegacyOpenDistroAnomalyDetectorSettings.MAX_PRIMARY_SHARDS.get(settings), Integer.valueOf(10)); settings = Settings.builder().put("plugins.anomaly_detection.max_cache_miss_handling_per_second", 79).build(); @@ -372,7 +372,7 @@ public void testSettingsGetValueWithLegacyFallback() { assertEquals(AnomalyDetectorSettings.MAX_ENTITIES_FOR_PREVIEW.get(settings), Integer.valueOf(5)); // INDEX_PRESSURE_SOFT_LIMIT uses default instead of legacy fallback assertEquals(AnomalyDetectorSettings.AD_INDEX_PRESSURE_SOFT_LIMIT.get(settings), Float.valueOf(0.6F)); - assertEquals(AnomalyDetectorSettings.MAX_PRIMARY_SHARDS.get(settings), Integer.valueOf(21)); + assertEquals(AnomalyDetectorSettings.AD_MAX_PRIMARY_SHARDS.get(settings), Integer.valueOf(21)); // MAX_CACHE_MISS_HANDLING_PER_SECOND is removed in the new release assertEquals(LegacyOpenDistroAnomalyDetectorSettings.MAX_CACHE_MISS_HANDLING_PER_SECOND.get(settings), Integer.valueOf(22)); assertEquals(AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE.get(settings), Integer.valueOf(23)); diff --git a/src/test/java/org/opensearch/ad/task/ADTaskManagerTests.java b/src/test/java/org/opensearch/ad/task/ADTaskManagerTests.java index 7858bfae5..cf24daea5 100644 --- a/src/test/java/org/opensearch/ad/task/ADTaskManagerTests.java +++ b/src/test/java/org/opensearch/ad/task/ADTaskManagerTests.java @@ -82,7 +82,7 @@ import org.opensearch.action.update.UpdateResponse; import org.opensearch.ad.ADUnitTestCase; import org.opensearch.ad.cluster.HashRing; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.mock.model.MockSimpleLog; import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.ADTaskAction; @@ -91,7 +91,6 @@ import org.opensearch.ad.model.ADTaskType; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; import org.opensearch.ad.stats.InternalStatNames; import org.opensearch.ad.transport.ADStatsNodeResponse; @@ -100,7 +99,6 @@ import org.opensearch.ad.transport.ADTaskProfileResponse; import org.opensearch.ad.transport.AnomalyDetectorJobResponse; import org.opensearch.ad.transport.ForwardADTaskRequest; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.node.DiscoveryNode; @@ -128,8 +126,10 @@ import org.opensearch.timeseries.TestHelpers; import org.opensearch.timeseries.common.exception.DuplicateTaskException; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.DateRange; import org.opensearch.timeseries.model.Entity; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; @@ -145,7 +145,7 @@ public class ADTaskManagerTests extends ADUnitTestCase { private ClusterService clusterService; private ClusterSettings clusterSettings; private DiscoveryNodeFilterer nodeFilter; - private AnomalyDetectionIndices detectionIndices; + private ADIndexManagement detectionIndices; private ADTaskCacheManager adTaskCacheManager; private HashRing hashRing; private ThreadContext.StoredContext context; @@ -231,7 +231,7 @@ public void setUp() throws Exception { client = mock(Client.class); nodeFilter = mock(DiscoveryNodeFilterer.class); - detectionIndices = mock(AnomalyDetectionIndices.class); + detectionIndices = mock(ADIndexManagement.class); adTaskCacheManager = mock(ADTaskCacheManager.class); hashRing = mock(HashRing.class); transportService = mock(TransportService.class); @@ -329,7 +329,7 @@ public void testCreateTaskIndexNotAcknowledged() throws IOException { listener.onResponse(new CreateIndexResponse(false, false, ANOMALY_RESULT_INDEX_ALIAS)); return null; }).when(detectionIndices).initDetectionStateIndex(any()); - doReturn(false).when(detectionIndices).doesDetectorStateIndexExist(); + doReturn(false).when(detectionIndices).doesStateIndexExist(); AnomalyDetector detector = randomDetector(ImmutableList.of(randomFeature(true)), randomAlphaOfLength(5), 1, randomAlphaOfLength(5)); setupGetDetector(detector); @@ -345,7 +345,7 @@ public void testCreateTaskIndexWithResourceAlreadyExistsException() throws IOExc listener.onFailure(new ResourceAlreadyExistsException("index created")); return null; }).when(detectionIndices).initDetectionStateIndex(any()); - doReturn(false).when(detectionIndices).doesDetectorStateIndexExist(); + doReturn(false).when(detectionIndices).doesStateIndexExist(); AnomalyDetector detector = randomDetector(ImmutableList.of(randomFeature(true)), randomAlphaOfLength(5), 1, randomAlphaOfLength(5)); setupGetDetector(detector); @@ -360,7 +360,7 @@ public void testCreateTaskIndexWithException() throws IOException { listener.onFailure(new RuntimeException(error)); return null; }).when(detectionIndices).initDetectionStateIndex(any()); - doReturn(false).when(detectionIndices).doesDetectorStateIndexExist(); + doReturn(false).when(detectionIndices).doesStateIndexExist(); AnomalyDetector detector = randomDetector(ImmutableList.of(randomFeature(true)), randomAlphaOfLength(5), 1, randomAlphaOfLength(5)); setupGetDetector(detector); @@ -1353,7 +1353,7 @@ public void testDeleteADTasks() { }).when(client).execute(any(), any(), any()); String detectorId = randomAlphaOfLength(5); - AnomalyDetectorFunction function = mock(AnomalyDetectorFunction.class); + ExecutorFunction function = mock(ExecutorFunction.class); ActionListener listener = mock(ActionListener.class); adTaskManager.deleteADTasks(detectorId, function, listener); verify(function, times(1)).execute(); @@ -1378,7 +1378,7 @@ public void testDeleteADTasksWithBulkFailures() { }).when(client).execute(any(), any(), any()); String detectorId = randomAlphaOfLength(5); - AnomalyDetectorFunction function = mock(AnomalyDetectorFunction.class); + ExecutorFunction function = mock(ExecutorFunction.class); ActionListener listener = mock(ActionListener.class); adTaskManager.deleteADTasks(detectorId, function, listener); verify(listener, times(1)).onFailure(any()); @@ -1397,7 +1397,7 @@ public void testDeleteADTasksWithException() { }).when(client).execute(any(), any(), any()); String detectorId = randomAlphaOfLength(5); - AnomalyDetectorFunction function = mock(AnomalyDetectorFunction.class); + ExecutorFunction function = mock(ExecutorFunction.class); ActionListener listener = mock(ActionListener.class); adTaskManager.deleteADTasks(detectorId, function, listener); @@ -1458,7 +1458,7 @@ public void testStartDetectorWithException() throws IOException { DateRange detectionDateRange = randomDetectionDateRange(); User user = null; ActionListener listener = mock(ActionListener.class); - when(detectionIndices.doesDetectorStateIndexExist()).thenReturn(false); + when(detectionIndices.doesStateIndexExist()).thenReturn(false); doThrow(new RuntimeException("test")).when(detectionIndices).initDetectionStateIndex(any()); adTaskManager.startDetector(detector, detectionDateRange, user, transportService, listener); verify(listener, times(1)).onFailure(any()); @@ -1618,7 +1618,7 @@ public void testDeleteTaskDocs() { String detectorId = randomAlphaOfLength(5); SearchRequest searchRequest = mock(SearchRequest.class); - AnomalyDetectorFunction function = mock(AnomalyDetectorFunction.class); + ExecutorFunction function = mock(ExecutorFunction.class); ActionListener listener = mock(ActionListener.class); adTaskManager.deleteTaskDocs(detectorId, searchRequest, function, listener); verify(adTaskCacheManager, times(1)).addDeletedDetectorTask(anyString()); diff --git a/src/test/java/org/opensearch/ad/transport/AnomalyDetectorJobActionTests.java b/src/test/java/org/opensearch/ad/transport/AnomalyDetectorJobActionTests.java index 68dd54c63..afbae1eb1 100644 --- a/src/test/java/org/opensearch/ad/transport/AnomalyDetectorJobActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/AnomalyDetectorJobActionTests.java @@ -26,7 +26,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.support.ActionFilters; import org.opensearch.ad.ExecuteADResultResponseRecorder; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.client.Client; @@ -75,7 +75,7 @@ public void setUp() throws Exception { client, clusterService, indexSettings(), - mock(AnomalyDetectionIndices.class), + mock(ADIndexManagement.class), xContentRegistry(), mock(ADTaskManager.class), mock(ExecuteADResultResponseRecorder.class) diff --git a/src/test/java/org/opensearch/ad/transport/DeleteAnomalyDetectorTests.java b/src/test/java/org/opensearch/ad/transport/DeleteAnomalyDetectorTests.java index b4d8a6b76..d2c3e634c 100644 --- a/src/test/java/org/opensearch/ad/transport/DeleteAnomalyDetectorTests.java +++ b/src/test/java/org/opensearch/ad/transport/DeleteAnomalyDetectorTests.java @@ -38,7 +38,6 @@ import org.opensearch.ad.model.ADTask; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyDetectorJob; -import org.opensearch.ad.rest.handler.AnomalyDetectorFunction; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.task.ADTaskManager; import org.opensearch.client.Client; @@ -57,6 +56,7 @@ import org.opensearch.timeseries.AbstractTimeSeriesTest; import org.opensearch.timeseries.TestHelpers; import org.opensearch.timeseries.constant.CommonName; +import org.opensearch.timeseries.function.ExecutorFunction; import org.opensearch.timeseries.model.IntervalTimeConfiguration; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportService; @@ -248,7 +248,7 @@ private void setupMocks( doAnswer(invocation -> { Object[] args = invocation.getArguments(); - AnomalyDetectorFunction function = (AnomalyDetectorFunction) args[1]; + ExecutorFunction function = (ExecutorFunction) args[1]; function.execute(); return null; diff --git a/src/test/java/org/opensearch/ad/transport/DeleteTests.java b/src/test/java/org/opensearch/ad/transport/DeleteTests.java index 3721f5dec..a8692f232 100644 --- a/src/test/java/org/opensearch/ad/transport/DeleteTests.java +++ b/src/test/java/org/opensearch/ad/transport/DeleteTests.java @@ -40,11 +40,9 @@ import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.PlainActionFuture; -import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.ad.common.exception.JsonPathNotFoundException; import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.client.Client; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.node.DiscoveryNode; @@ -61,6 +59,7 @@ import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.timeseries.AbstractTimeSeriesTest; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.TransportService; import test.org.opensearch.ad.util.ClusterCreation; @@ -200,14 +199,6 @@ public void testJsonRequestDeleteModel() throws IOException, JsonPathNotFoundExc testJsonRequestTemplate(request, request::getAdID); } - public void testNewResponse() throws IOException { - StreamInput input = mock(StreamInput.class); - when(input.readBoolean()).thenReturn(true); - AcknowledgedResponse response = new AcknowledgedResponse(input); - - assertTrue(response.isAcknowledged()); - } - private enum DetectorExecutionMode { DELETE_MODEL_NORMAL, DELETE_MODEL_FAILURE diff --git a/src/test/java/org/opensearch/ad/transport/EntityResultTransportActionTests.java b/src/test/java/org/opensearch/ad/transport/EntityResultTransportActionTests.java index 6b77d7471..30177220b 100644 --- a/src/test/java/org/opensearch/ad/transport/EntityResultTransportActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/EntityResultTransportActionTests.java @@ -58,7 +58,7 @@ import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.constant.CommonValue; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.CheckpointDao; import org.opensearch.ad.ml.EntityColdStarter; import org.opensearch.ad.ml.EntityModel; @@ -131,7 +131,7 @@ public class EntityResultTransportActionTests extends AbstractTimeSeriesTest { EntityColdStarter coldStarter; ColdEntityWorker coldEntityQueue; EntityColdStartWorker entityColdStartQueue; - AnomalyDetectionIndices indexUtil; + ADIndexManagement indexUtil; ClusterService clusterService; ADStats adStats; @@ -232,7 +232,7 @@ public void setUp() throws Exception { coldEntities.add(cacheMissEntityObj); when(entityCache.selectUpdateCandidate(any(), anyString(), any())).thenReturn(Pair.of(new ArrayList<>(), coldEntities)); - indexUtil = mock(AnomalyDetectionIndices.class); + indexUtil = mock(ADIndexManagement.class); when(indexUtil.getSchemaVersion(any())).thenReturn(CommonValue.NO_SCHEMA_VERSION); resultWriteQueue = mock(ResultWriteWorker.class); diff --git a/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTests.java b/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTests.java index 33652ebb4..620ed436f 100644 --- a/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTests.java +++ b/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTests.java @@ -46,7 +46,6 @@ import org.opensearch.ad.model.ADTaskType; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.task.ADTaskManager; -import org.opensearch.ad.util.DiscoveryNodeFilterer; import org.opensearch.ad.util.SecurityClientUtil; import org.opensearch.ad.util.Throttler; import org.opensearch.client.Client; @@ -59,6 +58,7 @@ import org.opensearch.timeseries.constant.CommonMessages; import org.opensearch.timeseries.constant.CommonName; import org.opensearch.timeseries.model.Entity; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportService; diff --git a/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTransportActionTests.java b/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTransportActionTests.java index b0d80272f..316308e55 100644 --- a/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTransportActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/GetAnomalyDetectorTransportActionTests.java @@ -52,6 +52,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.timeseries.TestHelpers; import org.opensearch.timeseries.model.Entity; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; import org.opensearch.timeseries.util.RestHandlerUtils; import org.opensearch.transport.TransportService; diff --git a/src/test/java/org/opensearch/ad/transport/IndexAnomalyDetectorTransportActionTests.java b/src/test/java/org/opensearch/ad/transport/IndexAnomalyDetectorTransportActionTests.java index 587265d4d..0a3859bc2 100644 --- a/src/test/java/org/opensearch/ad/transport/IndexAnomalyDetectorTransportActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/IndexAnomalyDetectorTransportActionTests.java @@ -39,7 +39,7 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.ad.NodeStateManager; import org.opensearch.ad.feature.SearchFeatureDao; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.settings.AnomalyDetectorSettings; import org.opensearch.ad.task.ADTaskManager; @@ -116,7 +116,7 @@ public void setUp() throws Exception { clientUtil, clusterService, indexSettings(), - mock(AnomalyDetectionIndices.class), + mock(ADIndexManagement.class), xContentRegistry(), adTaskManager, searchFeatureDao @@ -214,7 +214,7 @@ public void testIndexTransportActionWithUserAndFilterOn() { clientUtil, clusterService, settings, - mock(AnomalyDetectionIndices.class), + mock(ADIndexManagement.class), xContentRegistry(), adTaskManager, searchFeatureDao @@ -240,7 +240,7 @@ public void testIndexTransportActionWithUserAndFilterOff() { clientUtil, clusterService, settings, - mock(AnomalyDetectionIndices.class), + mock(ADIndexManagement.class), xContentRegistry(), adTaskManager, searchFeatureDao diff --git a/src/test/java/org/opensearch/ad/transport/MultiEntityResultTests.java b/src/test/java/org/opensearch/ad/transport/MultiEntityResultTests.java index cd527fc82..6c2a88240 100644 --- a/src/test/java/org/opensearch/ad/transport/MultiEntityResultTests.java +++ b/src/test/java/org/opensearch/ad/transport/MultiEntityResultTests.java @@ -76,7 +76,7 @@ import org.opensearch.ad.cluster.HashRing; import org.opensearch.ad.feature.CompositeRetriever; import org.opensearch.ad.feature.FeatureManager; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.ModelManager; import org.opensearch.ad.ml.ThresholdingResult; import org.opensearch.ad.model.AnomalyDetector; @@ -160,7 +160,7 @@ public class MultiEntityResultTests extends AbstractTimeSeriesTest { private String detectorId; private Instant now; private CacheProvider provider; - private AnomalyDetectionIndices indexUtil; + private ADIndexManagement indexUtil; private ResultWriteWorker resultWriteQueue; private CheckpointReadWorker checkpointReadQueue; private EntityColdStartWorker entityColdStartQueue; @@ -300,7 +300,7 @@ public void setUp() throws Exception { .thenReturn(MLUtil.randomModelState(new RandomModelStateConfig.Builder().fullModel(true).build())); when(entityCache.selectUpdateCandidate(any(), any(), any())).thenReturn(Pair.of(new ArrayList(), new ArrayList())); - indexUtil = mock(AnomalyDetectionIndices.class); + indexUtil = mock(ADIndexManagement.class); resultWriteQueue = mock(ResultWriteWorker.class); checkpointReadQueue = mock(CheckpointReadWorker.class); entityColdStartQueue = mock(EntityColdStartWorker.class); diff --git a/src/test/java/org/opensearch/ad/transport/PreviewAnomalyDetectorTransportActionTests.java b/src/test/java/org/opensearch/ad/transport/PreviewAnomalyDetectorTransportActionTests.java index 8a9a4d5b4..f2cda2046 100644 --- a/src/test/java/org/opensearch/ad/transport/PreviewAnomalyDetectorTransportActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/PreviewAnomalyDetectorTransportActionTests.java @@ -50,7 +50,7 @@ import org.opensearch.ad.breaker.ADCircuitBreakerService; import org.opensearch.ad.feature.FeatureManager; import org.opensearch.ad.feature.Features; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.ModelManager; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.ad.model.AnomalyResult; @@ -335,7 +335,7 @@ public void onFailure(Exception e) { public void testPreviewTransportActionWithDetector() throws IOException, InterruptedException { final CountDownLatch inProgressLatch = new CountDownLatch(1); CreateIndexResponse createResponse = TestHelpers - .createIndex(client().admin(), CommonName.CONFIG_INDEX, AnomalyDetectionIndices.getAnomalyDetectorMappings()); + .createIndex(client().admin(), CommonName.CONFIG_INDEX, ADIndexManagement.getConfigMappings()); Assert.assertNotNull(createResponse); AnomalyDetector detector = TestHelpers.randomAnomalyDetector(ImmutableMap.of("testKey", "testValue"), Instant.now()); diff --git a/src/test/java/org/opensearch/ad/transport/SearchAnomalyResultActionTests.java b/src/test/java/org/opensearch/ad/transport/SearchAnomalyResultActionTests.java index d5babf203..ac902a55e 100644 --- a/src/test/java/org/opensearch/ad/transport/SearchAnomalyResultActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/SearchAnomalyResultActionTests.java @@ -16,7 +16,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static org.opensearch.ad.indices.AnomalyDetectionIndices.ALL_AD_RESULTS_INDEX_PATTERN; +import static org.opensearch.ad.indices.ADIndexManagement.ALL_AD_RESULTS_INDEX_PATTERN; import static org.opensearch.timeseries.TestHelpers.createClusterState; import static org.opensearch.timeseries.TestHelpers.createSearchResponse; import static org.opensearch.timeseries.TestHelpers.matchAllRequest; diff --git a/src/test/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportActionTests.java b/src/test/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportActionTests.java index 13dd505a2..604fc2c46 100644 --- a/src/test/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportActionTests.java +++ b/src/test/java/org/opensearch/ad/transport/ValidateAnomalyDetectorTransportActionTests.java @@ -21,7 +21,7 @@ import org.opensearch.ad.ADIntegTestCase; import org.opensearch.ad.constant.ADCommonMessages; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyDetector; import org.opensearch.common.unit.TimeValue; import org.opensearch.search.aggregations.AggregationBuilder; @@ -299,7 +299,7 @@ public void testValidateAnomalyDetectorWithCustomResultIndexPresentButNotCreated @Test public void testValidateAnomalyDetectorWithCustomResultIndexWithInvalidMapping() throws IOException { String resultIndex = ADCommonName.CUSTOM_RESULT_INDEX_PREFIX + "test"; - URL url = AnomalyDetectionIndices.class.getClassLoader().getResource("mappings/anomaly-checkpoint.json"); + URL url = ADIndexManagement.class.getClassLoader().getResource("mappings/anomaly-checkpoint.json"); createIndex(resultIndex, Resources.toString(url, Charsets.UTF_8)); AnomalyDetector anomalyDetector = TestHelpers .randomDetector( diff --git a/src/test/java/org/opensearch/ad/transport/handler/AbstractIndexHandlerTest.java b/src/test/java/org/opensearch/ad/transport/handler/AbstractIndexHandlerTest.java index 3746c68e0..4d1c1ed44 100644 --- a/src/test/java/org/opensearch/ad/transport/handler/AbstractIndexHandlerTest.java +++ b/src/test/java/org/opensearch/ad/transport/handler/AbstractIndexHandlerTest.java @@ -28,7 +28,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.ad.constant.ADCommonName; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.transport.AnomalyResultTests; import org.opensearch.ad.util.ClientUtil; import org.opensearch.ad.util.IndexUtils; @@ -62,7 +62,7 @@ enum IndexCreation { protected Client client; @Mock - protected AnomalyDetectionIndices anomalyDetectionIndices; + protected ADIndexManagement anomalyDetectionIndices; @Mock protected Throttler throttler; @@ -137,8 +137,8 @@ protected void setUpSavingAnomalyResultIndex(boolean anomalyResultIndexExists, I break; } return null; - }).when(anomalyDetectionIndices).initDefaultAnomalyResultIndexDirectly(any()); - when(anomalyDetectionIndices.doesDefaultAnomalyResultIndexExist()).thenReturn(anomalyResultIndexExists); + }).when(anomalyDetectionIndices).initDefaultResultIndexDirectly(any()); + when(anomalyDetectionIndices.doesDefaultResultIndexExist()).thenReturn(anomalyResultIndexExists); } protected void setUpSavingAnomalyResultIndex(boolean anomalyResultIndexExists) throws IOException { diff --git a/src/test/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java b/src/test/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java index f799e14cb..0c6ff4727 100644 --- a/src/test/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java +++ b/src/test/java/org/opensearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java @@ -34,7 +34,7 @@ import org.opensearch.action.bulk.BulkResponse; import org.opensearch.action.index.IndexResponse; import org.opensearch.ad.ADUnitTestCase; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.model.AnomalyResult; import org.opensearch.ad.util.ClientUtil; import org.opensearch.ad.util.IndexUtils; @@ -56,12 +56,12 @@ public class AnomalyResultBulkIndexHandlerTests extends ADUnitTestCase { private Client client; private IndexUtils indexUtils; private ActionListener listener; - private AnomalyDetectionIndices anomalyDetectionIndices; + private ADIndexManagement anomalyDetectionIndices; @Override public void setUp() throws Exception { super.setUp(); - anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + anomalyDetectionIndices = mock(ADIndexManagement.class); client = mock(Client.class); Settings settings = Settings.EMPTY; Clock clock = mock(Clock.class); @@ -92,7 +92,7 @@ public void onFailure(Exception e) {} public void testNullAnomalyResults() { bulkIndexHandler.bulkIndexAnomalyResult(null, null, listener); verify(listener, times(1)).onResponse(null); - verify(anomalyDetectionIndices, never()).doesAnomalyDetectorIndexExist(); + verify(anomalyDetectionIndices, never()).doesConfigIndexExist(); } public void testAnomalyResultBulkIndexHandler_IndexNotExist() { @@ -133,7 +133,7 @@ public void testCreateADResultIndexNotAcknowledged() throws IOException { ActionListener listener = invocation.getArgument(0); listener.onResponse(new CreateIndexResponse(false, false, ANOMALY_RESULT_INDEX_ALIAS)); return null; - }).when(anomalyDetectionIndices).initDefaultAnomalyResultIndexDirectly(any()); + }).when(anomalyDetectionIndices).initDefaultResultIndexDirectly(any()); bulkIndexHandler.bulkIndexAnomalyResult(null, ImmutableList.of(mock(AnomalyResult.class)), listener); verify(listener, times(1)).onFailure(exceptionCaptor.capture()); assertEquals("Creating anomaly result index with mappings call not acknowledged", exceptionCaptor.getValue().getMessage()); @@ -142,7 +142,7 @@ public void testCreateADResultIndexNotAcknowledged() throws IOException { public void testWrongAnomalyResult() { BulkRequestBuilder bulkRequestBuilder = new BulkRequestBuilder(client, BulkAction.INSTANCE); doReturn(bulkRequestBuilder).when(client).prepareBulk(); - doReturn(true).when(anomalyDetectionIndices).doesDefaultAnomalyResultIndexExist(); + doReturn(true).when(anomalyDetectionIndices).doesDefaultResultIndexExist(); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(1); BulkItemResponse[] bulkItemResponses = new BulkItemResponse[2]; @@ -176,7 +176,7 @@ public void testWrongAnomalyResult() { public void testBulkSaveException() { BulkRequestBuilder bulkRequestBuilder = mock(BulkRequestBuilder.class); doReturn(bulkRequestBuilder).when(client).prepareBulk(); - doReturn(true).when(anomalyDetectionIndices).doesDefaultAnomalyResultIndexExist(); + doReturn(true).when(anomalyDetectionIndices).doesDefaultResultIndexExist(); String testError = randomAlphaOfLength(5); doAnswer(invocation -> { diff --git a/src/test/java/org/opensearch/ad/util/ThrowingSupplierWrapperTests.java b/src/test/java/org/opensearch/ad/util/ThrowingSupplierWrapperTests.java index 3dbe7eb58..f7db4a278 100644 --- a/src/test/java/org/opensearch/ad/util/ThrowingSupplierWrapperTests.java +++ b/src/test/java/org/opensearch/ad/util/ThrowingSupplierWrapperTests.java @@ -14,6 +14,7 @@ import java.io.IOException; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.timeseries.function.ThrowingSupplierWrapper; public class ThrowingSupplierWrapperTests extends OpenSearchTestCase { private static String foo() throws IOException { diff --git a/src/test/java/org/opensearch/forecast/indices/ForecastIndexManagementTests.java b/src/test/java/org/opensearch/forecast/indices/ForecastIndexManagementTests.java new file mode 100644 index 000000000..f7cbb96ea --- /dev/null +++ b/src/test/java/org/opensearch/forecast/indices/ForecastIndexManagementTests.java @@ -0,0 +1,347 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.forecast.indices; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Locale; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.hamcrest.MatcherAssert; +import org.junit.Before; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; +import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.ad.AnomalyDetectorPlugin; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.IndexNotFoundException; +import org.opensearch.plugins.Plugin; +import org.opensearch.timeseries.TestHelpers; +import org.opensearch.timeseries.function.ExecutorFunction; +import org.opensearch.timeseries.indices.IndexManagementIntegTestCase; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; + +/** + * Inherit from OpenSearchIntegTestCase we need methods like client() and nodePlugins(). + * Have to name this to Tests instead of IT. Otherwise, we have + * to run it using integTest and there are quite a few errors like + * "java.lang.IllegalArgumentException: Cannot run TEST scope test with tests.cluster". + * + */ +public class ForecastIndexManagementTests extends IndexManagementIntegTestCase { + private ForecastIndexManagement indices; + private Settings settings; + private DiscoveryNodeFilterer nodeFilter; + + // help register setting using AnomalyDetectorPlugin.getSettings. Otherwise, AnomalyDetectionIndices's constructor would fail due to + // unregistered settings like AD_RESULT_HISTORY_MAX_DOCS. + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(AnomalyDetectorPlugin.class); + } + + @Before + public void setup() throws IOException { + settings = Settings + .builder() + .put("plugins.forecast.forecast_result_history_rollover_period", TimeValue.timeValueHours(12)) + .put("plugins.forecast.forecast_result_history_retention_period", TimeValue.timeValueHours(24)) + .put("plugins.forecast.forecast_result_history_max_docs", 10000L) + .put("plugins.forecast.request_timeout", TimeValue.timeValueSeconds(10)) + .build(); + + nodeFilter = new DiscoveryNodeFilterer(clusterService()); + + indices = new ForecastIndexManagement( + client(), + clusterService(), + client().threadPool(), + settings, + nodeFilter, + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES + ); + } + + public void testForecastResultIndexNotExists() { + boolean exists = indices.doesDefaultResultIndexExist(); + assertFalse(exists); + } + + public void testForecastResultIndexExists() throws IOException { + indices.initDefaultResultIndexIfAbsent(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.RESULT.getIndexName()); + assertTrue(indices.doesDefaultResultIndexExist()); + } + + public void testForecastResultIndexExistsAndNotRecreate() throws IOException { + indices + .initDefaultResultIndexIfAbsent( + TestHelpers + .createActionListener( + response -> logger.info("Acknowledged: " + response.isAcknowledged()), + failure -> { throw new RuntimeException("should not recreate index"); } + ) + ); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.RESULT.getIndexName()); + if (client().admin().indices().prepareExists(ForecastIndex.RESULT.getIndexName()).get().isExists()) { + indices + .initDefaultResultIndexIfAbsent( + TestHelpers + .createActionListener( + response -> { throw new RuntimeException("should not recreate index " + ForecastIndex.RESULT.getIndexName()); }, + failure -> { + throw new RuntimeException("should not recreate index " + ForecastIndex.RESULT.getIndexName(), failure); + } + ) + ); + } + } + + public void testCheckpointIndexNotExists() { + boolean exists = indices.doesCheckpointIndexExist(); + assertFalse(exists); + } + + public void testCheckpointIndexExists() throws IOException { + indices.initCheckpointIndex(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.STATE.getIndexName()); + assertTrue(indices.doesCheckpointIndexExist()); + } + + public void testStateIndexNotExists() { + boolean exists = indices.doesStateIndexExist(); + assertFalse(exists); + } + + public void testStateIndexExists() throws IOException { + indices.initStateIndex(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.STATE.getIndexName()); + assertTrue(indices.doesStateIndexExist()); + } + + public void testConfigIndexNotExists() { + boolean exists = indices.doesConfigIndexExist(); + assertFalse(exists); + } + + public void testConfigIndexExists() throws IOException { + indices.initConfigIndex(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.CONFIG.getIndexName()); + assertTrue(indices.doesConfigIndexExist()); + } + + public void testCustomResultIndexExists() throws IOException { + String indexName = "a"; + assertTrue(!(client().admin().indices().prepareExists(indexName).get().isExists())); + indices + .initCustomResultIndexDirectly( + indexName, + TestHelpers + .createActionListener( + response -> logger.info("Acknowledged: " + response.isAcknowledged()), + failure -> { throw new RuntimeException("should not recreate index"); } + ) + ); + TestHelpers.waitForIndexCreationToComplete(client(), indexName); + assertTrue((client().admin().indices().prepareExists(indexName).get().isExists())); + } + + public void testJobIndexNotExists() { + boolean exists = indices.doesJobIndexExist(); + assertFalse(exists); + } + + public void testJobIndexExists() throws IOException { + indices.initJobIndex(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.JOB.getIndexName()); + assertTrue(indices.doesJobIndexExist()); + } + + public void testValidateCustomIndexForBackendJobNoIndex() { + validateCustomIndexForBackendJobNoIndex(indices); + } + + public void testValidateCustomIndexForBackendJobInvalidMapping() { + validateCustomIndexForBackendJobInvalidMapping(indices); + } + + public void testValidateCustomIndexForBackendJob() throws IOException, InterruptedException { + validateCustomIndexForBackendJob(indices, ForecastIndexManagement.getResultMappings()); + } + + public void testRollOver() throws IOException, InterruptedException { + indices.initDefaultResultIndexIfAbsent(TestHelpers.createActionListener(response -> { + boolean acknowledged = response.isAcknowledged(); + assertTrue(acknowledged); + }, failure -> { throw new RuntimeException("should not recreate index"); })); + TestHelpers.waitForIndexCreationToComplete(client(), ForecastIndex.RESULT.getIndexName()); + client().index(indices.createDummyIndexRequest(ForecastIndex.RESULT.getIndexName())).actionGet(); + + GetAliasesResponse getAliasesResponse = admin().indices().prepareGetAliases(ForecastIndex.RESULT.getIndexName()).get(); + String oldIndex = getAliasesResponse.getAliases().keySet().iterator().next(); + + settings = Settings + .builder() + .put("plugins.forecast.forecast_result_history_rollover_period", TimeValue.timeValueHours(12)) + .put("plugins.forecast.forecast_result_history_retention_period", TimeValue.timeValueHours(0)) + .put("plugins.forecast.forecast_result_history_max_docs", 0L) + .put("plugins.forecast.forecast_result_history_max_docs_per_shard", 0L) + .put("plugins.forecast.request_timeout", TimeValue.timeValueSeconds(10)) + .build(); + + nodeFilter = new DiscoveryNodeFilterer(clusterService()); + + indices = new ForecastIndexManagement( + client(), + clusterService(), + client().threadPool(), + settings, + nodeFilter, + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES + ); + indices.rolloverAndDeleteHistoryIndex(); + + // replace the last two characters "-1" to "000002"? + // Example: + // Input: opensearch-forecast-results-history-2023.06.15-1 + // Output: opensearch-forecast-results-history-2023.06.15-000002 + String newIndex = oldIndex.replaceFirst("-1$", "-000002"); + TestHelpers.waitForIndexCreationToComplete(client(), newIndex); + + getAliasesResponse = admin().indices().prepareGetAliases(ForecastIndex.RESULT.getIndexName()).get(); + String currentPointedIndex = getAliasesResponse.getAliases().keySet().iterator().next(); + assertEquals(newIndex, currentPointedIndex); + + client().index(indices.createDummyIndexRequest(ForecastIndex.RESULT.getIndexName())).actionGet(); + // now we have two indices + indices.rolloverAndDeleteHistoryIndex(); + + String thirdIndexName = getIncrementedIndex(newIndex); + TestHelpers.waitForIndexCreationToComplete(client(), thirdIndexName); + getAliasesResponse = admin().indices().prepareGetAliases(ForecastIndex.RESULT.getIndexName()).get(); + currentPointedIndex = getAliasesResponse.getAliases().keySet().iterator().next(); + assertEquals(thirdIndexName, currentPointedIndex); + + // we have already deleted the oldest index since retention period is 0 hrs + int retry = 0; + while (retry < 10) { + try { + client().admin().indices().prepareGetIndex().addIndices(oldIndex).get(); + retry++; + // wait for index to be deleted + Thread.sleep(1000); + } catch (IndexNotFoundException e) { + MatcherAssert.assertThat(e.getMessage(), is(String.format(Locale.ROOT, "no such index [%s]", oldIndex))); + break; + } + } + + assertTrue(retry < 20); + + // 2nd oldest index should be fine as we keep at one old index + GetIndexResponse response = client().admin().indices().prepareGetIndex().addIndices(newIndex).get(); + String[] indicesInResponse = response.indices(); + MatcherAssert.assertThat(indicesInResponse, notNullValue()); + MatcherAssert.assertThat(indicesInResponse.length, equalTo(1)); + MatcherAssert.assertThat(indicesInResponse[0], equalTo(newIndex)); + + response = client().admin().indices().prepareGetIndex().addIndices(thirdIndexName).get(); + indicesInResponse = response.indices(); + MatcherAssert.assertThat(indicesInResponse, notNullValue()); + MatcherAssert.assertThat(indicesInResponse.length, equalTo(1)); + MatcherAssert.assertThat(indicesInResponse[0], equalTo(thirdIndexName)); + } + + /** + * Increment the last digit oif an index name. + * @param input. Example: opensearch-forecast-results-history-2023.06.15-000002 + * @return Example: opensearch-forecast-results-history-2023.06.15-000003 + */ + private String getIncrementedIndex(String input) { + int lastDash = input.lastIndexOf('-'); + + String prefix = input.substring(0, lastDash + 1); + String numberPart = input.substring(lastDash + 1); + + // Increment the number part + int incrementedNumber = Integer.parseInt(numberPart) + 1; + + // Use String.format to keep the leading zeros + String newNumberPart = String.format(Locale.ROOT, "%06d", incrementedNumber); + + return prefix + newNumberPart; + } + + public void testInitCustomResultIndexAndExecuteIndexNotExist() throws InterruptedException { + String resultIndex = "abc"; + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(function).execute(); + + indices.initCustomResultIndexAndExecute(resultIndex, function, listener); + latch.await(20, TimeUnit.SECONDS); + verify(listener, never()).onFailure(any(Exception.class)); + } + + public void testInitCustomResultIndexAndExecuteIndex() throws InterruptedException, IOException { + String indexName = "abc"; + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + indices + .initCustomResultIndexDirectly( + indexName, + TestHelpers + .createActionListener( + response -> logger.info("Acknowledged: " + response.isAcknowledged()), + failure -> { throw new RuntimeException("should not recreate index"); } + ) + ); + TestHelpers.waitForIndexCreationToComplete(client(), indexName); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(function).execute(); + + indices.initCustomResultIndexAndExecute(indexName, function, listener); + latch.await(20, TimeUnit.SECONDS); + verify(listener, never()).onFailure(any(Exception.class)); + } +} diff --git a/src/test/java/org/opensearch/forecast/indices/ForecastIndexMappingTests.java b/src/test/java/org/opensearch/forecast/indices/ForecastIndexMappingTests.java new file mode 100644 index 000000000..915f7419c --- /dev/null +++ b/src/test/java/org/opensearch/forecast/indices/ForecastIndexMappingTests.java @@ -0,0 +1,87 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.forecast.indices; + +import java.io.IOException; + +import org.opensearch.test.OpenSearchTestCase; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +public class ForecastIndexMappingTests extends OpenSearchTestCase { + + public void testGetForecastResultMappings() throws IOException { + String mapping = ForecastIndexManagement.getResultMappings(); + + // Use Jackson to convert the string into a JsonNode + ObjectMapper mapper = new ObjectMapper(); + JsonNode mappingJson = mapper.readTree(mapping); + + // Check the existence of some fields + assertTrue("forecaster_id field is missing", mappingJson.path("properties").has("forecaster_id")); + assertTrue("feature_data field is missing", mappingJson.path("properties").has("feature_data")); + assertTrue("data_start_time field is missing", mappingJson.path("properties").has("data_start_time")); + assertTrue("execution_start_time field is missing", mappingJson.path("properties").has("execution_start_time")); + assertTrue("user field is missing", mappingJson.path("properties").has("user")); + assertTrue("entity field is missing", mappingJson.path("properties").has("entity")); + assertTrue("schema_version field is missing", mappingJson.path("properties").has("schema_version")); + assertTrue("task_id field is missing", mappingJson.path("properties").has("task_id")); + assertTrue("model_id field is missing", mappingJson.path("properties").has("model_id")); + assertTrue("forecast_series field is missing", mappingJson.path("properties").has("forecast_series")); + } + + public void testGetCheckpointMappings() throws IOException { + String mapping = ForecastIndexManagement.getCheckpointMappings(); + + // Use Jackson to convert the string into a JsonNode + ObjectMapper mapper = new ObjectMapper(); + JsonNode mappingJson = mapper.readTree(mapping); + + // Check the existence of some fields + assertTrue("forecaster_id field is missing", mappingJson.path("properties").has("forecaster_id")); + assertTrue("timestamp field is missing", mappingJson.path("properties").has("timestamp")); + assertTrue("schema_version field is missing", mappingJson.path("properties").has("schema_version")); + assertTrue("entity field is missing", mappingJson.path("properties").has("entity")); + assertTrue("model field is missing", mappingJson.path("properties").has("model")); + assertTrue("samples field is missing", mappingJson.path("properties").has("samples")); + assertTrue("last_processed_sample field is missing", mappingJson.path("properties").has("last_processed_sample")); + } + + public void testGetStateMappings() throws IOException { + String mapping = ForecastIndexManagement.getStateMappings(); + + // Use Jackson to convert the string into a JsonNode + ObjectMapper mapper = new ObjectMapper(); + JsonNode mappingJson = mapper.readTree(mapping); + + // Check the existence of some fields + assertTrue("schema_version field is missing", mappingJson.path("properties").has("schema_version")); + assertTrue("last_update_time field is missing", mappingJson.path("properties").has("last_update_time")); + assertTrue("error field is missing", mappingJson.path("properties").has("error")); + assertTrue("started_by field is missing", mappingJson.path("properties").has("started_by")); + assertTrue("stopped_by field is missing", mappingJson.path("properties").has("stopped_by")); + assertTrue("forecaster_id field is missing", mappingJson.path("properties").has("forecaster_id")); + assertTrue("state field is missing", mappingJson.path("properties").has("state")); + assertTrue("task_progress field is missing", mappingJson.path("properties").has("task_progress")); + assertTrue("init_progress field is missing", mappingJson.path("properties").has("init_progress")); + assertTrue("current_piece field is missing", mappingJson.path("properties").has("current_piece")); + assertTrue("execution_start_time field is missing", mappingJson.path("properties").has("execution_start_time")); + assertTrue("execution_end_time field is missing", mappingJson.path("properties").has("execution_end_time")); + assertTrue("is_latest field is missing", mappingJson.path("properties").has("is_latest")); + assertTrue("task_type field is missing", mappingJson.path("properties").has("task_type")); + assertTrue("checkpoint_id field is missing", mappingJson.path("properties").has("checkpoint_id")); + assertTrue("coordinating_node field is missing", mappingJson.path("properties").has("coordinating_node")); + assertTrue("worker_node field is missing", mappingJson.path("properties").has("worker_node")); + assertTrue("user field is missing", mappingJson.path("properties").has("user")); + assertTrue("forecaster field is missing", mappingJson.path("properties").has("forecaster")); + assertTrue("date_range field is missing", mappingJson.path("properties").has("date_range")); + assertTrue("parent_task_id field is missing", mappingJson.path("properties").has("parent_task_id")); + assertTrue("entity field is missing", mappingJson.path("properties").has("entity")); + assertTrue("estimated_minutes_left field is missing", mappingJson.path("properties").has("estimated_minutes_left")); + } + +} diff --git a/src/test/java/org/opensearch/forecast/indices/ForecastResultIndexTests.java b/src/test/java/org/opensearch/forecast/indices/ForecastResultIndexTests.java new file mode 100644 index 000000000..7b537de44 --- /dev/null +++ b/src/test/java/org/opensearch/forecast/indices/ForecastResultIndexTests.java @@ -0,0 +1,229 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.forecast.indices; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import org.mockito.ArgumentCaptor; +import org.opensearch.ResourceAlreadyExistsException; +import org.opensearch.Version; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.cluster.state.ClusterStateResponse; +import org.opensearch.action.admin.indices.create.CreateIndexResponse; +import org.opensearch.client.AdminClient; +import org.opensearch.client.Client; +import org.opensearch.client.ClusterAdminClient; +import org.opensearch.client.IndicesAdminClient; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.UUIDs; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.env.Environment; +import org.opensearch.forecast.settings.ForecastSettings; +import org.opensearch.index.Index; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.timeseries.AbstractTimeSeriesTest; +import org.opensearch.timeseries.common.exception.EndRunException; +import org.opensearch.timeseries.function.ExecutorFunction; +import org.opensearch.timeseries.indices.IndexManagement; +import org.opensearch.timeseries.settings.TimeSeriesSettings; +import org.opensearch.timeseries.util.DiscoveryNodeFilterer; + +public class ForecastResultIndexTests extends AbstractTimeSeriesTest { + private ForecastIndexManagement forecastIndices; + private IndicesAdminClient indicesClient; + private ClusterAdminClient clusterAdminClient; + private ClusterName clusterName; + private ClusterState clusterState; + private ClusterService clusterService; + private long defaultMaxDocs; + private int numberOfNodes; + private Client client; + + @Override + public void setUp() throws Exception { + super.setUp(); + client = mock(Client.class); + indicesClient = mock(IndicesAdminClient.class); + AdminClient adminClient = mock(AdminClient.class); + clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings( + Settings.EMPTY, + Collections + .unmodifiableSet( + new HashSet<>( + Arrays + .asList( + ForecastSettings.FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD, + ForecastSettings.FORECAST_RESULT_HISTORY_ROLLOVER_PERIOD, + ForecastSettings.FORECAST_RESULT_HISTORY_RETENTION_PERIOD, + ForecastSettings.FORECAST_MAX_PRIMARY_SHARDS + ) + ) + ) + ); + + clusterName = new ClusterName("test"); + + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + + ThreadPool threadPool = mock(ThreadPool.class); + Settings settings = Settings.EMPTY; + when(client.admin()).thenReturn(adminClient); + when(adminClient.indices()).thenReturn(indicesClient); + + DiscoveryNodeFilterer nodeFilter = mock(DiscoveryNodeFilterer.class); + numberOfNodes = 2; + when(nodeFilter.getNumberOfEligibleDataNodes()).thenReturn(numberOfNodes); + + forecastIndices = new ForecastIndexManagement( + client, + clusterService, + threadPool, + settings, + nodeFilter, + TimeSeriesSettings.MAX_UPDATE_RETRY_TIMES + ); + + clusterAdminClient = mock(ClusterAdminClient.class); + when(adminClient.cluster()).thenReturn(clusterAdminClient); + + doAnswer(invocation -> { + ClusterStateRequest clusterStateRequest = invocation.getArgument(0); + assertEquals(ForecastIndexManagement.ALL_FORECAST_RESULTS_INDEX_PATTERN, clusterStateRequest.indices()[0]); + @SuppressWarnings("unchecked") + ActionListener listener = (ActionListener) invocation.getArgument(1); + listener.onResponse(new ClusterStateResponse(clusterName, clusterState, true)); + return null; + }).when(clusterAdminClient).state(any(), any()); + + defaultMaxDocs = ForecastSettings.FORECAST_RESULT_HISTORY_MAX_DOCS_PER_SHARD.getDefault(Settings.EMPTY); + + clusterState = ClusterState.builder(clusterName).metadata(Metadata.builder().build()).build(); + when(clusterService.state()).thenReturn(clusterState); + } + + public void testMappingSetToUpdated() throws IOException { + try { + doAnswer(invocation -> { + ActionListener listener = (ActionListener) invocation.getArgument(1); + listener.onResponse(new CreateIndexResponse(true, true, "blah")); + return null; + }).when(indicesClient).create(any(), any()); + + super.setUpLog4jForJUnit(IndexManagement.class); + + ActionListener listener = mock(ActionListener.class); + forecastIndices.initDefaultResultIndexDirectly(listener); + verify(listener, times(1)).onResponse(any(CreateIndexResponse.class)); + assertTrue(testAppender.containsMessage("mapping up-to-date")); + } finally { + super.tearDownLog4jForJUnit(); + } + + } + + public void testInitCustomResultIndexNoAck() { + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + doAnswer(invocation -> { + ActionListener createIndexListener = (ActionListener) invocation.getArgument(1); + createIndexListener.onResponse(new CreateIndexResponse(false, false, "blah")); + return null; + }).when(indicesClient).create(any(), any()); + + ArgumentCaptor response = ArgumentCaptor.forClass(Exception.class); + forecastIndices.initCustomResultIndexAndExecute("abc", function, listener); + verify(listener, times(1)).onFailure(response.capture()); + Exception value = response.getValue(); + assertTrue(value instanceof EndRunException); + assertTrue( + "actual: " + value.getMessage(), + value.getMessage().contains("Creating result index with mappings call not acknowledged") + ); + + } + + public void testInitCustomResultIndexAlreadyExist() throws IOException { + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + String indexName = "abc"; + + Settings settings = Settings + .builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .build(); + IndexMetadata indexMetaData = IndexMetadata + .builder(indexName) + .settings(settings) + .putMapping(ForecastIndexManagement.getResultMappings()) + .build(); + final Map indices = new HashMap<>(); + indices.put(indexName, indexMetaData); + + clusterState = ClusterState.builder(clusterName).metadata(Metadata.builder().indices(indices).build()).build(); + when(clusterService.state()).thenReturn(clusterState); + + doAnswer(invocation -> { + ActionListener createIndexListener = (ActionListener) invocation.getArgument(1); + createIndexListener.onFailure(new ResourceAlreadyExistsException(new Index(indexName, indexName))); + return null; + }).when(indicesClient).create(any(), any()); + + forecastIndices.initCustomResultIndexAndExecute(indexName, function, listener); + verify(listener, never()).onFailure(any()); + } + + public void testInitCustomResultIndexUnknownException() throws IOException { + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + String indexName = "abc"; + String exceptionMsg = "blah"; + + doAnswer(invocation -> { + ActionListener createIndexListener = (ActionListener) invocation.getArgument(1); + createIndexListener.onFailure(new IllegalArgumentException(exceptionMsg)); + return null; + }).when(indicesClient).create(any(), any()); + super.setUpLog4jForJUnit(IndexManagement.class); + try { + forecastIndices.initCustomResultIndexAndExecute(indexName, function, listener); + ArgumentCaptor response = ArgumentCaptor.forClass(Exception.class); + verify(listener, times(1)).onFailure(response.capture()); + + Exception value = response.getValue(); + assertTrue(value instanceof IllegalArgumentException); + assertTrue("actual: " + value.getMessage(), value.getMessage().contains(exceptionMsg)); + } finally { + super.tearDownLog4jForJUnit(); + } + } +} diff --git a/src/test/java/org/opensearch/timeseries/TestHelpers.java b/src/test/java/org/opensearch/timeseries/TestHelpers.java index 56f504d0d..0eb65538a 100644 --- a/src/test/java/org/opensearch/timeseries/TestHelpers.java +++ b/src/test/java/org/opensearch/timeseries/TestHelpers.java @@ -63,7 +63,7 @@ import org.opensearch.ad.constant.ADCommonName; import org.opensearch.ad.constant.CommonValue; import org.opensearch.ad.feature.Features; -import org.opensearch.ad.indices.AnomalyDetectionIndices; +import org.opensearch.ad.indices.ADIndexManagement; import org.opensearch.ad.ml.ThresholdingResult; import org.opensearch.ad.mock.model.MockSimpleLog; import org.opensearch.ad.model.ADTask; @@ -1128,11 +1128,7 @@ public static void createEmptyIndexMapping(RestClient client, String indexName, public static void createEmptyAnomalyResultIndex(RestClient client) throws IOException { createEmptyIndex(client, ADCommonName.ANOMALY_RESULT_INDEX_ALIAS); - createIndexMapping( - client, - ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, - toHttpEntity(AnomalyDetectionIndices.getAnomalyResultMappings()) - ); + createIndexMapping(client, ADCommonName.ANOMALY_RESULT_INDEX_ALIAS, toHttpEntity(ADIndexManagement.getResultMappings())); } public static void createEmptyIndex(RestClient client, String indexName) throws IOException { diff --git a/src/test/java/org/opensearch/timeseries/indices/IndexManagementIntegTestCase.java b/src/test/java/org/opensearch/timeseries/indices/IndexManagementIntegTestCase.java new file mode 100644 index 000000000..dd1ec06da --- /dev/null +++ b/src/test/java/org/opensearch/timeseries/indices/IndexManagementIntegTestCase.java @@ -0,0 +1,104 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.timeseries.indices; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import org.mockito.ArgumentCaptor; +import org.opensearch.action.ActionListener; +import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.timeseries.common.exception.EndRunException; +import org.opensearch.timeseries.constant.CommonMessages; +import org.opensearch.timeseries.function.ExecutorFunction; + +public abstract class IndexManagementIntegTestCase & TimeSeriesIndex, ISMType extends IndexManagement> + extends OpenSearchIntegTestCase { + + public void validateCustomIndexForBackendJob(ISMType indices, String resultMapping) throws IOException, InterruptedException { + + Map asMap = XContentHelper.convertToMap(new BytesArray(resultMapping), false, XContentType.JSON).v2(); + String resultIndex = "test_index"; + + client() + .admin() + .indices() + .prepareCreate(resultIndex) + .setSettings(Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0)) + .setMapping(asMap) + .get(); + ensureGreen(resultIndex); + + String securityLogId = "logId"; + String user = "testUser"; + List roles = Arrays.asList("role1", "role2"); + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(function).execute(); + latch.await(20, TimeUnit.SECONDS); + indices.validateCustomIndexForBackendJob(resultIndex, securityLogId, user, roles, function, listener); + verify(listener, never()).onFailure(any(Exception.class)); + } + + public void validateCustomIndexForBackendJobInvalidMapping(ISMType indices) { + String resultIndex = "test_index"; + + client() + .admin() + .indices() + .prepareCreate(resultIndex) + .setSettings(Settings.builder().put("index.number_of_shards", 1).put("index.number_of_replicas", 0)) + .setMapping("ip", "type=ip") + .get(); + ensureGreen(resultIndex); + + String securityLogId = "logId"; + String user = "testUser"; + List roles = Arrays.asList("role1", "role2"); + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + indices.validateCustomIndexForBackendJob(resultIndex, securityLogId, user, roles, function, listener); + + ArgumentCaptor exceptionCaptor = ArgumentCaptor.forClass(EndRunException.class); + verify(listener).onFailure(exceptionCaptor.capture()); + assertEquals("Result index mapping is not correct", exceptionCaptor.getValue().getMessage()); + } + + public void validateCustomIndexForBackendJobNoIndex(ISMType indices) { + String resultIndex = "testIndex"; + String securityLogId = "logId"; + String user = "testUser"; + List roles = Arrays.asList("role1", "role2"); + ExecutorFunction function = mock(ExecutorFunction.class); + ActionListener listener = mock(ActionListener.class); + + indices.validateCustomIndexForBackendJob(resultIndex, securityLogId, user, roles, function, listener); + + ArgumentCaptor exceptionCaptor = ArgumentCaptor.forClass(EndRunException.class); + verify(listener).onFailure(exceptionCaptor.capture()); + assertEquals(CommonMessages.CAN_NOT_FIND_RESULT_INDEX + resultIndex, exceptionCaptor.getValue().getMessage()); + } +} diff --git a/src/test/java/test/org/opensearch/ad/util/FakeNode.java b/src/test/java/test/org/opensearch/ad/util/FakeNode.java index 672072884..1af160f91 100644 --- a/src/test/java/test/org/opensearch/ad/util/FakeNode.java +++ b/src/test/java/test/org/opensearch/ad/util/FakeNode.java @@ -37,6 +37,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.lease.Releasable; import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; @@ -44,7 +45,6 @@ import org.opensearch.common.transport.BoundTransportAddress; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.tasks.TaskManager; import org.opensearch.tasks.TaskResourceTrackingService;