diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorPlugin.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorPlugin.java index 47160911..8184ec78 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorPlugin.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorPlugin.java @@ -108,6 +108,13 @@ import com.amazon.opendistroforelasticsearch.ad.stats.suppliers.IndexStatusSupplier; import com.amazon.opendistroforelasticsearch.ad.stats.suppliers.ModelsOnNodeSupplier; import com.amazon.opendistroforelasticsearch.ad.stats.suppliers.SettableSupplier; +import com.amazon.opendistroforelasticsearch.ad.task.ADBatchTaskRunner; +import com.amazon.opendistroforelasticsearch.ad.task.ADTaskCacheManager; +import com.amazon.opendistroforelasticsearch.ad.task.ADTaskManager; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultTransportAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchTaskRemoteExecutionAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchTaskRemoteExecutionTransportAction; import com.amazon.opendistroforelasticsearch.ad.transport.ADResultBulkAction; import com.amazon.opendistroforelasticsearch.ad.transport.ADResultBulkTransportAction; import com.amazon.opendistroforelasticsearch.ad.transport.ADStatsNodesAction; @@ -151,6 +158,7 @@ import com.amazon.opendistroforelasticsearch.ad.transport.ThresholdResultAction; import com.amazon.opendistroforelasticsearch.ad.transport.ThresholdResultTransportAction; import com.amazon.opendistroforelasticsearch.ad.transport.handler.AnomalyIndexHandler; +import com.amazon.opendistroforelasticsearch.ad.transport.handler.AnomalyResultBulkIndexHandler; import com.amazon.opendistroforelasticsearch.ad.transport.handler.DetectionStateHandler; import com.amazon.opendistroforelasticsearch.ad.transport.handler.MultiEntityResultHandler; import com.amazon.opendistroforelasticsearch.ad.util.ClientUtil; @@ -176,7 +184,9 @@ public class AnomalyDetectorPlugin extends Plugin implements ActionPlugin, Scrip public static final String AD_BASE_URI = "/_opendistro/_anomaly_detection"; public static final String AD_BASE_DETECTORS_URI = AD_BASE_URI + "/detectors"; + public static final String AD_THREAD_POOL_PREFIX = "opendistro.ad."; public static final String AD_THREAD_POOL_NAME = "ad-threadpool"; + 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; @@ -189,6 +199,9 @@ public class AnomalyDetectorPlugin extends Plugin implements ActionPlugin, Scrip private DiscoveryNodeFilterer nodeFilter; private IndexUtils indexUtils; private DetectionStateHandler detectorStateHandler; + private ADTaskCacheManager adTaskCacheManager; + private ADTaskManager adTaskManager; + private ADBatchTaskRunner adBatchTaskRunner; static { SpecialPermission.check(); @@ -458,7 +471,7 @@ public Collection createComponents( ) .put( StatNames.ANOMALY_DETECTION_STATE_STATUS.getName(), - new ADStat<>(true, new IndexStatusSupplier(indexUtils, DetectorInternalState.DETECTOR_STATE_INDEX)) + new ADStat<>(true, new IndexStatusSupplier(indexUtils, CommonName.DETECTION_STATE_INDEX)) ) .put(StatNames.DETECTOR_COUNT.getName(), new ADStat<>(true, new SettableSupplier())) .put(StatNames.HISTORICAL_SINGLE_ENTITY_DETECTOR_COUNT.getName(), new ADStat<>(true, new SettableSupplier())) @@ -474,7 +487,7 @@ public Collection createComponents( client, settings, threadPool, - ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectorStateIndex), + ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectionStateIndex), anomalyDetectionIndices::doesDetectorStateIndexExist, this.clientUtil, this.indexUtils, @@ -494,6 +507,35 @@ public Collection createComponents( stateManager ); + adTaskCacheManager = new ADTaskCacheManager(settings, clusterService, memoryTracker); + adTaskManager = new ADTaskManager(settings, clusterService, client, xContentRegistry, anomalyDetectionIndices); + AnomalyResultBulkIndexHandler anomalyResultBulkIndexHandler = new AnomalyResultBulkIndexHandler( + client, + settings, + threadPool, + ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initAnomalyResultIndexDirectly), + anomalyDetectionIndices::doesAnomalyResultIndexExist, + this.clientUtil, + this.indexUtils, + clusterService, + anomalyDetectionIndices + ); + adBatchTaskRunner = new ADBatchTaskRunner( + settings, + threadPool, + clusterService, + client, + nodeFilter, + indexNameExpressionResolver, + adCircuitBreakerService, + featureManager, + adTaskManager, + anomalyDetectionIndices, + adStats, + anomalyResultBulkIndexHandler, + adTaskCacheManager + ); + // return objects used by Guice to inject dependencies for e.g., // transport action handler constructors return ImmutableList @@ -518,7 +560,9 @@ public Collection createComponents( multiEntityResultHandler, checkpoint, modelPartitioner, - cacheProvider + cacheProvider, + adTaskManager, + adBatchTaskRunner ); } @@ -533,14 +577,21 @@ protected Clock getClock() { @Override public List> getExecutorBuilders(Settings settings) { - return Collections - .singletonList( + return ImmutableList + .of( new FixedExecutorBuilder( settings, AD_THREAD_POOL_NAME, Math.max(1, EsExecutors.allocatedProcessors(settings) / 4), AnomalyDetectorSettings.AD_THEAD_POOL_QUEUE_SIZE, - "opendistro.ad." + AD_THREAD_POOL_NAME + AD_THREAD_POOL_PREFIX + AD_THREAD_POOL_NAME + ), + new FixedExecutorBuilder( + settings, + AD_BATCH_TASK_THREAD_POOL_NAME, + Math.max(1, EsExecutors.allocatedProcessors(settings) / 8), + AnomalyDetectorSettings.AD_THEAD_POOL_QUEUE_SIZE, + AD_THREAD_POOL_PREFIX + AD_BATCH_TASK_THREAD_POOL_NAME ) ); } @@ -572,7 +623,10 @@ public List> getSettings() { AnomalyDetectorSettings.MAX_PRIMARY_SHARDS, AnomalyDetectorSettings.FILTER_BY_BACKEND_ROLES, AnomalyDetectorSettings.MAX_CACHE_MISS_HANDLING_PER_SECOND, - AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE + AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE, + AnomalyDetectorSettings.BATCH_TASK_PIECE_INTERVAL_SECONDS, + AnomalyDetectorSettings.MAX_OLD_AD_TASK_DOCS_PER_DETECTOR, + AnomalyDetectorSettings.BATCH_TASK_PIECE_SIZE ); return unmodifiableList(Stream.concat(enabledSetting.stream(), systemSetting.stream()).collect(Collectors.toList())); } @@ -615,7 +669,9 @@ public List getNamedXContent() { new ActionHandler<>(EntityResultAction.INSTANCE, EntityResultTransportAction.class), new ActionHandler<>(EntityProfileAction.INSTANCE, EntityProfileTransportAction.class), new ActionHandler<>(SearchAnomalyDetectorInfoAction.INSTANCE, SearchAnomalyDetectorInfoTransportAction.class), - new ActionHandler<>(PreviewAnomalyDetectorAction.INSTANCE, PreviewAnomalyDetectorTransportAction.class) + new ActionHandler<>(PreviewAnomalyDetectorAction.INSTANCE, PreviewAnomalyDetectorTransportAction.class), + new ActionHandler<>(ADBatchAnomalyResultAction.INSTANCE, ADBatchAnomalyResultTransportAction.class), + new ActionHandler<>(ADBatchTaskRemoteExecutionAction.INSTANCE, ADBatchTaskRemoteExecutionTransportAction.class) ); } @@ -652,7 +708,7 @@ public Collection getSystemIndexDescriptors(Settings sett new SystemIndexDescriptor(AnomalyDetector.ANOMALY_DETECTORS_INDEX, "detector definition"), new SystemIndexDescriptor(AnomalyDetectorJob.ANOMALY_DETECTOR_JOB_INDEX, "detector job"), new SystemIndexDescriptor(CommonName.CHECKPOINT_INDEX_NAME, "model checkpoint"), - new SystemIndexDescriptor(DetectorInternalState.DETECTOR_STATE_INDEX, "detector information like total rcf updates") + new SystemIndexDescriptor(CommonName.DETECTION_STATE_INDEX, "detector information like total rcf updates") ) ); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunner.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunner.java index 264dce21..b1c26ce4 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunner.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunner.java @@ -189,7 +189,7 @@ private void prepareProfile( false ); if (profilesToCollect.contains(DetectorProfileName.ERROR)) { - GetRequest getStateRequest = new GetRequest(DetectorInternalState.DETECTOR_STATE_INDEX, detectorId); + GetRequest getStateRequest = new GetRequest(CommonName.DETECTION_STATE_INDEX, detectorId); client.get(getStateRequest, onGetDetectorState(delegateListener, detectorId, enabledTimeMs)); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunner.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunner.java index 3f03ebfc..43a5fdc7 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunner.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunner.java @@ -349,7 +349,7 @@ private SearchRequest createLastSampleTimeRequest(String detectorId, long enable SearchSourceBuilder source = new SearchSourceBuilder() .query(boolQueryBuilder) - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(AnomalyResult.EXECUTION_END_TIME_FIELD)) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(AnomalyResult.EXECUTION_END_TIME_FIELD)) .trackTotalHits(false) .size(0); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/breaker/MemoryCircuitBreaker.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/breaker/MemoryCircuitBreaker.java index d4e13613..1ebec593 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/breaker/MemoryCircuitBreaker.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/breaker/MemoryCircuitBreaker.java @@ -22,11 +22,11 @@ */ public class MemoryCircuitBreaker extends ThresholdCircuitBreaker { - private static final short defaultThreshold = 85; + public static final short DEFAULT_JVM_HEAP_USAGE_THRESHOLD = 85; private final JvmService jvmService; public MemoryCircuitBreaker(JvmService jvmService) { - super(defaultThreshold); + super(DEFAULT_JVM_HEAP_USAGE_THRESHOLD); this.jvmService = jvmService; } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledException.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledException.java new file mode 100644 index 00000000..939aef47 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledException.java @@ -0,0 +1,30 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.common.exception; + +public class ADTaskCancelledException extends AnomalyDetectionException { + private String cancelledBy; + + public ADTaskCancelledException(String msg, String user) { + super(msg); + this.cancelledBy = user; + this.countedInStats(false); + } + + public String getCancelledBy() { + return cancelledBy; + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/LimitExceededException.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/LimitExceededException.java index 7a3f6bcf..079760cb 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/LimitExceededException.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/common/exception/LimitExceededException.java @@ -37,7 +37,17 @@ public LimitExceededException(String anomalyDetectorId, String message) { * @param message explanation for the limit */ public LimitExceededException(String message) { - super(null, message, true); + super(message, true); + } + + /** + * Constructor with error message. + * + * @param message explanation for the limit + * @param endRun end detector run or not + */ + public LimitExceededException(String message, boolean endRun) { + super(null, message, endRun); } /** diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/constant/CommonName.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/constant/CommonName.java index 5f407238..b57da50f 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/constant/CommonName.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/constant/CommonName.java @@ -21,6 +21,9 @@ public class CommonName { // ====================================== // index name for anomaly checkpoint of each model. One model one document. public static final String CHECKPOINT_INDEX_NAME = ".opendistro-anomaly-checkpoints"; + // index name for anomaly detection state. Will store AD task in this index as well. + public static final String DETECTION_STATE_INDEX = ".opendistro-anomaly-detection-state"; + // TODO: move other index name here // The alias of the index in which to write AD result history public static final String ANOMALY_RESULT_INDEX_ALIAS = ".opendistro-anomaly-results"; @@ -83,5 +86,11 @@ public class CommonName { // Query // ====================================== // Used in finding the max timestamp - public static final String AGG_NAME_MAX = "max_timefield"; + public static final String AGG_NAME_MAX_TIME = "max_timefield"; + // Used in finding the min timestamp + public static final String AGG_NAME_MIN_TIME = "min_timefield"; + // date histogram aggregation name + public static final String DATE_HISTOGRAM = "date_histogram"; + // feature aggregation name + public static final String FEATURE_AGGS = "feature_aggs"; } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManager.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManager.java index bbfc6e45..e92fcbae 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManager.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManager.java @@ -23,6 +23,7 @@ import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; +import java.util.AbstractMap; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayDeque; import java.util.ArrayList; @@ -172,7 +173,7 @@ public void getCurrentFeatures(AnomalyDetector detector, long startTime, long en listener.onFailure(new EndRunException(detector.getDetectorId(), CommonErrorMessages.INVALID_SEARCH_QUERY_MSG, e, true)); } } else { - getProcessedFeatures(shingle, detector, endTime, listener); + listener.onResponse(getProcessedFeatures(shingle, detector, endTime)); } } @@ -220,26 +221,7 @@ private void updateUnprocessedFeatures( .mapToObj(time -> featuresMap.getOrDefault(time, new SimpleImmutableEntry<>(time, Optional.empty()))) .forEach(e -> shingle.add(e)); - getProcessedFeatures(shingle, detector, endTime, listener); - } - - private void getProcessedFeatures( - Deque>> shingle, - AnomalyDetector detector, - long endTime, - ActionListener listener - ) { - int shingleSize = detector.getShingleSize(); - Optional currentPoint = shingle.peekLast().getValue(); - listener - .onResponse( - new SinglePointFeatures( - currentPoint, - Optional - .ofNullable(currentPoint.isPresent() ? filterAndFill(shingle, endTime, detector) : null) - .map(points -> batchShingle(points, shingleSize)[0]) - ) - ); + listener.onResponse(getProcessedFeatures(shingle, detector, endTime)); } private double[][] filterAndFill(Deque>> shingle, long endTime, AnomalyDetector detector) { @@ -699,4 +681,53 @@ public int getShingleSize(String detectorId) { return -1; } } + + public void getFeatureDataPointsByBatch( + AnomalyDetector detector, + long startTime, + long endTime, + ActionListener>> listener + ) { + try { + searchFeatureDao.getFeaturesForPeriodByBatch(detector, startTime, endTime, ActionListener.wrap(points -> { + logger.debug("features size: {}", points.size()); + listener.onResponse(points); + }, listener::onFailure)); + } catch (Exception e) { + logger.error("Failed to get features for detector: " + detector.getDetectorId()); + listener.onFailure(e); + } + } + + public SinglePointFeatures getShingledFeatureForHistoricalDetector( + AnomalyDetector detector, + Deque>> shingle, + Optional dataPoint, + long endTime + ) { + while (shingle.size() >= detector.getShingleSize()) { + shingle.poll(); + } + shingle.add(new AbstractMap.SimpleEntry<>(endTime, dataPoint)); + + return getProcessedFeatures(shingle, detector, endTime); + } + + private SinglePointFeatures getProcessedFeatures( + Deque>> shingle, + AnomalyDetector detector, + long endTime + ) { + int shingleSize = detector.getShingleSize(); + Optional currentPoint = shingle.peekLast().getValue(); + return new SinglePointFeatures( + currentPoint, + Optional + // if current point is not present or current shingle has more missing data points than + // max missing rate, will return null + .ofNullable(currentPoint.isPresent() ? filterAndFill(shingle, endTime, detector) : null) + .map(points -> batchShingle(points, shingleSize)[0]) + ); + } + } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDao.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDao.java index 48868319..d33e4ab0 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDao.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDao.java @@ -15,8 +15,10 @@ package com.amazon.opendistroforelasticsearch.ad.feature; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.DATE_HISTOGRAM; import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_ENTITIES_FOR_PREVIEW; import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_ENTITIES_PER_QUERY; +import static com.amazon.opendistroforelasticsearch.ad.util.ParseUtils.batchFeatureQuery; import static org.apache.commons.math3.linear.MatrixUtils.createRealMatrix; import java.io.IOException; @@ -53,6 +55,7 @@ import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.bucket.composite.InternalComposite; import org.elasticsearch.search.aggregations.bucket.range.InternalDateRange; import org.elasticsearch.search.aggregations.bucket.range.InternalDateRange.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.Terms; @@ -138,14 +141,14 @@ public SearchFeatureDao( @Deprecated public Optional getLatestDataTime(AnomalyDetector detector) { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .size(0); SearchRequest searchRequest = new SearchRequest().indices(detector.getIndices().toArray(new String[0])).source(searchSourceBuilder); return clientUtil .timedRequest(searchRequest, logger, client::search) .map(SearchResponse::getAggregations) .map(aggs -> aggs.asMap()) - .map(map -> (Max) map.get(CommonName.AGG_NAME_MAX)) + .map(map -> (Max) map.get(CommonName.AGG_NAME_MAX_TIME)) .map(agg -> (long) agg.getValue()); } @@ -157,7 +160,7 @@ public Optional getLatestDataTime(AnomalyDetector detector) { */ public void getLatestDataTime(AnomalyDetector detector, ActionListener> listener) { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .size(0); SearchRequest searchRequest = new SearchRequest().indices(detector.getIndices().toArray(new String[0])).source(searchSourceBuilder); client @@ -231,7 +234,7 @@ public void getEntityMinMaxDataTime( SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() .query(internalFilterQuery) - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .aggregation(AggregationBuilders.min(AGG_NAME_MIN).field(detector.getTimeField())) .trackTotalHits(false) .size(0); @@ -249,7 +252,7 @@ private Entry, Optional> parseMinMaxDataTime(SearchResponse .map(SearchResponse::getAggregations) .map(aggs -> aggs.asMap()); - Optional latest = mapOptional.map(map -> (Max) map.get(CommonName.AGG_NAME_MAX)).map(agg -> (long) agg.getValue()); + Optional latest = mapOptional.map(map -> (Max) map.get(CommonName.AGG_NAME_MAX_TIME)).map(agg -> (long) agg.getValue()); Optional earliest = mapOptional.map(map -> (Min) map.get(AGG_NAME_MIN)).map(agg -> (long) agg.getValue()); @@ -299,6 +302,41 @@ public void getFeaturesForPeriod(AnomalyDetector detector, long startTime, long ); } + public void getFeaturesForPeriodByBatch( + AnomalyDetector detector, + long startTime, + long endTime, + ActionListener>> listener + ) throws IOException { + SearchSourceBuilder searchSourceBuilder = batchFeatureQuery(detector, startTime, endTime, xContent); + logger.debug("Batch query for detector {}: {} ", detector.getDetectorId(), searchSourceBuilder); + + SearchRequest searchRequest = new SearchRequest(detector.getIndices().toArray(new String[0])).source(searchSourceBuilder); + client + .search( + searchRequest, + ActionListener + .wrap( + response -> { listener.onResponse(parseBucketAggregationResponse(response, detector.getEnabledFeatureIds())); }, + listener::onFailure + ) + ); + } + + private Map> parseBucketAggregationResponse(SearchResponse response, List featureIds) { + Map> dataPoints = new HashMap<>(); + List aggregations = response.getAggregations().asList(); + logger.debug("Feature aggregation result size {}", aggregations.size()); + for (Aggregation agg : aggregations) { + List buckets = ((InternalComposite) agg).getBuckets(); + buckets.forEach(bucket -> { + Optional featureData = parseAggregations(Optional.ofNullable(bucket.getAggregations()), featureIds); + dataPoints.put((Long) bucket.getKey().get(DATE_HISTOGRAM), featureData); + }); + } + return dataPoints; + } + private Optional parseResponse(SearchResponse response, List featureIds) { return parseAggregations(Optional.ofNullable(response).map(resp -> resp.getAggregations()), featureIds); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/ADIndex.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/ADIndex.java index f33c03c7..bb6a2ad9 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/ADIndex.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/ADIndex.java @@ -20,7 +20,6 @@ import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; -import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.util.ThrowingSupplierWrapper; /** @@ -51,9 +50,9 @@ public enum ADIndex { ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getCheckpointMappings) ), STATE( - DetectorInternalState.DETECTOR_STATE_INDEX, + CommonName.DETECTION_STATE_INDEX, false, - ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getDetectorStateMappings) + ThrowingSupplierWrapper.throwingSupplierWrapper(AnomalyDetectionIndices::getDetectionStateMappings) ); private final String indexName; diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/AnomalyDetectionIndices.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/AnomalyDetectionIndices.java index f3c105f9..42e3f766 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/AnomalyDetectionIndices.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/indices/AnomalyDetectionIndices.java @@ -74,7 +74,6 @@ import com.amazon.opendistroforelasticsearch.ad.constant.CommonValue; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; -import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.util.DiscoveryNodeFilterer; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; @@ -211,7 +210,7 @@ public static String getAnomalyDetectorJobMappings() throws IOException { * @return anomaly detector state index mapping * @throws IOException IOException if mapping file can't be read correctly */ - public static String getDetectorStateMappings() throws IOException { + public static String getDetectionStateMappings() throws IOException { URL url = AnomalyDetectionIndices.class.getClassLoader().getResource(ANOMALY_DETECTION_STATE_INDEX_MAPPING_FILE); return Resources.toString(url, Charsets.UTF_8); } @@ -260,7 +259,7 @@ public boolean doesAnomalyResultIndexExist() { * @return true if anomaly state index exists */ public boolean doesDetectorStateIndexExist() { - return clusterService.state().getRoutingTable().hasIndex(DetectorInternalState.DETECTOR_STATE_INDEX); + return clusterService.state().getRoutingTable().hasIndex(CommonName.DETECTION_STATE_INDEX); } /** @@ -376,26 +375,33 @@ public void initAnomalyResultIndexDirectly(ActionListener a * Create anomaly detector job index. * * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyDetectorJobMappings} */ - public void initAnomalyDetectorJobIndex(ActionListener actionListener) throws IOException { - // TODO: specify replica setting - CreateIndexRequest request = new CreateIndexRequest(AnomalyDetectorJob.ANOMALY_DETECTOR_JOB_INDEX) - .mapping(AnomalyDetector.TYPE, getAnomalyDetectorJobMappings(), XContentType.JSON); - choosePrimaryShards(request); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.JOB, actionListener)); + public void initAnomalyDetectorJobIndex(ActionListener actionListener) { + try { + CreateIndexRequest request = new CreateIndexRequest(AnomalyDetectorJob.ANOMALY_DETECTOR_JOB_INDEX) + .mapping(AnomalyDetector.TYPE, getAnomalyDetectorJobMappings(), XContentType.JSON); + choosePrimaryShards(request); + adminClient.indices().create(request, markMappingUpToDate(ADIndex.JOB, actionListener)); + } catch (IOException e) { + logger.error("Fail to init AD job index", e); + actionListener.onFailure(e); + } } /** * Create the state index. * * @param actionListener action called after create index - * @throws IOException IOException from {@link AnomalyDetectionIndices#getDetectorStateMappings} */ - public void initDetectorStateIndex(ActionListener actionListener) throws IOException { - CreateIndexRequest request = new CreateIndexRequest(DetectorInternalState.DETECTOR_STATE_INDEX) - .mapping(AnomalyDetector.TYPE, getDetectorStateMappings(), XContentType.JSON); - adminClient.indices().create(request, markMappingUpToDate(ADIndex.STATE, actionListener)); + public void initDetectionStateIndex(ActionListener actionListener) { + try { + CreateIndexRequest request = new CreateIndexRequest(CommonName.DETECTION_STATE_INDEX) + .mapping(AnomalyDetector.TYPE, getDetectionStateMappings(), XContentType.JSON); + adminClient.indices().create(request, markMappingUpToDate(ADIndex.STATE, actionListener)); + } catch (IOException e) { + logger.error("Fail to init AD detection state index", e); + actionListener.onFailure(e); + } } /** diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/ADTask.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/ADTask.java index c83836ab..b14ec0fa 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/ADTask.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/ADTask.java @@ -36,8 +36,6 @@ */ public class ADTask implements ToXContentObject, Writeable { - public static final String DETECTOR_STATE_INDEX = ".opendistro-anomaly-detection-state"; - public static final String TASK_ID_FIELD = "task_id"; public static final String LAST_UPDATE_TIME_FIELD = "last_update_time"; public static final String STARTED_BY_FIELD = "started_by"; diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetector.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetector.java index ade2c310..e4cd2c70 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetector.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetector.java @@ -51,6 +51,7 @@ import com.amazon.opendistroforelasticsearch.ad.util.ParseUtils; import com.amazon.opendistroforelasticsearch.commons.authuser.User; import com.google.common.base.Objects; +import com.google.common.collect.ImmutableList; /** * An AnomalyDetector is used to represent anomaly detection model(RCF) related parameters. @@ -211,7 +212,7 @@ public AnomalyDetector( this.description = description; this.timeField = timeField; this.indices = indices; - this.featureAttributes = features; + this.featureAttributes = features == null ? ImmutableList.of() : ImmutableList.copyOf(features); this.filterQuery = filterQuery; this.detectionInterval = detectionInterval; this.windowDelay = windowDelay; @@ -311,11 +312,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws .field(DETECTION_INTERVAL_FIELD, detectionInterval) .field(WINDOW_DELAY_FIELD, windowDelay) .field(SHINGLE_SIZE_FIELD, shingleSize) - .field(CommonName.SCHEMA_VERSION_FIELD, schemaVersion); - - if (featureAttributes != null) { - xContentBuilder.field(FEATURE_ATTRIBUTES_FIELD, featureAttributes.toArray()); - } + .field(CommonName.SCHEMA_VERSION_FIELD, schemaVersion) + .field(FEATURE_ATTRIBUTES_FIELD, featureAttributes.toArray()); if (uiMetadata != null && !uiMetadata.isEmpty()) { xContentBuilder.field(UI_METADATA_FIELD, uiMetadata); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRange.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRange.java index e6076fc8..8f2483c2 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRange.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRange.java @@ -43,28 +43,32 @@ public class DetectionDateRange implements ToXContentObject, Writeable { public DetectionDateRange(Instant startTime, Instant endTime) { this.startTime = startTime; this.endTime = endTime; + validate(); + } + + public DetectionDateRange(StreamInput in) throws IOException { + this.startTime = in.readInstant(); + this.endTime = in.readInstant(); + validate(); + } + + private void validate() { if (startTime == null) { throw new IllegalArgumentException("Detection data range's start time must not be null"); } if (endTime == null) { throw new IllegalArgumentException("Detection data range's end time must not be null"); } - } - - public DetectionDateRange(StreamInput in) throws IOException { - this.startTime = in.readInstant(); - this.endTime = in.readInstant(); + if (startTime.isAfter(endTime)) { + throw new IllegalArgumentException("Detection data range's end time must be after start time"); + } } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { XContentBuilder xContentBuilder = builder.startObject(); - if (startTime != null) { - xContentBuilder.field(START_TIME_FIELD, startTime.toEpochMilli()); - } - if (endTime != null) { - xContentBuilder.field(END_TIME_FIELD, endTime.toEpochMilli()); - } + xContentBuilder.field(START_TIME_FIELD, startTime.toEpochMilli()); + xContentBuilder.field(END_TIME_FIELD, endTime.toEpochMilli()); return xContentBuilder.endObject(); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectorInternalState.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectorInternalState.java index 995eb9db..c0e0abda 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectorInternalState.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/model/DetectorInternalState.java @@ -42,8 +42,6 @@ public class DetectorInternalState implements ToXContentObject, Cloneable { it -> parse(it) ); - public static final String DETECTOR_STATE_INDEX = ".opendistro-anomaly-detection-state"; - public static final String LAST_UPDATE_TIME_FIELD = "last_update_time"; public static final String ERROR_FIELD = "error"; diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java index 54d0f7f7..78f4d2ab 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorActionHandler.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.ad.rest.handler; import static com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector.ANOMALY_DETECTORS_INDEX; +import static com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtil.getShardsFailure; import static com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils.XCONTENT_WITH_TYPE; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; @@ -44,7 +45,6 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.unit.TimeValue; @@ -448,7 +448,9 @@ private void indexAnomalyDetector(String detectorId) throws IOException { anomalyDetector.getSchemaVersion(), Instant.now(), anomalyDetector.getCategoryField(), - user + user, + anomalyDetector.getDetectorType(), + anomalyDetector.getDetectionDateRange() ); IndexRequest indexRequest = new IndexRequest(ANOMALY_DETECTORS_INDEX) .setRefreshPolicy(refreshPolicy) @@ -462,7 +464,7 @@ private void indexAnomalyDetector(String detectorId) throws IOException { client.index(indexRequest, new ActionListener() { @Override public void onResponse(IndexResponse indexResponse) { - String errorMsg = checkShardsFailure(indexResponse); + String errorMsg = getShardsFailure(indexResponse); if (errorMsg != null) { listener.onFailure(new ElasticsearchStatusException(errorMsg, indexResponse.status())); return; @@ -503,14 +505,4 @@ private void onCreateMappingsResponse(CreateIndexResponse response) throws IOExc } } - private String checkShardsFailure(IndexResponse response) { - StringBuilder failureReasons = new StringBuilder(); - if (response.getShardInfo().getFailed() > 0) { - for (ReplicationResponse.ShardInfo.Failure failure : response.getShardInfo().getFailures()) { - failureReasons.append(failure); - } - return failureReasons.toString(); - } - return null; - } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java index f547c61a..f1eb79bc 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/rest/handler/IndexAnomalyDetectorJobActionHandler.java @@ -16,6 +16,7 @@ package com.amazon.opendistroforelasticsearch.ad.rest.handler; import static com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector.ANOMALY_DETECTORS_INDEX; +import static com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtil.getShardsFailure; import static org.elasticsearch.action.DocWriteResponse.Result.CREATED; import static org.elasticsearch.action.DocWriteResponse.Result.UPDATED; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; @@ -28,13 +29,11 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -104,75 +103,34 @@ public IndexAnomalyDetectorJobActionHandler( /** * Start anomaly detector job. - * 1.If job not exists, create new job. - * 2.If job exists: a). if job enabled, return error message; b). if job disabled, enable job. - * - * @throws IOException IOException from {@link AnomalyDetectionIndices#getAnomalyDetectorJobMappings} + * 1. If job doesn't exist, create new job. + * 2. If job exists: a). if job enabled, return error message; b). if job disabled, enable job. + * @param detector anomaly detector */ - public void startAnomalyDetectorJob() throws IOException { + public void startAnomalyDetectorJob(AnomalyDetector detector) { if (!anomalyDetectionIndices.doesAnomalyDetectorJobIndexExist()) { - anomalyDetectionIndices - .initAnomalyDetectorJobIndex( - ActionListener.wrap(response -> onCreateMappingsResponse(response), exception -> listener.onFailure(exception)) - ); - } else { - prepareAnomalyDetectorJobIndexing(); - } - } - - private void onCreateMappingsResponse(CreateIndexResponse response) throws IOException { - if (response.isAcknowledged()) { - logger.info("Created {} with mappings.", ANOMALY_DETECTORS_INDEX); - prepareAnomalyDetectorJobIndexing(); + anomalyDetectionIndices.initAnomalyDetectorJobIndex(ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + logger.info("Created {} with mappings.", ANOMALY_DETECTORS_INDEX); + createJob(detector); + } else { + logger.warn("Created {} with mappings call not acknowledged.", ANOMALY_DETECTORS_INDEX); + listener + .onFailure( + new ElasticsearchStatusException( + "Created " + ANOMALY_DETECTORS_INDEX + " with mappings call not acknowledged.", + RestStatus.INTERNAL_SERVER_ERROR + ) + ); + } + }, exception -> listener.onFailure(exception))); } else { - logger.warn("Created {} with mappings call not acknowledged.", ANOMALY_DETECTORS_INDEX); - listener - .onFailure( - new ElasticsearchStatusException( - "Created " + ANOMALY_DETECTORS_INDEX + " with mappings call not acknowledged.", - RestStatus.INTERNAL_SERVER_ERROR - ) - ); + createJob(detector); } } - private void prepareAnomalyDetectorJobIndexing() { - GetRequest getRequest = new GetRequest(AnomalyDetector.ANOMALY_DETECTORS_INDEX).id(detectorId); - client - .get( - getRequest, - ActionListener.wrap(response -> onGetAnomalyDetectorResponse(response), exception -> listener.onFailure(exception)) - ); - } - - private void onGetAnomalyDetectorResponse(GetResponse response) throws IOException { - if (!response.isExists()) { - listener - .onFailure(new ElasticsearchStatusException("AnomalyDetector is not found with id: " + detectorId, RestStatus.NOT_FOUND)); - return; - } - try (XContentParser parser = RestHandlerUtils.createXContentParserFromRegistry(xContentRegistry, response.getSourceAsBytesRef())) { - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); - AnomalyDetector detector = AnomalyDetector.parse(parser, response.getId(), response.getVersion()); - - if (detector.getFeatureAttributes().size() == 0) { - listener - .onFailure( - new ElasticsearchStatusException("Can't start detector job as no features configured", RestStatus.BAD_REQUEST) - ); - return; - } - if (detector.getEnabledFeatureIds().size() == 0) { - listener - .onFailure( - new ElasticsearchStatusException( - "Can't start detector job as no enabled features configured", - RestStatus.BAD_REQUEST - ) - ); - return; - } - + private void createJob(AnomalyDetector detector) { + try { IntervalTimeConfiguration interval = (IntervalTimeConfiguration) detector.getDetectionInterval(); Schedule schedule = new IntervalSchedule(Instant.now(), (int) interval.getInterval(), interval.getUnit()); Duration duration = Duration.of(interval.getInterval(), interval.getUnit()); @@ -190,7 +148,7 @@ private void onGetAnomalyDetectorResponse(GetResponse response) throws IOExcepti ); getAnomalyDetectorJobForWrite(job); - } catch (IOException e) { + } catch (Exception e) { String message = "Failed to parse anomaly detector job " + detectorId; logger.error(message, e); listener.onFailure(new ElasticsearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR)); @@ -262,7 +220,7 @@ private void indexAnomalyDetectorJob(AnomalyDetectorJob job, AnomalyDetectorFunc private void onIndexAnomalyDetectorJobResponse(IndexResponse response, AnomalyDetectorFunction function) throws IOException { if (response == null || (response.getResult() != CREATED && response.getResult() != UPDATED)) { - String errorMsg = checkShardsFailure(response); + String errorMsg = getShardsFailure(response); listener.onFailure(new ElasticsearchStatusException(errorMsg, response.status())); return; } @@ -363,15 +321,4 @@ public void onFailure(Exception e) { }; } - private String checkShardsFailure(IndexResponse response) { - StringBuilder failureReasons = new StringBuilder(); - if (response.getShardInfo().getFailed() > 0) { - for (ReplicationResponse.ShardInfo.Failure failure : response.getShardInfo().getFailures()) { - failureReasons.append(failure); - } - return failureReasons.toString(); - } - return null; - } - } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/settings/AnomalyDetectorSettings.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/settings/AnomalyDetectorSettings.java index 7077b3a6..ea30e5f6 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/settings/AnomalyDetectorSettings.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/settings/AnomalyDetectorSettings.java @@ -335,4 +335,37 @@ private AnomalyDetectorSettings() {} ); public static int THRESHOLD_MODEL_TRAINING_SIZE = 1000; + + public static final Setting MAX_OLD_AD_TASK_DOCS_PER_DETECTOR = Setting + .intSetting( + "opendistro.anomaly_detection.max_old_ad_task_docs_per_detector", + // One AD task is roughly 1.5KB for normal case. Suppose task's size + // is 2KB conservatively. If we store 1000 AD tasks for one detector, + // that will be 2GB. + 10, + 1, // keep at least 1 old AD task per detector + 1000, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting BATCH_TASK_PIECE_SIZE = Setting + .intSetting( + "opendistro.anomaly_detection.batch_task_piece_size", + 1000, + 1, + 10_000, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting BATCH_TASK_PIECE_INTERVAL_SECONDS = Setting + .intSetting( + "opendistro.anomaly_detection.batch_task_piece_interval_seconds", + 5, + 1, + 600, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/ADStat.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/ADStat.java index a5f5f228..3d61c762 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/ADStat.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/ADStat.java @@ -75,4 +75,13 @@ public void increment() { ((CounterSupplier) supplier).increment(); } } + + /** + * Decrease the supplier if it can be decreased. + */ + public void decrement() { + if (supplier instanceof CounterSupplier) { + ((CounterSupplier) supplier).decrement(); + } + } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/suppliers/CounterSupplier.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/suppliers/CounterSupplier.java index cd6655d8..c9dc2641 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/suppliers/CounterSupplier.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/stats/suppliers/CounterSupplier.java @@ -42,4 +42,11 @@ public Long get() { public void increment() { counter.increment(); } + + /** + * Decrease the value of the counter by 1 + */ + public void decrement() { + counter.decrement(); + } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADBatchTaskRunner.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADBatchTaskRunner.java new file mode 100644 index 00000000..c284c41f --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADBatchTaskRunner.java @@ -0,0 +1,731 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.task; + +import static com.amazon.opendistroforelasticsearch.ad.AnomalyDetectorPlugin.AD_BATCH_TASK_THREAD_POOL_NAME; +import static com.amazon.opendistroforelasticsearch.ad.breaker.MemoryCircuitBreaker.DEFAULT_JVM_HEAP_USAGE_THRESHOLD; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.AGG_NAME_MAX_TIME; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.AGG_NAME_MIN_TIME; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.CURRENT_PIECE_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.EXECUTION_END_TIME_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.INIT_PROGRESS_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.STATE_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.TASK_PROGRESS_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.BATCH_TASK_PIECE_INTERVAL_SECONDS; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.BATCH_TASK_PIECE_SIZE; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.NUM_MIN_SAMPLES; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.THRESHOLD_MODEL_TRAINING_SIZE; +import static com.amazon.opendistroforelasticsearch.ad.stats.InternalStatNames.JVM_HEAP_USAGE; +import static com.amazon.opendistroforelasticsearch.ad.stats.StatNames.AD_EXECUTING_BATCH_TASK_COUNT; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Deque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.support.ThreadedActionListener; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.metrics.InternalMax; +import org.elasticsearch.search.aggregations.metrics.InternalMin; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportService; + +import com.amazon.opendistroforelasticsearch.ad.breaker.ADCircuitBreakerService; +import com.amazon.opendistroforelasticsearch.ad.common.exception.ADTaskCancelledException; +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; +import com.amazon.opendistroforelasticsearch.ad.common.exception.EndRunException; +import com.amazon.opendistroforelasticsearch.ad.common.exception.LimitExceededException; +import com.amazon.opendistroforelasticsearch.ad.common.exception.ResourceNotFoundException; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonErrorMessages; +import com.amazon.opendistroforelasticsearch.ad.feature.FeatureManager; +import com.amazon.opendistroforelasticsearch.ad.feature.SinglePointFeatures; +import com.amazon.opendistroforelasticsearch.ad.indices.ADIndex; +import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.ml.ThresholdingModel; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskState; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyResult; +import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; +import com.amazon.opendistroforelasticsearch.ad.model.FeatureData; +import com.amazon.opendistroforelasticsearch.ad.model.IntervalTimeConfiguration; +import com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings; +import com.amazon.opendistroforelasticsearch.ad.settings.EnabledSetting; +import com.amazon.opendistroforelasticsearch.ad.stats.ADStats; +import com.amazon.opendistroforelasticsearch.ad.stats.StatNames; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultRequest; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultResponse; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchTaskRemoteExecutionAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADStatsNodeResponse; +import com.amazon.opendistroforelasticsearch.ad.transport.ADStatsNodesAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADStatsRequest; +import com.amazon.opendistroforelasticsearch.ad.transport.handler.AnomalyResultBulkIndexHandler; +import com.amazon.opendistroforelasticsearch.ad.util.DiscoveryNodeFilterer; +import com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtil; +import com.amazon.opendistroforelasticsearch.ad.util.ParseUtils; +import com.amazon.randomcutforest.RandomCutForest; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.RateLimiter; + +public class ADBatchTaskRunner { + private final Logger logger = LogManager.getLogger(ADBatchTaskRunner.class); + + private final RateLimiter rateLimiter = RateLimiter.create(1); + + private final ThreadPool threadPool; + private final Client client; + private final ADStats adStats; + private final DiscoveryNodeFilterer nodeFilter; + private final ClusterService clusterService; + private final FeatureManager featureManager; + private final ADCircuitBreakerService adCircuitBreakerService; + private final ADTaskManager adTaskManager; + private final AnomalyResultBulkIndexHandler anomalyResultBulkIndexHandler; + private final IndexNameExpressionResolver indexNameExpressionResolver; + private AnomalyDetectionIndices anomalyDetectionIndices; + + private final ADTaskCacheManager adTaskCacheManager; + private final TransportRequestOptions option; + + private volatile Integer maxAdBatchTaskPerNode; + private volatile Integer pieceSize; + private volatile Integer pieceIntervalSeconds; + + public ADBatchTaskRunner( + Settings settings, + ThreadPool threadPool, + ClusterService clusterService, + Client client, + DiscoveryNodeFilterer nodeFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + ADCircuitBreakerService adCircuitBreakerService, + FeatureManager featureManager, + ADTaskManager adTaskManager, + AnomalyDetectionIndices anomalyDetectionIndices, + ADStats adStats, + AnomalyResultBulkIndexHandler anomalyResultBulkIndexHandler, + ADTaskCacheManager adTaskCacheManager + ) { + this.threadPool = threadPool; + this.clusterService = clusterService; + this.client = client; + this.anomalyResultBulkIndexHandler = anomalyResultBulkIndexHandler; + this.indexNameExpressionResolver = indexNameExpressionResolver; + this.nodeFilter = nodeFilter; + this.adStats = adStats; + this.adCircuitBreakerService = adCircuitBreakerService; + this.adTaskManager = adTaskManager; + this.featureManager = featureManager; + this.anomalyDetectionIndices = anomalyDetectionIndices; + + this.option = TransportRequestOptions + .builder() + .withType(TransportRequestOptions.Type.REG) + .withTimeout(AnomalyDetectorSettings.REQUEST_TIMEOUT.get(settings)) + .build(); + + this.adTaskCacheManager = adTaskCacheManager; + + this.maxAdBatchTaskPerNode = MAX_BATCH_TASK_PER_NODE.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_BATCH_TASK_PER_NODE, it -> maxAdBatchTaskPerNode = it); + + this.pieceSize = BATCH_TASK_PIECE_SIZE.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(BATCH_TASK_PIECE_SIZE, it -> pieceSize = it); + + this.pieceIntervalSeconds = BATCH_TASK_PIECE_INTERVAL_SECONDS.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(BATCH_TASK_PIECE_INTERVAL_SECONDS, it -> pieceIntervalSeconds = it); + } + + /** + * Run AD task. + * 1. Set AD task state as {@link ADTaskState#INIT} + * 2. Gather node stats and find node with least load to run AD task. + * + * @param adTask AD task + * @param transportService transport service + * @param listener action listener + */ + public void run(ADTask adTask, TransportService transportService, ActionListener listener) { + Map updatedFields = new HashMap<>(); + updatedFields.put(STATE_FIELD, ADTaskState.INIT.name()); + updatedFields.put(INIT_PROGRESS_FIELD, 0.0f); + + ActionListener delegatedListener = ActionListener.wrap(r -> { listener.onResponse(r); }, e -> { + listener.onFailure(e); + handleException(adTask, e); + }); + + adTaskManager + .updateADTask(adTask.getTaskId(), updatedFields, ActionListener.wrap(r -> dispatchTask(adTask, ActionListener.wrap(node -> { + if (clusterService.localNode().getId().equals(node.getId())) { + // Execute batch task locally + logger + .info( + "execute AD task {} locally on node {} for detector {}", + adTask.getTaskId(), + node.getId(), + adTask.getDetectorId() + ); + startADBatchTask(adTask, false, delegatedListener); + } else { + // Execute batch task remotely + logger + .info( + "execute AD task {} remotely on node {} for detector {}", + adTask.getTaskId(), + node.getId(), + adTask.getDetectorId() + ); + transportService + .sendRequest( + node, + ADBatchTaskRemoteExecutionAction.NAME, + new ADBatchAnomalyResultRequest(adTask), + option, + new ActionListenerResponseHandler<>(delegatedListener, ADBatchAnomalyResultResponse::new) + ); + } + }, e -> delegatedListener.onFailure(e))), e -> delegatedListener.onFailure(e))); + } + + private void dispatchTask(ADTask adTask, ActionListener listener) { + DiscoveryNode[] dataNodes = nodeFilter.getEligibleDataNodes(); + ADStatsRequest adStatsRequest = new ADStatsRequest(dataNodes); + adStatsRequest.addAll(ImmutableSet.of(AD_EXECUTING_BATCH_TASK_COUNT.getName(), JVM_HEAP_USAGE.getName())); + + client.execute(ADStatsNodesAction.INSTANCE, adStatsRequest, ActionListener.wrap(adStatsResponse -> { + List candidateNodeResponse = adStatsResponse + .getNodes() + .stream() + .filter(stat -> (long) stat.getStatsMap().get(JVM_HEAP_USAGE.getName()) < DEFAULT_JVM_HEAP_USAGE_THRESHOLD) + .collect(Collectors.toList()); + + if (candidateNodeResponse.size() == 0) { + String errorMessage = "All nodes' memory usage exceeds limitation" + + DEFAULT_JVM_HEAP_USAGE_THRESHOLD + + ". No eligible node to run detector " + + adTask.getDetectorId(); + logger.warn(errorMessage); + listener.onFailure(new LimitExceededException(adTask.getDetectorId(), errorMessage)); + return; + } + candidateNodeResponse = candidateNodeResponse + .stream() + .filter(stat -> (Long) stat.getStatsMap().get(AD_EXECUTING_BATCH_TASK_COUNT.getName()) < maxAdBatchTaskPerNode) + .collect(Collectors.toList()); + if (candidateNodeResponse.size() == 0) { + String errorMessage = "All nodes' executing historical detector count exceeds limitation. No eligible node to run detector " + + adTask.getDetectorId(); + logger.warn(errorMessage); + listener.onFailure(new LimitExceededException(adTask.getDetectorId(), errorMessage)); + return; + } + Optional targetNode = candidateNodeResponse + .stream() + .sorted((ADStatsNodeResponse r1, ADStatsNodeResponse r2) -> { + int result = ((Long) r1.getStatsMap().get(AD_EXECUTING_BATCH_TASK_COUNT.getName())) + .compareTo((Long) r2.getStatsMap().get(AD_EXECUTING_BATCH_TASK_COUNT.getName())); + if (result == 0) { + // if multiple nodes have same running task count, choose the one with least + // JVM heap usage. + return ((Long) r1.getStatsMap().get(JVM_HEAP_USAGE.getName())) + .compareTo((Long) r2.getStatsMap().get(JVM_HEAP_USAGE.getName())); + } + return result; + }) + .findFirst(); + listener.onResponse(targetNode.get().getNode()); + }, exception -> { + logger.error("Failed to get node's task stats", exception); + listener.onFailure(exception); + })); + } + + /** + * Start AD task in dedicated batch task thread pool. + * + * @param adTask ad task + * @param runTaskRemotely run task remotely or not + * @param listener action listener + */ + public void startADBatchTask(ADTask adTask, boolean runTaskRemotely, ActionListener listener) { + try { + // check if cluster is eligible to run AD currently, if not eligible like + // circuit breaker open, will throw exception. + checkClusterState(adTask); + threadPool.executor(AD_BATCH_TASK_THREAD_POOL_NAME).execute(() -> { + ActionListener internalListener = internalBatchTaskListener(adTask); + try { + executeADBatchTask(adTask, internalListener); + } catch (Exception e) { + internalListener.onFailure(e); + } + }); + listener.onResponse(new ADBatchAnomalyResultResponse(clusterService.localNode().getId(), runTaskRemotely)); + } catch (Exception e) { + logger.error("Fail to start AD batch task " + adTask.getTaskId(), e); + listener.onFailure(e); + } + } + + private ActionListener internalBatchTaskListener(ADTask adTask) { + String taskId = adTask.getTaskId(); + ActionListener listener = ActionListener.wrap(response -> { + // If batch task finished normally, remove task from cache and decrease executing task count by 1. + adTaskCacheManager.remove(taskId); + adStats.getStat(AD_EXECUTING_BATCH_TASK_COUNT.getName()).decrement(); + }, e -> { + // If batch task failed, remove task from cache and decrease executing task count by 1. + adTaskCacheManager.remove(taskId); + adStats.getStat(AD_EXECUTING_BATCH_TASK_COUNT.getName()).decrement(); + handleException(adTask, e); + }); + return listener; + } + + private void handleException(ADTask adTask, Exception e) { + logger.debug("Failed to run task " + adTask.getTaskId() + " for detector " + adTask.getDetectorId(), e); + // Check if batch task was cancelled or not by exception type. + // If it's cancelled, then increase cancelled task count by 1, otherwise increase failure count by 1. + if (e instanceof ADTaskCancelledException) { + adStats.getStat(StatNames.AD_CANCELED_BATCH_TASK_COUNT.getName()).increment(); + } else if (ExceptionUtil.countInStats(e)) { + adStats.getStat(StatNames.AD_BATCH_TASK_FAILURE_COUNT.getName()).increment(); + } + // Handle AD task exception + adTaskManager.handleADTaskException(adTask, e); + } + + private void executeADBatchTask(ADTask adTask, ActionListener internalListener) { + // track AD executing batch task and total batch task execution count + adStats.getStat(AD_EXECUTING_BATCH_TASK_COUNT.getName()).increment(); + adStats.getStat(StatNames.AD_TOTAL_BATCH_TASK_EXECUTION_COUNT.getName()).increment(); + + // put AD task into cache + adTaskCacheManager.put(adTask); + + // start to run first piece + Instant executeStartTime = Instant.now(); + // TODO: refactor to make the workflow more clear + runFirstPiece(adTask, executeStartTime, internalListener); + } + + private void checkClusterState(ADTask adTask) { + // check if AD plugin is enabled + checkADPluginEnabled(adTask.getDetectorId()); + + // check if circuit breaker is open + checkCircuitBreaker(adTask); + } + + private void checkADPluginEnabled(String detectorId) { + if (!EnabledSetting.isADPluginEnabled()) { + throw new EndRunException(detectorId, CommonErrorMessages.DISABLED_ERR_MSG, true).countedInStats(false); + } + } + + private void checkCircuitBreaker(ADTask adTask) { + String taskId = adTask.getTaskId(); + if (adCircuitBreakerService.isOpen()) { + String error = "Circuit breaker is open"; + logger.error("AD task: {}, {}", taskId, error); + throw new LimitExceededException(adTask.getDetectorId(), error, true); + } + } + + private void runFirstPiece(ADTask adTask, Instant executeStartTime, ActionListener internalListener) { + try { + adTaskManager + .updateADTask( + adTask.getTaskId(), + ImmutableMap + .of( + STATE_FIELD, + ADTaskState.INIT.name(), + CURRENT_PIECE_FIELD, + adTask.getDetector().getDetectionDateRange().getStartTime().toEpochMilli(), + TASK_PROGRESS_FIELD, + 0.0f, + INIT_PROGRESS_FIELD, + 0.0f + ), + ActionListener.wrap(r -> { + try { + checkIfADTaskCancelled(adTask.getTaskId()); + getDateRangeOfSourceData(adTask, (minDate, maxDate) -> { + long interval = ((IntervalTimeConfiguration) adTask.getDetector().getDetectionInterval()) + .toDuration() + .toMillis(); + + DetectionDateRange detectionDateRange = adTask.getDetector().getDetectionDateRange(); + long dataStartTime = detectionDateRange.getStartTime().toEpochMilli(); + long dataEndTime = detectionDateRange.getEndTime().toEpochMilli(); + + if (minDate >= dataEndTime || maxDate <= dataStartTime) { + internalListener + .onFailure( + new ResourceNotFoundException( + adTask.getDetectorId(), + "There is no data in the detection date range" + ) + ); + return; + } + if (minDate > dataStartTime) { + dataStartTime = minDate; + } + if (maxDate < dataEndTime) { + dataEndTime = maxDate; + } + + // normalize start/end time to make it consistent with feature data agg result + dataStartTime = dataStartTime - dataStartTime % interval; + dataEndTime = dataEndTime - dataEndTime % interval; + if ((dataEndTime - dataStartTime) < THRESHOLD_MODEL_TRAINING_SIZE * interval) { + internalListener + .onFailure( + new AnomalyDetectionException("There is no enough data to train model").countedInStats(false) + ); + return; + } + long expectedPieceEndTime = dataStartTime + pieceSize * interval; + long firstPieceEndTime = Math.min(expectedPieceEndTime, dataEndTime); + logger + .debug( + "start first piece from {} to {}, interval {}, dataStartTime {}, dataEndTime {}," + + " detectorId {}, taskId {}", + dataStartTime, + firstPieceEndTime, + interval, + dataStartTime, + dataEndTime, + adTask.getDetectorId(), + adTask.getTaskId() + ); + getFeatureData( + adTask, + dataStartTime, // first piece start time + firstPieceEndTime, // first piece end time + dataStartTime, + dataEndTime, + interval, + executeStartTime, + internalListener + ); + }, internalListener); + } catch (Exception e) { + internalListener.onFailure(e); + } + }, internalListener::onFailure) + ); + } catch (Exception exception) { + internalListener.onFailure(exception); + } + } + + private void getDateRangeOfSourceData(ADTask adTask, BiConsumer consumer, ActionListener listener) { + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() + .aggregation(AggregationBuilders.min(AGG_NAME_MIN_TIME).field(adTask.getDetector().getTimeField())) + .aggregation(AggregationBuilders.max(AGG_NAME_MAX_TIME).field(adTask.getDetector().getTimeField())) + .size(0); + SearchRequest request = new SearchRequest() + .indices(adTask.getDetector().getIndices().toArray(new String[0])) + .source(searchSourceBuilder); + + client.search(request, ActionListener.wrap(r -> { + InternalMin minAgg = r.getAggregations().get(AGG_NAME_MIN_TIME); + InternalMax maxAgg = r.getAggregations().get(AGG_NAME_MAX_TIME); + double minValue = minAgg.getValue(); + double maxValue = maxAgg.getValue(); + // If time field not exist or there is no value, will return infinity value + if (minValue == Double.POSITIVE_INFINITY) { + listener.onFailure(new ResourceNotFoundException(adTask.getDetectorId(), "There is no data in the time field")); + return; + } + consumer.accept((long) minValue, (long) maxValue); + }, e -> { listener.onFailure(e); })); + } + + private void getFeatureData( + ADTask adTask, + long pieceStartTime, + long pieceEndTime, + long dataStartTime, + long dataEndTime, + long interval, + Instant executeStartTime, + ActionListener internalListener + ) { + ActionListener>> actionListener = ActionListener.wrap(dataPoints -> { + try { + if (dataPoints.size() == 0) { + logger.debug("No data in current piece with end time: " + pieceEndTime); + runNextPiece(adTask, pieceEndTime, dataStartTime, dataEndTime, interval, internalListener); + } else { + detectAnomaly( + adTask, + dataPoints, + pieceStartTime, + pieceEndTime, + dataStartTime, + dataEndTime, + interval, + executeStartTime, + internalListener + ); + } + } catch (Exception e) { + internalListener.onFailure(e); + } + }, exception -> { + logger.debug("Fail to get feature data by batch for this piece with end time: " + pieceEndTime); + // TODO: Exception may be caused by wrong feature query or some bad data. Differentiate these + // and skip current piece if error caused by bad data. + internalListener.onFailure(exception); + }); + ThreadedActionListener>> threadedActionListener = new ThreadedActionListener<>( + logger, + threadPool, + AD_BATCH_TASK_THREAD_POOL_NAME, + actionListener, + false + ); + + featureManager.getFeatureDataPointsByBatch(adTask.getDetector(), pieceStartTime, pieceEndTime, threadedActionListener); + } + + private void detectAnomaly( + ADTask adTask, + Map> dataPoints, + long pieceStartTime, + long pieceEndTime, + long dataStartTime, + long dataEndTime, + long interval, + Instant executeStartTime, + ActionListener internalListener + ) { + String taskId = adTask.getTaskId(); + RandomCutForest rcf = adTaskCacheManager.getRcfModel(taskId); + ThresholdingModel threshold = adTaskCacheManager.getThresholdModel(taskId); + Deque>> shingle = adTaskCacheManager.getShingle(taskId); + + List anomalyResults = new ArrayList<>(); + + long intervalEndTime = pieceStartTime; + for (int i = 0; i < pieceSize && intervalEndTime < dataEndTime; i++) { + Optional dataPoint = dataPoints.containsKey(intervalEndTime) ? dataPoints.get(intervalEndTime) : Optional.empty(); + intervalEndTime = intervalEndTime + interval; + SinglePointFeatures feature = featureManager + .getShingledFeatureForHistoricalDetector(adTask.getDetector(), shingle, dataPoint, intervalEndTime); + List featureData = null; + if (feature.getUnprocessedFeatures().isPresent()) { + featureData = ParseUtils.getFeatureData(feature.getUnprocessedFeatures().get(), adTask.getDetector()); + } + if (!feature.getProcessedFeatures().isPresent()) { + String error = feature.getUnprocessedFeatures().isPresent() + ? "No full shingle in current detection window" + : "No data in current detection window"; + AnomalyResult anomalyResult = new AnomalyResult( + adTask.getDetectorId(), + taskId, + Double.NaN, + Double.NaN, + Double.NaN, + featureData, + Instant.ofEpochMilli(intervalEndTime - interval), + Instant.ofEpochMilli(intervalEndTime), + executeStartTime, + Instant.now(), + error, + null, + adTask.getDetector().getUser(), + anomalyDetectionIndices.getSchemaVersion(ADIndex.RESULT) + ); + anomalyResults.add(anomalyResult); + } else { + double[] point = feature.getProcessedFeatures().get(); + double score = rcf.getAnomalyScore(point); + rcf.update(point); + double grade = 0d; + double confidence = 0d; + if (!adTaskCacheManager.isThresholdModelTrained(taskId)) { + if (adTaskCacheManager.getThresholdModelTrainingDataSize(taskId) < THRESHOLD_MODEL_TRAINING_SIZE) { + if (score > 0) { + adTaskCacheManager.addThresholdModelTrainingData(taskId, score); + } + } else { + logger.debug("training threshold model"); + threshold.train(adTaskCacheManager.getThresholdModelTrainingData(taskId)); + adTaskCacheManager.setThresholdModelTrained(taskId, true); + } + } else { + grade = threshold.grade(score); + confidence = threshold.confidence(); + if (score > 0) { + threshold.update(score); + } + } + + AnomalyResult anomalyResult = new AnomalyResult( + adTask.getDetectorId(), + taskId, + score, + grade, + confidence, + featureData, + Instant.ofEpochMilli(intervalEndTime - interval), + Instant.ofEpochMilli(intervalEndTime), + executeStartTime, + Instant.now(), + null, + null, + adTask.getDetector().getUser(), + anomalyDetectionIndices.getSchemaVersion(ADIndex.RESULT) + ); + anomalyResults.add(anomalyResult); + } + } + + anomalyResultBulkIndexHandler + .bulkIndexAnomalyResult( + anomalyResults, + new ThreadedActionListener<>(logger, threadPool, AD_BATCH_TASK_THREAD_POOL_NAME, ActionListener.wrap(r -> { + try { + runNextPiece(adTask, pieceEndTime, dataStartTime, dataEndTime, interval, internalListener); + } catch (Exception e) { + internalListener.onFailure(e); + } + }, e -> { + logger.error("Fail to bulk index anomaly result", e); + internalListener.onFailure(e); + }), false) + ); + } + + private void runNextPiece( + ADTask adTask, + long pieceStartTime, + long dataStartTime, + long dataEndTime, + long interval, + ActionListener internalListener + ) { + String taskId = adTask.getTaskId(); + float initProgress = calculateInitProgress(taskId); + String taskState = initProgress >= 1.0f ? ADTaskState.RUNNING.name() : ADTaskState.INIT.name(); + + if (pieceStartTime < dataEndTime) { + checkClusterState(adTask); + long expectedPieceEndTime = pieceStartTime + pieceSize * interval; + long pieceEndTime = expectedPieceEndTime > dataEndTime ? dataEndTime : expectedPieceEndTime; + int i = 0; + while (i < pieceIntervalSeconds) { + // check if task cancelled every second, so frontend can get STOPPED state + // in 1 second once task cancelled. + checkIfADTaskCancelled(taskId); + rateLimiter.acquire(1); + i++; + } + logger.debug("start next piece start from {} to {}, interval {}", pieceStartTime, pieceEndTime, interval); + float taskProgress = (float) (pieceStartTime - dataStartTime) / (dataEndTime - dataStartTime); + adTaskManager + .updateADTask( + taskId, + ImmutableMap + .of( + STATE_FIELD, + taskState, + CURRENT_PIECE_FIELD, + pieceStartTime, + TASK_PROGRESS_FIELD, + taskProgress, + INIT_PROGRESS_FIELD, + initProgress + ), + ActionListener + .wrap( + r -> getFeatureData( + adTask, + pieceStartTime, + pieceEndTime, + dataStartTime, + dataEndTime, + interval, + Instant.now(), + internalListener + ), + e -> internalListener.onFailure(e) + ) + ); + } else { + logger.info("AD task finished for detector {}, task id: {}", adTask.getDetectorId(), taskId); + adTaskCacheManager.remove(taskId); + adTaskManager + .updateADTask( + taskId, + ImmutableMap + .of( + STATE_FIELD, + ADTaskState.FINISHED.name(), + CURRENT_PIECE_FIELD, + dataEndTime, + TASK_PROGRESS_FIELD, + 1.0f, + EXECUTION_END_TIME_FIELD, + Instant.now().toEpochMilli(), + INIT_PROGRESS_FIELD, + initProgress + ), + ActionListener.wrap(r -> internalListener.onResponse("task execution done"), e -> internalListener.onFailure(e)) + ); + } + } + + private float calculateInitProgress(String taskId) { + RandomCutForest rcf = adTaskCacheManager.getRcfModel(taskId); + if (rcf == null) { + return 0.0f; + } + float initProgress = (float) rcf.getTotalUpdates() / NUM_MIN_SAMPLES; + return initProgress > 1.0f ? 1.0f : initProgress; + } + + private void checkIfADTaskCancelled(String taskId) { + if (adTaskCacheManager.contains(taskId) && adTaskCacheManager.isCancelled(taskId)) { + throw new ADTaskCancelledException(adTaskCacheManager.getCancelReason(taskId), adTaskCacheManager.getCancelledBy(taskId)); + } + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskCacheManager.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskCacheManager.java index 8b203b68..764e6706 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskCacheManager.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskCacheManager.java @@ -131,6 +131,10 @@ public double[] getThresholdModelTrainingData(String taskId) { return getBatchTaskCache(taskId).getThresholdModelTrainingData(); } + public int getThresholdModelTrainingDataSize(String taskId) { + return getBatchTaskCache(taskId).getThresholdModelTrainingDataSize().get(); + } + public int addThresholdModelTrainingData(String taskId, double... data) { ADBatchTaskCache taskCache = getBatchTaskCache(taskId); double[] thresholdModelTrainingData = taskCache.getThresholdModelTrainingData(); diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManager.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManager.java new file mode 100644 index 00000000..1391bdac --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManager.java @@ -0,0 +1,458 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.task; + +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.DETECTOR_ID_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.ERROR_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.EXECUTION_END_TIME_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.EXECUTION_START_TIME_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.IS_LATEST_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.LAST_UPDATE_TIME_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.STATE_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_OLD_AD_TASK_DOCS_PER_DETECTOR; +import static com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtil.getErrorMessage; +import static com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtil.getShardsFailure; +import static org.elasticsearch.action.DocWriteResponse.Result.CREATED; +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; + +import java.time.Instant; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.action.update.UpdateResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.RangeQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.query.TermsQueryBuilder; +import org.elasticsearch.index.reindex.DeleteByQueryAction; +import org.elasticsearch.index.reindex.DeleteByQueryRequest; +import org.elasticsearch.index.reindex.UpdateByQueryAction; +import org.elasticsearch.index.reindex.UpdateByQueryRequest; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.script.Script; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.sort.SortOrder; + +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; +import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskState; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskType; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultAction; +import com.amazon.opendistroforelasticsearch.ad.transport.ADBatchAnomalyResultRequest; +import com.amazon.opendistroforelasticsearch.ad.transport.AnomalyDetectorJobResponse; +import com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils; +import com.amazon.opendistroforelasticsearch.commons.authuser.User; + +/** + * Manage AD task. + */ +public class ADTaskManager { + private final Logger logger = LogManager.getLogger(this.getClass()); + + private final Client client; + private final NamedXContentRegistry xContentRegistry; + private final AnomalyDetectionIndices detectionIndices; + private volatile Integer maxAdTaskDocsPerDetector; + + public ADTaskManager( + Settings settings, + ClusterService clusterService, + Client client, + NamedXContentRegistry xContentRegistry, + AnomalyDetectionIndices detectionIndices + ) { + this.client = client; + this.xContentRegistry = xContentRegistry; + this.detectionIndices = detectionIndices; + + this.maxAdTaskDocsPerDetector = MAX_OLD_AD_TASK_DOCS_PER_DETECTOR.get(settings); + clusterService + .getClusterSettings() + .addSettingsUpdateConsumer(MAX_OLD_AD_TASK_DOCS_PER_DETECTOR, it -> maxAdTaskDocsPerDetector = it); + } + + /** + * Start detector. Will create schedule job for realtime detector, + * and start AD task for historical detector. + * + * @param detectorId detector id + * @param handler anomaly detector job action handler + * @param user user + * @param listener action listener + */ + public void startDetector( + String detectorId, + IndexAnomalyDetectorJobActionHandler handler, + User user, + ActionListener listener + ) { + getDetector( + detectorId, + (detector) -> handler.startAnomalyDetectorJob(detector), // run realtime detector + (detector) -> createADTaskIndex(detector, user, listener), // run historical detector + listener + ); + } + + private void getDetector( + String detectorId, + Consumer realTimeDetectorConsumer, + Consumer historicalDetectorConsumer, + ActionListener listener + ) { + GetRequest getRequest = new GetRequest(AnomalyDetector.ANOMALY_DETECTORS_INDEX).id(detectorId); + client.get(getRequest, ActionListener.wrap(response -> { + if (!response.isExists()) { + listener + .onFailure( + new ElasticsearchStatusException("AnomalyDetector is not found with id: " + detectorId, RestStatus.NOT_FOUND) + ); + return; + } + try ( + XContentParser parser = RestHandlerUtils.createXContentParserFromRegistry(xContentRegistry, response.getSourceAsBytesRef()) + ) { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + AnomalyDetector detector = AnomalyDetector.parse(parser, response.getId(), response.getVersion()); + + String error = validateDetector(detector); + if (error != null) { + listener.onFailure(new ElasticsearchStatusException(error, RestStatus.BAD_REQUEST)); + return; + } + + if (detector.isRealTimeDetector()) { + // run realtime detector + realTimeDetectorConsumer.accept(detector); + } else { + // run historical detector + historicalDetectorConsumer.accept(detector); + } + } catch (Exception e) { + String message = "Failed to start anomaly detector"; + logger.error(message, e); + listener.onFailure(new ElasticsearchStatusException(message, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, exception -> listener.onFailure(exception))); + } + + private String validateDetector(AnomalyDetector detector) { + if (detector.getFeatureAttributes().size() == 0) { + return "Can't start detector job as no features configured"; + } + if (detector.getEnabledFeatureIds().size() == 0) { + return "Can't start detector job as no enabled features configured"; + } + return null; + } + + protected void createADTaskIndex(AnomalyDetector detector, User user, ActionListener listener) { + if (detectionIndices.doesDetectorStateIndexExist()) { + checkCurrentTaskState(detector, user, listener); + } else { + detectionIndices.initDetectionStateIndex(ActionListener.wrap(r -> { + if (r.isAcknowledged()) { + logger.info("Created {} with mappings.", CommonName.DETECTION_STATE_INDEX); + executeHistoricalDetector(detector, user, listener); + } else { + String error = "Create index " + CommonName.DETECTION_STATE_INDEX + " with mappings not acknowledged"; + logger.warn(error); + listener.onFailure(new ElasticsearchStatusException(error, RestStatus.INTERNAL_SERVER_ERROR)); + } + }, e -> { + if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { + executeHistoricalDetector(detector, user, listener); + } else { + logger.error("Failed to init anomaly detection state index", e); + listener.onFailure(e); + } + })); + } + } + + private void checkCurrentTaskState(AnomalyDetector detector, User user, ActionListener listener) { + BoolQueryBuilder query = new BoolQueryBuilder(); + query.filter(new TermQueryBuilder(DETECTOR_ID_FIELD, detector.getDetectorId())); + query.filter(new TermsQueryBuilder(STATE_FIELD, ADTaskState.CREATED.name(), ADTaskState.INIT.name(), ADTaskState.RUNNING.name())); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.query(query); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.source(searchSourceBuilder); + searchRequest.indices(CommonName.DETECTION_STATE_INDEX); + + client.search(searchRequest, ActionListener.wrap(r -> { + if (r.getHits().getTotalHits().value > 0) { + listener.onFailure(new ElasticsearchStatusException("Detector is already running", RestStatus.BAD_REQUEST)); + } else { + executeHistoricalDetector(detector, user, listener); + } + }, e -> { + logger.error("Failed to search current running task for detector " + detector.getDetectorId(), e); + listener.onFailure(e); + })); + } + + private void executeHistoricalDetector(AnomalyDetector detector, User user, ActionListener listener) { + UpdateByQueryRequest updateByQueryRequest = new UpdateByQueryRequest(); + updateByQueryRequest.indices(CommonName.DETECTION_STATE_INDEX); + BoolQueryBuilder query = new BoolQueryBuilder(); + query.filter(new TermQueryBuilder(DETECTOR_ID_FIELD, detector.getDetectorId())); + query.filter(new TermQueryBuilder(IS_LATEST_FIELD, true)); + updateByQueryRequest.setQuery(query); + updateByQueryRequest.setRefresh(true); + updateByQueryRequest.setScript(new Script("ctx._source.is_latest = false;")); + + client.execute(UpdateByQueryAction.INSTANCE, updateByQueryRequest, ActionListener.wrap(r -> { + List bulkFailures = r.getBulkFailures(); + if (bulkFailures.isEmpty()) { + createNewADTask(detector, user, listener); + } else { + logger.error("Failed to update old task's state for detector: {}, response: {} ", detector.getDetectorId(), r.toString()); + listener.onFailure(bulkFailures.get(0).getCause()); + } + }, e -> { + logger.error("Failed to reset old tasks as not latest for detector " + detector.getDetectorId(), e); + listener.onFailure(e); + })); + } + + private void createNewADTask(AnomalyDetector detector, User user, ActionListener listener) { + String userName = user == null ? null : user.getName(); + Instant now = Instant.now(); + ADTask adTask = new ADTask.Builder() + .detectorId(detector.getDetectorId()) + .detector(detector) + .isLatest(true) + .taskType(ADTaskType.HISTORICAL.name()) + .executionStartTime(now) + .taskProgress(0.0f) + .initProgress(0.0f) + .state(ADTaskState.CREATED.name()) + .lastUpdateTime(now) + .startedBy(userName) + .build(); + + IndexRequest request = new IndexRequest(CommonName.DETECTION_STATE_INDEX); + try (XContentBuilder builder = XContentFactory.jsonBuilder()) { + request + .source(adTask.toXContent(builder, RestHandlerUtils.XCONTENT_WITH_TYPE)) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + client + .index( + request, + ActionListener + .wrap( + r -> onIndexADTaskResponse( + r, + adTask, + (response, delegatedListener) -> cleanOldAdTaskDocs(response, adTask, delegatedListener), + listener + ), + e -> { + logger.error("Failed to create AD task for detector " + detector.getDetectorId(), e); + listener.onFailure(e); + } + ) + ); + } catch (Exception e) { + logger.error("Failed to create AD task for detector " + detector.getDetectorId(), e); + listener.onFailure(e); + } + } + + private void onIndexADTaskResponse( + IndexResponse response, + ADTask adTask, + BiConsumer> function, + ActionListener listener + ) { + if (response == null || response.getResult() != CREATED) { + String errorMsg = getShardsFailure(response); + listener.onFailure(new ElasticsearchStatusException(errorMsg, response.status())); + return; + } + adTask.setTaskId(response.getId()); + ActionListener delegatedListener = ActionListener.wrap(r -> { listener.onResponse(r); }, e -> { + listener.onFailure(e); + handleADTaskException(adTask, e); + }); + if (function != null) { + function.accept(response, delegatedListener); + } + } + + private void cleanOldAdTaskDocs(IndexResponse response, ADTask adTask, ActionListener listener) { + BoolQueryBuilder query = new BoolQueryBuilder(); + query.filter(new TermQueryBuilder(DETECTOR_ID_FIELD, adTask.getDetectorId())); + query.filter(new TermQueryBuilder(IS_LATEST_FIELD, false)); + SearchRequest searchRequest = new SearchRequest(); + SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); + sourceBuilder + .query(query) + .sort(EXECUTION_START_TIME_FIELD, SortOrder.DESC) + // Search query "from" starts from 0. + .from(maxAdTaskDocsPerDetector - 1) + .trackTotalHits(true) + .size(1); + searchRequest.source(sourceBuilder).indices(CommonName.DETECTION_STATE_INDEX); + String detectorId = adTask.getDetectorId(); + + client.search(searchRequest, ActionListener.wrap(r -> { + Iterator iterator = r.getHits().iterator(); + if (iterator.hasNext()) { + logger + .debug( + "AD tasks count for detector {} is {}, exceeds limit of {}", + detectorId, + r.getHits().getTotalHits().value, + maxAdTaskDocsPerDetector + ); + SearchHit searchHit = r.getHits().getAt(0); + try ( + XContentParser parser = RestHandlerUtils.createXContentParserFromRegistry(xContentRegistry, searchHit.getSourceRef()) + ) { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser); + ADTask task = ADTask.parse(parser, searchHit.getId()); + + DeleteByQueryRequest request = new DeleteByQueryRequest(CommonName.DETECTION_STATE_INDEX); + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder(EXECUTION_START_TIME_FIELD); + rangeQueryBuilder.lt(task.getExecutionStartTime().toEpochMilli()).format("epoch_millis"); + request.setQuery(rangeQueryBuilder); + client.execute(DeleteByQueryAction.INSTANCE, request, ActionListener.wrap(res -> { + logger + .debug( + "Deleted {} old AD tasks started equals or before {} for detector {}", + res.getDeleted(), + adTask.getExecutionStartTime().toEpochMilli(), + detectorId + ); + runBatchResultAction(response, adTask, listener); + }, e -> { + logger.warn("Failed to clean AD tasks for detector " + detectorId, e); + listener.onFailure(e); + })); + } catch (Exception e) { + logger.warn("Failed to parse AD tasks for detector " + detectorId, e); + listener.onFailure(e); + } + } else { + runBatchResultAction(response, adTask, listener); + } + }, e -> { + logger.warn("Failed to search AD tasks for detector " + detectorId, e); + listener.onFailure(e); + })); + } + + private void runBatchResultAction(IndexResponse response, ADTask adTask, ActionListener listener) { + client.execute(ADBatchAnomalyResultAction.INSTANCE, new ADBatchAnomalyResultRequest(adTask), ActionListener.wrap(r -> { + String remoteOrLocal = r.isRunTaskRemotely() ? "remote" : "local"; + logger + .info( + "AD task {} of detector {} dispatched to {} node {}", + adTask.getTaskId(), + adTask.getDetectorId(), + remoteOrLocal, + r.getNodeId() + ); + AnomalyDetectorJobResponse anomalyDetectorJobResponse = new AnomalyDetectorJobResponse( + response.getId(), + response.getVersion(), + response.getSeqNo(), + response.getPrimaryTerm(), + RestStatus.OK + ); + listener.onResponse(anomalyDetectorJobResponse); + }, e -> listener.onFailure(e))); + } + + /** + * Handle exceptions for AD task. Update task state and record error message. + * + * @param adTask AD task + * @param e exception + */ + public void handleADTaskException(ADTask adTask, Exception e) { + // TODO: handle timeout exception + // TODO: handle TaskCancelledException + Map updatedFields = new HashMap<>(); + logger.error("Failed to execute AD batch task, task id: " + adTask.getTaskId() + ", detector id: " + adTask.getDetectorId(), e); + updatedFields.put(STATE_FIELD, ADTaskState.FAILED.name()); + updatedFields.put(ERROR_FIELD, getErrorMessage(e)); + updatedFields.put(EXECUTION_END_TIME_FIELD, Instant.now().toEpochMilli()); + updateADTask(adTask.getTaskId(), updatedFields); + } + + private void updateADTask(String taskId, Map updatedFields) { + updateADTask(taskId, updatedFields, ActionListener.wrap(response -> { + if (response.status() == RestStatus.OK) { + logger.info("Updated AD task successfully: {}", response.status()); + } else { + logger.error("Failed to update AD task {}, status: {}", taskId, response.status()); + } + }, e -> logger.error("Failed to update task: " + taskId, e))); + } + + /** + * Update AD task for specific fields. + * + * @param taskId task id + * @param updatedFields updated fields, key: filed name, value: new value + * @param listener action listener + */ + public void updateADTask(String taskId, Map updatedFields, ActionListener listener) { + UpdateRequest updateRequest = new UpdateRequest(CommonName.DETECTION_STATE_INDEX, taskId); + Map updatedContent = new HashMap<>(); + updatedContent.putAll(updatedFields); + updatedContent.put(LAST_UPDATE_TIME_FIELD, Instant.now().toEpochMilli()); + updateRequest.doc(updatedContent); + updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + client + .update( + updateRequest, + ActionListener.wrap(response -> listener.onResponse(response), exception -> listener.onFailure(exception)) + ); + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultAction.java new file mode 100644 index 00000000..43ce68a4 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultAction.java @@ -0,0 +1,30 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import org.elasticsearch.action.ActionType; + +import com.amazon.opendistroforelasticsearch.ad.constant.CommonValue; + +public class ADBatchAnomalyResultAction extends ActionType { + public static final String NAME = CommonValue.EXTERNAL_ACTION_PREFIX + "detector/ad_task"; + public static final ADBatchAnomalyResultAction INSTANCE = new ADBatchAnomalyResultAction(); + + private ADBatchAnomalyResultAction() { + super(NAME, ADBatchAnomalyResultResponse::new); + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultRequest.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultRequest.java new file mode 100644 index 00000000..ea6c0188 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultRequest.java @@ -0,0 +1,69 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import static org.elasticsearch.action.ValidateActions.addValidationError; + +import java.io.IOException; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; + +public class ADBatchAnomalyResultRequest extends ActionRequest { + private ADTask adTask; + + public ADBatchAnomalyResultRequest(StreamInput in) throws IOException { + super(in); + adTask = new ADTask(in); + } + + public ADBatchAnomalyResultRequest(ADTask adTask) { + super(); + this.adTask = adTask; + } + + public ADTask getAdTask() { + return adTask; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + adTask.writeTo(out); + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (Strings.isEmpty(adTask.getTaskId())) { + validationException = addValidationError("Task id can't be null", validationException); + } + AnomalyDetector detector = adTask.getDetector(); + if (detector == null) { + validationException = addValidationError("Detector can't be null", validationException); + } else if (detector.isRealTimeDetector()) { + validationException = addValidationError("Can't run batch task for realtime detector", validationException); + } + return validationException; + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultResponse.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultResponse.java new file mode 100644 index 00000000..e59a4bbf --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultResponse.java @@ -0,0 +1,53 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import java.io.IOException; + +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +public class ADBatchAnomalyResultResponse extends ActionResponse { + public String nodeId; + public boolean runTaskRemotely; + + public ADBatchAnomalyResultResponse(String nodeId, boolean runTaskRemotely) { + this.nodeId = nodeId; + this.runTaskRemotely = runTaskRemotely; + } + + public ADBatchAnomalyResultResponse(StreamInput in) throws IOException { + super(in); + nodeId = in.readString(); + runTaskRemotely = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(nodeId); + out.writeBoolean(runTaskRemotely); + } + + public String getNodeId() { + return nodeId; + } + + public boolean isRunTaskRemotely() { + return runTaskRemotely; + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportAction.java new file mode 100644 index 00000000..d070ed29 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportAction.java @@ -0,0 +1,47 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; + +import com.amazon.opendistroforelasticsearch.ad.task.ADBatchTaskRunner; + +public class ADBatchAnomalyResultTransportAction extends HandledTransportAction { + + private final TransportService transportService; + private final ADBatchTaskRunner adBatchTaskRunner; + + @Inject + public ADBatchAnomalyResultTransportAction( + TransportService transportService, + ActionFilters actionFilters, + ADBatchTaskRunner adBatchTaskRunner + ) { + super(ADBatchAnomalyResultAction.NAME, transportService, actionFilters, ADBatchAnomalyResultRequest::new); + this.transportService = transportService; + this.adBatchTaskRunner = adBatchTaskRunner; + } + + @Override + protected void doExecute(Task task, ADBatchAnomalyResultRequest request, ActionListener actionListener) { + adBatchTaskRunner.run(request.getAdTask(), transportService, actionListener); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionAction.java new file mode 100644 index 00000000..66da10a3 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionAction.java @@ -0,0 +1,30 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import org.elasticsearch.action.ActionType; + +import com.amazon.opendistroforelasticsearch.ad.constant.CommonValue; + +public class ADBatchTaskRemoteExecutionAction extends ActionType { + public static final String NAME = CommonValue.EXTERNAL_ACTION_PREFIX + "detector/ad_task_remote"; + public static final ADBatchTaskRemoteExecutionAction INSTANCE = new ADBatchTaskRemoteExecutionAction(); + + private ADBatchTaskRemoteExecutionAction() { + super(NAME, ADBatchAnomalyResultResponse::new); + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionTransportAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionTransportAction.java new file mode 100644 index 00000000..95b76baf --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchTaskRemoteExecutionTransportAction.java @@ -0,0 +1,46 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; + +import com.amazon.opendistroforelasticsearch.ad.task.ADBatchTaskRunner; + +public class ADBatchTaskRemoteExecutionTransportAction extends + HandledTransportAction { + + private final ADBatchTaskRunner adBatchTaskRunner; + + @Inject + public ADBatchTaskRemoteExecutionTransportAction( + ActionFilters actionFilters, + TransportService transportService, + ADBatchTaskRunner adBatchTaskRunner + ) { + super(ADBatchTaskRemoteExecutionAction.NAME, transportService, actionFilters, ADBatchAnomalyResultRequest::new); + this.adBatchTaskRunner = adBatchTaskRunner; + } + + @Override + protected void doExecute(Task task, ADBatchAnomalyResultRequest request, ActionListener listener) { + adBatchTaskRunner.startADBatchTask(request.getAdTask(), true, listener); + } +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportAction.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportAction.java index 7787def6..e2224df9 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportAction.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportAction.java @@ -37,7 +37,7 @@ import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; import com.amazon.opendistroforelasticsearch.ad.rest.handler.IndexAnomalyDetectorJobActionHandler; -import com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings; +import com.amazon.opendistroforelasticsearch.ad.task.ADTaskManager; import com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils; import com.amazon.opendistroforelasticsearch.commons.authuser.User; @@ -50,6 +50,7 @@ public class AnomalyDetectorJobTransportAction extends HandledTransportAction filterByEnabled = it); } @Override protected void doExecute(Task task, AnomalyDetectorJobRequest request, ActionListener listener) { String detectorId = request.getDetectorID(); + long seqNo = request.getSeqNo(); + long primaryTerm = request.getPrimaryTerm(); + String rawPath = request.getRawPath(); + TimeValue requestTimeout = REQUEST_TIMEOUT.get(settings); + // By the time request reaches here, the user permissions are validated by Security plugin. User user = getUserContext(client); try (ThreadContext.StoredContext context = client.threadPool().getThreadContext().stashContext()) { @@ -82,7 +90,7 @@ protected void doExecute(Task task, AnomalyDetectorJobRequest request, ActionLis detectorId, filterByEnabled, listener, - () -> adJobExecute(request, listener), + () -> executeDetector(listener, detectorId, seqNo, primaryTerm, rawPath, requestTimeout, user), client, clusterService, xContentRegistry @@ -93,13 +101,15 @@ protected void doExecute(Task task, AnomalyDetectorJobRequest request, ActionLis } } - private void adJobExecute(AnomalyDetectorJobRequest request, ActionListener listener) { - String detectorId = request.getDetectorID(); - long seqNo = request.getSeqNo(); - long primaryTerm = request.getPrimaryTerm(); - String rawPath = request.getRawPath(); - TimeValue requestTimeout = REQUEST_TIMEOUT.get(settings); - + private void executeDetector( + ActionListener listener, + String detectorId, + long seqNo, + long primaryTerm, + String rawPath, + TimeValue requestTimeout, + User user + ) { IndexAnomalyDetectorJobActionHandler handler = new IndexAnomalyDetectorJobActionHandler( client, listener, @@ -110,15 +120,11 @@ private void adJobExecute(AnomalyDetectorJobRequest request, ActionListener listener) { LOG.info("Delete detector info {}", detectorId); - DeleteRequest deleteRequest = new DeleteRequest(DetectorInternalState.DETECTOR_STATE_INDEX, detectorId); + DeleteRequest deleteRequest = new DeleteRequest(CommonName.DETECTION_STATE_INDEX, detectorId); client .delete( deleteRequest, diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java new file mode 100644 index 00000000..1da85234 --- /dev/null +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandler.java @@ -0,0 +1,131 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport.handler; + +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.ANOMALY_RESULT_INDEX_ALIAS; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; + +import java.util.List; +import java.util.function.BooleanSupplier; +import java.util.function.Consumer; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.threadpool.ThreadPool; + +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; +import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyResult; +import com.amazon.opendistroforelasticsearch.ad.util.ClientUtil; +import com.amazon.opendistroforelasticsearch.ad.util.IndexUtils; +import com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils; + +public class AnomalyResultBulkIndexHandler extends AnomalyIndexHandler { + private static final Logger LOG = LogManager.getLogger(AnomalyResultBulkIndexHandler.class); + + private AnomalyDetectionIndices anomalyDetectionIndices; + + public AnomalyResultBulkIndexHandler( + Client client, + Settings settings, + ThreadPool threadPool, + Consumer> createIndex, + BooleanSupplier indexExists, + ClientUtil clientUtil, + IndexUtils indexUtils, + ClusterService clusterService, + AnomalyDetectionIndices anomalyDetectionIndices + ) { + super(client, settings, threadPool, ANOMALY_RESULT_INDEX_ALIAS, createIndex, indexExists, clientUtil, indexUtils, clusterService); + this.anomalyDetectionIndices = anomalyDetectionIndices; + } + + /** + * Bulk index anomaly results. Create anomaly result index first if it doesn't exist. + * + * @param anomalyResults anomaly results + * @param listener action listener + */ + public void bulkIndexAnomalyResult(List anomalyResults, ActionListener listener) { + if (anomalyResults == null || anomalyResults.size() == 0) { + listener.onResponse(null); + return; + } + try { + if (!anomalyDetectionIndices.doesAnomalyResultIndexExist()) { + anomalyDetectionIndices.initAnomalyResultIndexDirectly(ActionListener.wrap(response -> { + if (response.isAcknowledged()) { + bulkSaveDetectorResult(anomalyResults, listener); + } else { + String error = "Creating anomaly result index with mappings call not acknowledged"; + LOG.error(error); + listener.onFailure(new AnomalyDetectionException(error)); + } + }, exception -> { + if (ExceptionsHelper.unwrapCause(exception) instanceof ResourceAlreadyExistsException) { + // It is possible the index has been created while we sending the create request + bulkSaveDetectorResult(anomalyResults, listener); + } else { + listener.onFailure(exception); + } + })); + } else { + bulkSaveDetectorResult(anomalyResults, listener); + } + } catch (AnomalyDetectionException e) { + listener.onFailure(e); + } catch (Exception e) { + String error = "Failed to bulk index anomaly result"; + LOG.error(error, e); + listener.onFailure(new AnomalyDetectionException(error, e)); + } + } + + private void bulkSaveDetectorResult(List anomalyResults, ActionListener listener) { + LOG.debug("Start to bulk save {} anomaly results", anomalyResults.size()); + BulkRequestBuilder bulkRequestBuilder = client.prepareBulk(); + anomalyResults.forEach(anomalyResult -> { + try (XContentBuilder builder = jsonBuilder()) { + IndexRequest indexRequest = new IndexRequest(ANOMALY_RESULT_INDEX_ALIAS) + .source(anomalyResult.toXContent(builder, RestHandlerUtils.XCONTENT_WITH_TYPE)); + bulkRequestBuilder.add(indexRequest); + } catch (Exception e) { + String error = "Failed to prepare request to bulk index anomaly results"; + LOG.error(error, e); + throw new AnomalyDetectionException(error); + } + }); + client.bulk(bulkRequestBuilder.request(), ActionListener.wrap(r -> { + LOG.debug("bulk index AD result successfully, took: {}", r.getTook().duration()); + listener.onResponse(r); + }, e -> { + LOG.error("bulk index ad result failed", e); + listener.onFailure(e); + })); + } + +} diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectionStateHandler.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectionStateHandler.java index ca7cf4ce..a694b422 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectionStateHandler.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectionStateHandler.java @@ -40,6 +40,7 @@ import org.elasticsearch.threadpool.ThreadPool; import com.amazon.opendistroforelasticsearch.ad.NodeStateManager; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.util.ClientUtil; import com.amazon.opendistroforelasticsearch.ad.util.IndexUtils; @@ -97,7 +98,7 @@ public DetectionStateHandler( client, settings, threadPool, - DetectorInternalState.DETECTOR_STATE_INDEX, + CommonName.DETECTION_STATE_INDEX, createIndex, indexExists, clientUtil, diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtil.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtil.java index 7ae1ffaa..25d81d04 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtil.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtil.java @@ -15,9 +15,13 @@ package com.amazon.opendistroforelasticsearch.ad.util; +import org.apache.commons.lang.exception.ExceptionUtils; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper; +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; import com.amazon.opendistroforelasticsearch.ad.common.exception.ResourceNotFoundException; public class ExceptionUtil { @@ -57,4 +61,49 @@ public static boolean isException(Throwable exception, Class 0) { + for (ReplicationResponse.ShardInfo.Failure failure : response.getShardInfo().getFailures()) { + failureReasons.append(failure.reason()); + } + return failureReasons.toString(); + } + return null; + } + + /** + * Count exception in AD failure stats of not. + * + * @param e exception + * @return true if should count in AD failure stats; otherwise return false + */ + public static boolean countInStats(Exception e) { + if (!(e instanceof AnomalyDetectionException) || ((AnomalyDetectionException) e).isCountedInStats()) { + return true; + } + return false; + } + + /** + * Get error message from exception. + * + * @param e exception + * @return readable error message or full stack trace + */ + public static String getErrorMessage(Exception e) { + if (e instanceof IllegalArgumentException || e instanceof AnomalyDetectionException) { + return e.getMessage(); + } else if (e instanceof ElasticsearchException) { + return ((ElasticsearchException) e).getDetailedMessage(); + } else { + return ExceptionUtils.getFullStackTrace(e); + } + } } diff --git a/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtils.java b/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtils.java index 68e515de..33facdce 100644 --- a/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtils.java +++ b/src/main/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtils.java @@ -15,6 +15,9 @@ package com.amazon.opendistroforelasticsearch.ad.util; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.DATE_HISTOGRAM; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.EPOCH_MILLIS_FORMAT; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.FEATURE_AGGS; import static com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector.QUERY_PARAM_PERIOD_END; import static com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector.QUERY_PARAM_PERIOD_START; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; @@ -58,14 +61,20 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.BaseAggregationBuilder; import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.composite.DateHistogramValuesSourceBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.range.DateRangeAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.builder.SearchSourceBuilder; +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.Feature; import com.amazon.opendistroforelasticsearch.ad.model.FeatureData; +import com.amazon.opendistroforelasticsearch.ad.model.IntervalTimeConfiguration; import com.amazon.opendistroforelasticsearch.ad.rest.handler.AnomalyDetectorFunction; import com.amazon.opendistroforelasticsearch.ad.transport.GetAnomalyDetectorResponse; import com.amazon.opendistroforelasticsearch.commons.ConfigConstants; @@ -594,7 +603,69 @@ public static Optional getLatestDataTime(SearchResponse searchResponse) { .ofNullable(searchResponse) .map(SearchResponse::getAggregations) .map(aggs -> aggs.asMap()) - .map(map -> (Max) map.get(CommonName.AGG_NAME_MAX)) + .map(map -> (Max) map.get(CommonName.AGG_NAME_MAX_TIME)) .map(agg -> (long) agg.getValue()); } + + /** + * Generate batch query request for feature aggregation on given date range. + * + * @param detector anomaly detector + * @param startTime start time + * @param endTime end time + * @param xContentRegistry content registry + * @return search source builder + * @throws IOException throw IO exception if fail to parse feature aggregation + * @throws AnomalyDetectionException throw AD exception if no enabled feature + */ + public static SearchSourceBuilder batchFeatureQuery( + AnomalyDetector detector, + long startTime, + long endTime, + NamedXContentRegistry xContentRegistry + ) throws IOException { + RangeQueryBuilder rangeQuery = new RangeQueryBuilder(detector.getTimeField()) + .from(startTime) + .to(endTime) + .format(EPOCH_MILLIS_FORMAT) + .includeLower(true) + .includeUpper(false); + + BoolQueryBuilder internalFilterQuery = QueryBuilders.boolQuery().must(rangeQuery).must(detector.getFilterQuery()); + + long intervalSeconds = ((IntervalTimeConfiguration) detector.getDetectionInterval()).toDuration().getSeconds(); + + List> sources = new ArrayList<>(); + sources + .add( + new DateHistogramValuesSourceBuilder(DATE_HISTOGRAM) + .field(detector.getTimeField()) + .fixedInterval(DateHistogramInterval.seconds((int) intervalSeconds)) + ); + + CompositeAggregationBuilder aggregationBuilder = new CompositeAggregationBuilder(FEATURE_AGGS, sources).size(1000); + + if (detector.getEnabledFeatureIds().size() == 0) { + throw new AnomalyDetectionException("No enabled feature configured").countedInStats(false); + } + + for (Feature feature : detector.getFeatureAttributes()) { + if (feature.getEnabled()) { + AggregatorFactories.Builder internalAgg = parseAggregators( + feature.getAggregation().toString(), + xContentRegistry, + feature.getId() + ); + aggregationBuilder.subAggregation(internalAgg.getAggregatorFactories().iterator().next()); + } + } + + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.aggregation(aggregationBuilder); + searchSourceBuilder.query(internalFilterQuery); + searchSourceBuilder.size(0); + + return searchSourceBuilder; + } + } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADIntegTestCase.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADIntegTestCase.java index e54c05c1..a632737b 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADIntegTestCase.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADIntegTestCase.java @@ -15,7 +15,9 @@ package com.amazon.opendistroforelasticsearch.ad; +import static com.amazon.opendistroforelasticsearch.ad.AbstractADTest.LOG; import static com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils.XCONTENT_WITH_TYPE; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import java.io.IOException; import java.util.Collection; @@ -23,25 +25,45 @@ import java.util.List; import java.util.Map; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.test.ESIntegTestCase; import org.junit.Before; +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils; public abstract class ADIntegTestCase extends ESIntegTestCase { private long timeout = 5_000; + protected String timeField = "timestamp"; + protected String categoryField = "type"; + protected String valueField = "value"; @Override protected Collection> nodePlugins() { @@ -65,16 +87,35 @@ public void createDetectors(List detectors, boolean createIndex } for (AnomalyDetector detector : detectors) { - indexDoc(AnomalyDetector.ANOMALY_DETECTORS_INDEX, detector.toXContent(XContentFactory.jsonBuilder(), XCONTENT_WITH_TYPE)); + indexDoc(AnomalyDetector.ANOMALY_DETECTORS_INDEX, detector.toXContent(jsonBuilder(), XCONTENT_WITH_TYPE)); } } + public String createDetector(AnomalyDetector detector) throws IOException { + return indexDoc(AnomalyDetector.ANOMALY_DETECTORS_INDEX, detector.toXContent(jsonBuilder(), XCONTENT_WITH_TYPE)); + } + + public String createADTask(ADTask adTask) throws IOException { + return indexDoc(CommonName.DETECTION_STATE_INDEX, adTask.toXContent(jsonBuilder(), XCONTENT_WITH_TYPE)); + } + public void createDetectorIndex() throws IOException { createIndex(AnomalyDetector.ANOMALY_DETECTORS_INDEX, AnomalyDetectionIndices.getAnomalyDetectorMappings()); } - public String createDetectors(AnomalyDetector detector) throws IOException { - return indexDoc(AnomalyDetector.ANOMALY_DETECTORS_INDEX, detector.toXContent(XContentFactory.jsonBuilder(), XCONTENT_WITH_TYPE)); + public void createDetectionStateIndex() throws IOException { + createIndex(CommonName.DETECTION_STATE_INDEX, AnomalyDetectionIndices.getDetectionStateMappings()); + } + + public void createTestDataIndex(String indexName) { + String mappings = "{\"properties\":{\"" + + timeField + + "\":{\"type\":\"date\",\"format\":\"strict_date_time||epoch_millis\"}," + + "\"value\":{\"type\":\"double\"}, \"" + + categoryField + + "\":{\"type\":\"keyword\"}," + + "\"is_error\":{\"type\":\"boolean\"}, \"message\":{\"type\":\"text\"}}}"; + createIndex(indexName, mappings); } public void createIndex(String indexName, String mappings) { @@ -82,6 +123,15 @@ public void createIndex(String indexName, String mappings) { assertEquals(true, createIndexResponse.isAcknowledged()); } + public AcknowledgedResponse deleteDetectorIndex() { + return deleteIndex(AnomalyDetector.ANOMALY_DETECTORS_INDEX); + } + + public AcknowledgedResponse deleteIndex(String indexName) { + DeleteIndexRequest deleteIndexRequest = new DeleteIndexRequest(indexName); + return admin().indices().delete(deleteIndexRequest).actionGet(timeout); + } + public String indexDoc(String indexName, XContentBuilder source) { IndexRequest indexRequest = new IndexRequest(indexName).setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).source(source); IndexResponse indexResponse = client().index(indexRequest).actionGet(timeout); @@ -96,8 +146,51 @@ public String indexDoc(String indexName, Map source) { return indexResponse.getId(); } + public BulkResponse bulkIndexObjects(String indexName, List objects) { + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); + objects.forEach(obj -> { + try (XContentBuilder builder = jsonBuilder()) { + IndexRequest indexRequest = new IndexRequest(indexName) + .source(obj.toXContent(builder, RestHandlerUtils.XCONTENT_WITH_TYPE)); + bulkRequestBuilder.add(indexRequest); + } catch (Exception e) { + String error = "Failed to prepare request to bulk index docs"; + LOG.error(error, e); + throw new AnomalyDetectionException(error); + } + }); + return client().bulk(bulkRequestBuilder.request()).actionGet(timeout); + } + + public BulkResponse bulkIndexDocs(String indexName, List> docs, long timeout) { + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); + docs.forEach(doc -> bulkRequestBuilder.add(new IndexRequest(indexName).source(doc))); + return client().bulk(bulkRequestBuilder.request().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)).actionGet(timeout); + } + public GetResponse getDoc(String indexName, String id) { GetRequest getRequest = new GetRequest(indexName).id(id); return client().get(getRequest).actionGet(timeout); } + + public long countDocs(String indexName) { + SearchRequest request = new SearchRequest(); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchSourceBuilder.query(new MatchAllQueryBuilder()).size(0); + request.indices(indexName).source(searchSourceBuilder); + SearchResponse searchResponse = client().search(request).actionGet(timeout); + return searchResponse.getHits().getTotalHits().value; + } + + public ClusterUpdateSettingsResponse updateTransientSettings(Map settings) { + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.transientSettings(settings); + return clusterAdmin().updateSettings(updateSettingsRequest).actionGet(timeout); + } + + public ImmutableOpenMap getDataNodes() { + DiscoveryNodes nodes = clusterService().state().getNodes(); + return nodes.getDataNodes(); + } + } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADUnitTestCase.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADUnitTestCase.java new file mode 100644 index 00000000..b2a891ab --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/ADUnitTestCase.java @@ -0,0 +1,58 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad; + +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.MockitoAnnotations; + +import com.google.common.collect.Sets; + +public class ADUnitTestCase extends ESTestCase { + + @Captor + protected ArgumentCaptor exceptionCaptor; + + @Override + public void setUp() throws Exception { + super.setUp(); + MockitoAnnotations.initMocks(this); + } + + /** + * Create cluster setting. + * + * @param settings cluster settings + * @param setting add setting if the code to be tested contains setting update consumer + * @return instance of ClusterSettings + */ + public ClusterSettings clusterSetting(Settings settings, Setting... setting) { + final Set> settingsSet = Stream + .concat(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.stream(), Sets.newHashSet(setting).stream()) + .collect(Collectors.toSet()); + ClusterSettings clusterSettings = new ClusterSettings(settings, settingsSet); + return clusterSettings; + } + +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorJobRunnerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorJobRunnerTests.java index ea8948c7..2ff7d81b 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorJobRunnerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorJobRunnerTests.java @@ -162,7 +162,7 @@ public void setup() throws Exception { client, settings, threadPool, - ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectorStateIndex), + ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectionStateIndex), anomalyDetectionIndices::doesDetectorStateIndexExist, this.clientUtil, indexUtils, diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunnerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunnerTests.java index bc40ab6d..8bb18a16 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunnerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/AnomalyDetectorProfileRunnerTests.java @@ -145,7 +145,7 @@ private void setUpClientGet( } } else { if (errorResultStatus == ErrorResultStatus.INDEX_NOT_EXIT) { - listener.onFailure(new IndexNotFoundException(DetectorInternalState.DETECTOR_STATE_INDEX)); + listener.onFailure(new IndexNotFoundException(CommonName.DETECTION_STATE_INDEX)); return null; } DetectorInternalState.Builder result = new DetectorInternalState.Builder().lastUpdateTime(Instant.now()); @@ -164,9 +164,7 @@ private void setUpClientGet( break; } listener - .onResponse( - TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), DetectorInternalState.DETECTOR_STATE_INDEX) - ); + .onResponse(TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), CommonName.DETECTION_STATE_INDEX)); } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunnerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunnerTests.java index 30b0f8b1..bf479f93 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunnerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/EntityProfileRunnerTests.java @@ -144,7 +144,7 @@ private void setUpSearch() { String indexName = request.indices()[0]; ActionListener listener = (ActionListener) args[1]; if (indexName.equals(CommonName.ANOMALY_RESULT_INDEX_ALIAS)) { - InternalMax maxAgg = new InternalMax(CommonName.AGG_NAME_MAX, latestSampleTimestamp, DocValueFormat.RAW, emptyMap()); + InternalMax maxAgg = new InternalMax(CommonName.AGG_NAME_MAX_TIME, latestSampleTimestamp, DocValueFormat.RAW, emptyMap()); InternalAggregations internalAggregations = InternalAggregations.from(Collections.singletonList(maxAgg)); SearchHits hits = new SearchHits(new SearchHit[] {}, null, Float.NaN); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/HistoricalDetectorIntegTestCase.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/HistoricalDetectorIntegTestCase.java new file mode 100644 index 00000000..e3032dd0 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/HistoricalDetectorIntegTestCase.java @@ -0,0 +1,173 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad; + +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.DETECTOR_ID_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.EXECUTION_START_TIME_FIELD; +import static com.amazon.opendistroforelasticsearch.ad.model.ADTask.IS_LATEST_FIELD; + +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.index.query.BoolQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.sort.SortOrder; + +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskState; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskType; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; +import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; +import com.amazon.opendistroforelasticsearch.ad.model.Feature; +import com.google.common.collect.ImmutableMap; + +public abstract class HistoricalDetectorIntegTestCase extends ADIntegTestCase { + + protected String testIndex = "test_historical_data"; + protected int detectionIntervalInMinutes = 1; + protected int DEFAULT_TEST_DATA_DOCS = 3000; + + public void ingestTestData(String testIndex, Instant startTime, int detectionIntervalInMinutes, String type) { + ingestTestData(testIndex, startTime, detectionIntervalInMinutes, type, DEFAULT_TEST_DATA_DOCS); + } + + public void ingestTestData(String testIndex, Instant startTime, int detectionIntervalInMinutes, String type, int totalDocs) { + createTestDataIndex(testIndex); + List> docs = new ArrayList<>(); + Instant currentInterval = Instant.from(startTime); + + for (int i = 0; i < totalDocs; i++) { + currentInterval = currentInterval.plus(detectionIntervalInMinutes, ChronoUnit.MINUTES); + double value = i % 500 == 0 ? randomDoubleBetween(1000, 2000, true) : randomDoubleBetween(10, 100, true); + docs + .add( + ImmutableMap + .of( + timeField, + currentInterval.toEpochMilli(), + "value", + value, + "type", + type, + "is_error", + randomBoolean(), + "message", + randomAlphaOfLength(5) + ) + ); + } + BulkResponse bulkResponse = bulkIndexDocs(testIndex, docs, 30_000); + assertEquals(RestStatus.OK, bulkResponse.status()); + assertFalse(bulkResponse.hasFailures()); + long count = countDocs(testIndex); + assertEquals(totalDocs, count); + } + + public Feature maxValueFeature() throws IOException { + AggregationBuilder aggregationBuilder = TestHelpers.parseAggregation("{\"test\":{\"max\":{\"field\":\"" + valueField + "\"}}}"); + return new Feature(randomAlphaOfLength(5), randomAlphaOfLength(10), true, aggregationBuilder); + } + + public AnomalyDetector randomDetector(DetectionDateRange dateRange, List features) throws IOException { + return TestHelpers.randomDetector(dateRange, features, testIndex, detectionIntervalInMinutes, timeField); + } + + public ADTask randomCreatedADTask(String taskId, AnomalyDetector detector) { + String detectorId = detector == null ? null : detector.getDetectorId(); + return randomCreatedADTask(taskId, detector, detectorId); + } + + public ADTask randomCreatedADTask(String taskId, AnomalyDetector detector, String detectorId) { + return randomADTask(taskId, detector, detectorId, ADTaskState.CREATED); + } + + public ADTask randomADTask(String taskId, AnomalyDetector detector, String detectorId, ADTaskState state) { + ADTask.Builder builder = ADTask + .builder() + .taskId(taskId) + .taskType(ADTaskType.HISTORICAL.name()) + .detectorId(detectorId) + .detector(detector) + .state(state.name()) + .taskProgress(0.0f) + .initProgress(0.0f) + .isLatest(true) + .startedBy(randomAlphaOfLength(5)) + .executionStartTime(Instant.now().minus(randomLongBetween(10, 100), ChronoUnit.MINUTES)); + if (ADTaskState.FINISHED == state) { + setPropertyForNotRunningTask(builder); + } else if (ADTaskState.FAILED == state) { + setPropertyForNotRunningTask(builder); + builder.error(randomAlphaOfLength(5)); + } else if (ADTaskState.STOPPED == state) { + setPropertyForNotRunningTask(builder); + builder.error(randomAlphaOfLength(5)); + builder.stoppedBy(randomAlphaOfLength(5)); + } + return builder.build(); + } + + private ADTask.Builder setPropertyForNotRunningTask(ADTask.Builder builder) { + builder.executionEndTime(Instant.now().minus(randomLongBetween(1, 5), ChronoUnit.MINUTES)); + builder.isLatest(false); + return builder; + } + + public List searchADTasks(String detectorId, Boolean isLatest, int size) throws IOException { + BoolQueryBuilder query = new BoolQueryBuilder(); + query.filter(new TermQueryBuilder(DETECTOR_ID_FIELD, detectorId)); + if (isLatest != null) { + query.filter(new TermQueryBuilder(IS_LATEST_FIELD, false)); + } + SearchRequest searchRequest = new SearchRequest(); + SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); + sourceBuilder.query(query).sort(EXECUTION_START_TIME_FIELD, SortOrder.DESC).trackTotalHits(true).size(size); + searchRequest.source(sourceBuilder).indices(CommonName.DETECTION_STATE_INDEX); + SearchResponse searchResponse = client().search(searchRequest).actionGet(); + Iterator iterator = searchResponse.getHits().iterator(); + + List adTasks = new ArrayList<>(); + while (iterator.hasNext()) { + SearchHit next = iterator.next(); + ADTask task = ADTask.parse(TestHelpers.parser(next.getSourceAsString()), next.getId()); + adTasks.add(task); + } + return adTasks; + } + + public ADTask toADTask(GetResponse doc) throws IOException { + return ADTask.parse(TestHelpers.parser(doc.getSourceAsString())); + } + + public AnomalyDetectorJob toADJob(GetResponse doc) throws IOException { + return AnomalyDetectorJob.parse(TestHelpers.parser(doc.getSourceAsString())); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/MultiEntityProfileRunnerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/MultiEntityProfileRunnerTests.java index f71161fe..a9e87d28 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/MultiEntityProfileRunnerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/MultiEntityProfileRunnerTests.java @@ -47,6 +47,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.junit.Before; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyResult; @@ -113,11 +114,9 @@ public void setUp() throws Exception { if (indexName.equals(ANOMALY_DETECTORS_INDEX)) { listener .onResponse(TestHelpers.createGetResponse(detector, detector.getDetectorId(), AnomalyDetector.ANOMALY_DETECTORS_INDEX)); - } else if (indexName.equals(DetectorInternalState.DETECTOR_STATE_INDEX)) { + } else if (indexName.equals(CommonName.DETECTION_STATE_INDEX)) { listener - .onResponse( - TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), DetectorInternalState.DETECTOR_STATE_INDEX) - ); + .onResponse(TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), CommonName.DETECTION_STATE_INDEX)); } else if (indexName.equals(ANOMALY_DETECTOR_JOB_INDEX)) { listener .onResponse( diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/TestHelpers.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/TestHelpers.java index 82144629..472dfb98 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/TestHelpers.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/TestHelpers.java @@ -122,6 +122,7 @@ import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorExecutionInput; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorType; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyResult; import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; @@ -255,6 +256,7 @@ public static AnomalyDetector randomAnomalyDetector( uiMetadata, lastUpdateTime, detectorType, + ESRestTestCase.randomLongBetween(1, 1000), dateRange, withUser ); @@ -266,6 +268,7 @@ public static AnomalyDetector randomAnomalyDetector( Map uiMetadata, Instant lastUpdateTime, String detectorType, + long detectionIntervalInMinutes, DetectionDateRange dateRange, boolean withUser ) throws IOException { @@ -279,7 +282,7 @@ public static AnomalyDetector randomAnomalyDetector( indices, features, randomQuery(), - randomIntervalTimeConfiguration(), + new IntervalTimeConfiguration(detectionIntervalInMinutes, ChronoUnit.MINUTES), randomIntervalTimeConfiguration(), randomIntBetween(1, 2000), uiMetadata, @@ -292,6 +295,38 @@ public static AnomalyDetector randomAnomalyDetector( ); } + public static AnomalyDetector randomDetector( + DetectionDateRange dateRange, + List features, + String indexName, + int detectionIntervalInMinutes, + String timeField + ) throws IOException { + String detectorType = dateRange == null + ? AnomalyDetectorType.REALTIME_SINGLE_ENTITY.name() + : AnomalyDetectorType.HISTORICAL_SINGLE_ENTITY.name(); + return new AnomalyDetector( + randomAlphaOfLength(10), + randomLong(), + randomAlphaOfLength(20), + randomAlphaOfLength(30), + timeField, + ImmutableList.of(indexName), + features, + randomQuery("{\"bool\":{\"filter\":[{\"exists\":{\"field\":\"value\"}}]}}"), + new IntervalTimeConfiguration(detectionIntervalInMinutes, ChronoUnit.MINUTES), + new IntervalTimeConfiguration(ESRestTestCase.randomLongBetween(1, 5), ChronoUnit.MINUTES), + 8, + null, + randomInt(), + Instant.now(), + null, + null, + detectorType, + dateRange + ); + } + public static DetectionDateRange randomDetectionDateRange() { return new DetectionDateRange( Instant.now().truncatedTo(ChronoUnit.SECONDS).minus(10, ChronoUnit.DAYS), diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledExceptionTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledExceptionTests.java new file mode 100644 index 00000000..0990e820 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/common/exception/ADTaskCancelledExceptionTests.java @@ -0,0 +1,29 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.common.exception; + +import org.elasticsearch.test.ESTestCase; + +public class ADTaskCancelledExceptionTests extends ESTestCase { + + public void testConstructor() { + String message = randomAlphaOfLength(5); + String user = randomAlphaOfLength(5); + ADTaskCancelledException exception = new ADTaskCancelledException(message, user); + assertEquals(message, exception.getMessage()); + assertEquals(user, exception.getCancelledBy()); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManagerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManagerTests.java index 6f08e260..7628be74 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManagerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/FeatureManagerTests.java @@ -15,6 +15,7 @@ package com.amazon.opendistroforelasticsearch.ad.feature; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_IMPUTATION_NEIGHBOR_DISTANCE; import static java.util.Arrays.asList; import static java.util.Optional.empty; import static java.util.Optional.ofNullable; @@ -42,8 +43,10 @@ import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.AbstractMap.SimpleEntry; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; +import java.util.Deque; import java.util.List; import java.util.Map.Entry; import java.util.Optional; @@ -1013,4 +1016,103 @@ public void getCurrentFeatures_setsShingleSizeFromDetectorConfig(int shingleSize assertEquals(listenerResponse.getProcessedFeatures().get().length, shingleSize); assertEquals(featureManager.getShingleSize(detector.getDetectorId()), shingleSize); } + + @Test + public void testGetShingledFeatureForHistoricalDetectorFromEmptyShingleWithoutMissingData() { + long millisecondsPerMinute = 60000; + int shingleSize = 8; + when(detector.getShingleSize()).thenReturn(shingleSize); + + Deque>> shingle = new ArrayDeque<>(); + + long endTime = Instant.now().toEpochMilli(); + int i = 0; + for (; i < shingleSize - MAX_IMPUTATION_NEIGHBOR_DISTANCE; i++) { + double[] testData = new double[] { i }; + Optional dataPoint = Optional.of(testData); + SinglePointFeatures feature = featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, dataPoint, endTime); + endTime += millisecondsPerMinute; + + assertTrue(Arrays.equals(testData, feature.getUnprocessedFeatures().get())); + assertFalse(feature.getProcessedFeatures().isPresent()); + } + + double[] testData = new double[] { i++ }; + Optional dataPoint = Optional.of(testData); + SinglePointFeatures feature = featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, dataPoint, endTime); + assertTrue(feature.getProcessedFeatures().isPresent()); + assertTrue(Arrays.equals(new double[] { 0.0, 0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0 }, feature.getProcessedFeatures().get())); + + endTime += millisecondsPerMinute; + testData = new double[] { i++ }; + dataPoint = Optional.of(testData); + feature = featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, dataPoint, endTime); + assertTrue(feature.getProcessedFeatures().isPresent()); + assertTrue(Arrays.equals(new double[] { 0.0, 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0 }, feature.getProcessedFeatures().get())); + + for (; i < 2 * shingleSize; i++) { + endTime += millisecondsPerMinute; + SinglePointFeatures singlePointFeatures = featureManager + .getShingledFeatureForHistoricalDetector(detector, shingle, Optional.of(new double[] { i }), endTime); + assertTrue(singlePointFeatures.getProcessedFeatures().isPresent()); + assertTrue( + Arrays + .equals( + new double[] { i - 7, i - 6, i - 5, i - 4, i - 3, i - 2, i - 1, i }, + singlePointFeatures.getProcessedFeatures().get() + ) + ); + } + } + + @Test + public void testGetShingledFeatureForHistoricalDetectorWithTooManyMissingData() { + long millisecondsPerMinute = 60000; + int shingleSize = 8; + when(detector.getShingleSize()).thenReturn(shingleSize); + + Deque>> shingle = new ArrayDeque<>(); + + long endTime = Instant.now().toEpochMilli(); + int i = 0; + for (; i < shingleSize; i++) { + featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, Optional.of(new double[] { i }), endTime); + endTime += millisecondsPerMinute; + } + + for (int j = 0; j < MAX_IMPUTATION_NEIGHBOR_DISTANCE + 1; j++) { + SinglePointFeatures feature = featureManager + .getShingledFeatureForHistoricalDetector(detector, shingle, Optional.empty(), endTime); + endTime += millisecondsPerMinute; + assertFalse(feature.getProcessedFeatures().isPresent()); + } + SinglePointFeatures feature = featureManager + .getShingledFeatureForHistoricalDetector(detector, shingle, Optional.of(new double[] { i }), endTime); + assertFalse(feature.getProcessedFeatures().isPresent()); + } + + @Test + public void testGetShingledFeatureForHistoricalDetectorWithOneMissingData() { + long millisecondsPerMinute = 60000; + int shingleSize = 8; + when(detector.getShingleSize()).thenReturn(shingleSize); + + Deque>> shingle = new ArrayDeque<>(); + + long endTime = Instant.now().toEpochMilli(); + int i = 0; + for (; i < shingleSize; i++) { + featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, Optional.of(new double[] { i }), endTime); + endTime += millisecondsPerMinute; + } + + SinglePointFeatures feature1 = featureManager.getShingledFeatureForHistoricalDetector(detector, shingle, Optional.empty(), endTime); + assertFalse(feature1.getProcessedFeatures().isPresent()); + + endTime += millisecondsPerMinute; + SinglePointFeatures feature2 = featureManager + .getShingledFeatureForHistoricalDetector(detector, shingle, Optional.of(new double[] { i }), endTime); + assertTrue(feature2.getProcessedFeatures().isPresent()); + assertTrue(Arrays.equals(new double[] { 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 8.0 }, feature2.getProcessedFeatures().get())); + } } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDaoTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDaoTests.java index 2374e8ee..eea2d25e 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDaoTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/feature/SearchFeatureDaoTests.java @@ -237,7 +237,7 @@ public void setup() throws Exception { aggsMap = new HashMap<>(); // aggsList = new ArrayList<>(); - when(max.getName()).thenReturn(CommonName.AGG_NAME_MAX); + when(max.getName()).thenReturn(CommonName.AGG_NAME_MAX_TIME); List list = new ArrayList<>(); list.add(max); Aggregations aggregations = new Aggregations(list); @@ -276,12 +276,12 @@ public void setup() throws Exception { public void test_getLatestDataTime_returnExpectedTime_givenData() { // pre-conditions SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .size(0); searchRequest.source(searchSourceBuilder); long epochTime = 100L; - aggsMap.put(CommonName.AGG_NAME_MAX, max); + aggsMap.put(CommonName.AGG_NAME_MAX_TIME, max); when(max.getValue()).thenReturn((double) epochTime); // test @@ -295,7 +295,7 @@ public void test_getLatestDataTime_returnExpectedTime_givenData() { public void test_getLatestDataTime_returnEmpty_givenNoData() { // pre-conditions SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .size(0); searchRequest.source(searchSourceBuilder); @@ -312,11 +312,11 @@ public void test_getLatestDataTime_returnEmpty_givenNoData() { @SuppressWarnings("unchecked") public void getLatestDataTime_returnExpectedToListener() { SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder() - .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX).field(detector.getTimeField())) + .aggregation(AggregationBuilders.max(CommonName.AGG_NAME_MAX_TIME).field(detector.getTimeField())) .size(0); searchRequest.source(searchSourceBuilder); long epochTime = 100L; - aggsMap.put(CommonName.AGG_NAME_MAX, max); + aggsMap.put(CommonName.AGG_NAME_MAX_TIME, max); when(max.getValue()).thenReturn((double) epochTime); doAnswer(invocation -> { ActionListener listener = invocation.getArgument(1); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/indices/InitAnomalyDetectionIndicesTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/indices/InitAnomalyDetectionIndicesTests.java index ce6fa42f..8324286b 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/indices/InitAnomalyDetectionIndicesTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/indices/InitAnomalyDetectionIndicesTests.java @@ -49,7 +49,6 @@ import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; -import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings; import com.amazon.opendistroforelasticsearch.ad.util.DiscoveryNodeFilterer; @@ -123,7 +122,7 @@ private void fixedPrimaryShardsIndexCreationTemplate(String index) throws IOExce if (index.equals(AnomalyDetector.ANOMALY_DETECTORS_INDEX)) { adIndices.initAnomalyDetectorIndexIfAbsent(listener); } else { - adIndices.initDetectorStateIndex(listener); + adIndices.initDetectionStateIndex(listener); } ArgumentCaptor captor = ArgumentCaptor.forClass(CreateIndexResponse.class); @@ -177,8 +176,8 @@ private void adaptivePrimaryShardsIndexCreationTemplate(String index) throws IOE ActionListener listener = mock(ActionListener.class); if (index.equals(AnomalyDetector.ANOMALY_DETECTORS_INDEX)) { adIndices.initAnomalyDetectorIndexIfAbsent(listener); - } else if (index.equals(DetectorInternalState.DETECTOR_STATE_INDEX)) { - adIndices.initDetectorStateIndex(listener); + } else if (index.equals(CommonName.DETECTION_STATE_INDEX)) { + adIndices.initDetectionStateIndex(listener); } else if (index.equals(CommonName.CHECKPOINT_INDEX_NAME)) { adIndices.initCheckpointIndex(listener); } else if (index.equals(AnomalyDetectorJob.ANOMALY_DETECTOR_JOB_INDEX)) { @@ -206,7 +205,7 @@ public void testCreateDetector() throws IOException { } public void testCreateState() throws IOException { - fixedPrimaryShardsIndexCreationTemplate(DetectorInternalState.DETECTOR_STATE_INDEX); + fixedPrimaryShardsIndexCreationTemplate(CommonName.DETECTION_STATE_INDEX); } public void testCreateJob() throws IOException { diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetectorTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetectorTests.java index eb9359ea..1ffffd98 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetectorTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/AnomalyDetectorTests.java @@ -474,4 +474,27 @@ public void testGetShingleSizeReturnsDefaultValue() throws IOException { ); assertEquals((int) anomalyDetector.getShingleSize(), AnomalyDetectorSettings.DEFAULT_SHINGLE_SIZE); } + + public void testNullFeatureAttributes() throws IOException { + AnomalyDetector anomalyDetector = new AnomalyDetector( + randomAlphaOfLength(5), + randomLong(), + randomAlphaOfLength(5), + randomAlphaOfLength(5), + randomAlphaOfLength(5), + ImmutableList.of(randomAlphaOfLength(5)), + null, + TestHelpers.randomQuery(), + TestHelpers.randomIntervalTimeConfiguration(), + TestHelpers.randomIntervalTimeConfiguration(), + null, + null, + 1, + Instant.now(), + null, + TestHelpers.randomUser() + ); + assertNotNull(anomalyDetector.getFeatureAttributes()); + assertEquals(0, anomalyDetector.getFeatureAttributes().size()); + } } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRangeTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRangeTests.java index 1cc6d501..f6d803f1 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRangeTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/model/DetectionDateRangeTests.java @@ -17,11 +17,14 @@ import java.io.IOException; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Collection; +import java.util.Locale; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.InternalSettingsPlugin; @@ -57,6 +60,14 @@ public void testParseDetectionDateRangeWithNullEndTime() { assertEquals("Detection data range's end time must not be null", exception.getMessage()); } + public void testInvalidDateRange() { + IllegalArgumentException exception = expectThrows( + IllegalArgumentException.class, + () -> new DetectionDateRange(Instant.now(), Instant.now().minus(10, ChronoUnit.MINUTES)) + ); + assertEquals("Detection data range's end time must be after start time", exception.getMessage()); + } + public void testSerializeDetectoinDateRange() throws IOException { DetectionDateRange dateRange = TestHelpers.randomDetectionDateRange(); BytesStreamOutput output = new BytesStreamOutput(); @@ -65,4 +76,14 @@ public void testSerializeDetectoinDateRange() throws IOException { DetectionDateRange parsedDateRange = new DetectionDateRange(input); assertTrue(parsedDateRange.equals(dateRange)); } + + public void testParseDetectionDateRange() throws IOException { + DetectionDateRange dateRange = TestHelpers.randomDetectionDateRange(); + String dateRangeString = TestHelpers.xContentBuilderToString(dateRange.toXContent(TestHelpers.builder(), ToXContent.EMPTY_PARAMS)); + dateRangeString = dateRangeString + .replaceFirst("\\{", String.format(Locale.ROOT, "{\"%s\":\"%s\",", randomAlphaOfLength(5), randomAlphaOfLength(5))); + DetectionDateRange parsedDateRange = DetectionDateRange.parse(TestHelpers.parser(dateRangeString)); + assertEquals("Parsing detection range doesn't work", dateRange, parsedDateRange); + } + } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/plugin/MockReindexPlugin.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/plugin/MockReindexPlugin.java new file mode 100644 index 00000000..03086b89 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/plugin/MockReindexPlugin.java @@ -0,0 +1,182 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.plugin; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.reindex.BulkByScrollResponse; +import org.elasticsearch.index.reindex.BulkByScrollTask; +import org.elasticsearch.index.reindex.DeleteByQueryAction; +import org.elasticsearch.index.reindex.DeleteByQueryRequest; +import org.elasticsearch.index.reindex.UpdateByQueryAction; +import org.elasticsearch.index.reindex.UpdateByQueryRequest; +import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; + +import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; +import com.google.common.collect.ImmutableList; + +public class MockReindexPlugin extends Plugin implements ActionPlugin { + + @Override + public List> getActions() { + return Arrays + .asList( + new ActionHandler<>(UpdateByQueryAction.INSTANCE, MockTransportUpdateByQueryAction.class), + new ActionHandler<>(DeleteByQueryAction.INSTANCE, MockTransportDeleteByQueryAction.class) + ); + } + + public static class MockTransportUpdateByQueryAction extends HandledTransportAction { + + @Inject + public MockTransportUpdateByQueryAction(ActionFilters actionFilters, TransportService transportService) { + super(UpdateByQueryAction.NAME, transportService, actionFilters, UpdateByQueryRequest::new); + } + + @Override + protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener listener) { + BulkByScrollResponse response = null; + try { + XContentParser parser = TestHelpers + .parser( + "{\"slice_id\":1,\"total\":2,\"updated\":3,\"created\":0,\"deleted\":0,\"batches\":6," + + "\"version_conflicts\":0,\"noops\":0,\"retries\":{\"bulk\":0,\"search\":10}," + + "\"throttled_millis\":0,\"requests_per_second\":13.0,\"canceled\":\"reasonCancelled\"," + + "\"throttled_until_millis\":14}" + ); + parser.nextToken(); + response = new BulkByScrollResponse( + TimeValue.timeValueMillis(10), + BulkByScrollTask.Status.innerFromXContent(parser), + ImmutableList.of(), + ImmutableList.of(), + false + ); + } catch (IOException exception) { + exception.printStackTrace(); + } + listener.onResponse(response); + } + } + + public static class MockTransportDeleteByQueryAction extends HandledTransportAction { + + private Client client; + + @Inject + public MockTransportDeleteByQueryAction(ActionFilters actionFilters, TransportService transportService, Client client) { + super(DeleteByQueryAction.NAME, transportService, actionFilters, DeleteByQueryRequest::new); + this.client = client; + } + + private class MultiResponsesActionListener implements ActionListener { + private final ActionListener delegate; + private final AtomicInteger collectedResponseCount; + private final AtomicLong maxResponseCount; + private final AtomicBoolean hasFailure; + + MultiResponsesActionListener(ActionListener delegate, long maxResponseCount) { + this.delegate = delegate; + this.collectedResponseCount = new AtomicInteger(0); + this.maxResponseCount = new AtomicLong(maxResponseCount); + this.hasFailure = new AtomicBoolean(false); + } + + @Override + public void onResponse(DeleteResponse deleteResponse) { + if (collectedResponseCount.incrementAndGet() >= maxResponseCount.get()) { + finish(); + } + } + + @Override + public void onFailure(Exception e) { + this.hasFailure.set(true); + if (collectedResponseCount.incrementAndGet() >= maxResponseCount.get()) { + finish(); + } + } + + private void finish() { + if (this.hasFailure.get()) { + this.delegate.onFailure(new RuntimeException("failed to delete old AD tasks")); + } else { + try { + XContentParser parser = TestHelpers + .parser( + "{\"slice_id\":1,\"total\":2,\"updated\":0,\"created\":0,\"deleted\":" + + maxResponseCount + + ",\"batches\":6,\"version_conflicts\":0,\"noops\":0,\"retries\":{\"bulk\":0," + + "\"search\":10},\"throttled_millis\":0,\"requests_per_second\":13.0,\"canceled\":" + + "\"reasonCancelled\",\"throttled_until_millis\":14}" + ); + parser.nextToken(); + BulkByScrollResponse response = new BulkByScrollResponse( + TimeValue.timeValueMillis(10), + BulkByScrollTask.Status.innerFromXContent(parser), + ImmutableList.of(), + ImmutableList.of(), + false + ); + this.delegate.onResponse(response); + } catch (IOException exception) { + this.delegate.onFailure(new RuntimeException("failed to parse BulkByScrollResponse")); + } + } + } + } + + @Override + protected void doExecute(Task task, DeleteByQueryRequest request, ActionListener listener) { + SearchRequest searchRequest = request.getSearchRequest(); + client.search(searchRequest, ActionListener.wrap(r -> { + long totalHits = r.getHits().getTotalHits().value; + MultiResponsesActionListener delegateListener = new MultiResponsesActionListener(listener, totalHits); + Iterator iterator = r.getHits().iterator(); + while (iterator.hasNext()) { + String id = iterator.next().getId(); + DeleteRequest deleteRequest = new DeleteRequest(CommonName.DETECTION_STATE_INDEX, id) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + client.delete(deleteRequest, delegateListener); + } + }, e -> listener.onFailure(e))); + } + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManagerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManagerTests.java new file mode 100644 index 00000000..920cb050 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/task/ADTaskManagerTests.java @@ -0,0 +1,148 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.task; + +import static com.amazon.opendistroforelasticsearch.ad.TestHelpers.randomDetector; +import static com.amazon.opendistroforelasticsearch.ad.TestHelpers.randomFeature; +import static com.amazon.opendistroforelasticsearch.ad.TestHelpers.randomUser; +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.ANOMALY_RESULT_INDEX_ALIAS; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_OLD_AD_TASK_DOCS_PER_DETECTOR; +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.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; + +import com.amazon.opendistroforelasticsearch.ad.ADUnitTestCase; +import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; +import com.amazon.opendistroforelasticsearch.ad.transport.AnomalyDetectorJobResponse; +import com.google.common.collect.ImmutableList; + +public class ADTaskManagerTests extends ADUnitTestCase { + + private Settings settings; + private Client client; + private ClusterSettings clusterSettings; + private AnomalyDetectionIndices anomalyDetectionIndices; + private ADTaskManager adTaskManager; + + private Instant startTime; + private Instant endTime; + private ActionListener listener; + + @Override + public void setUp() throws Exception { + super.setUp(); + Instant now = Instant.now(); + startTime = now.minus(10, ChronoUnit.DAYS); + endTime = now.minus(1, ChronoUnit.DAYS); + + settings = Settings.builder().put(MAX_OLD_AD_TASK_DOCS_PER_DETECTOR.getKey(), 2).build(); + + clusterSettings = clusterSetting(settings, MAX_OLD_AD_TASK_DOCS_PER_DETECTOR); + + final ClusterService clusterService = new ClusterService(settings, clusterSettings, null); + + client = mock(Client.class); + anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + adTaskManager = new ADTaskManager(settings, clusterService, client, NamedXContentRegistry.EMPTY, anomalyDetectionIndices); + + listener = spy(new ActionListener() { + @Override + public void onResponse(AnomalyDetectorJobResponse bulkItemResponses) {} + + @Override + public void onFailure(Exception e) {} + }); + } + + public void testCreateTaskIndexNotAcknowledged() throws IOException { + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(0); + listener.onResponse(new CreateIndexResponse(false, false, ANOMALY_RESULT_INDEX_ALIAS)); + return null; + }).when(anomalyDetectionIndices).initDetectionStateIndex(any()); + AnomalyDetector detector = randomDetector( + new DetectionDateRange(startTime, endTime), + ImmutableList.of(randomFeature(true)), + randomAlphaOfLength(5), + 1, + randomAlphaOfLength(5) + ); + + adTaskManager.createADTaskIndex(detector, randomUser(), listener); + verify(listener, times(1)).onFailure(exceptionCaptor.capture()); + assertEquals( + "Create index .opendistro-anomaly-detection-state with mappings not acknowledged", + exceptionCaptor.getValue().getMessage() + ); + } + + public void testCreateTaskIndexWithResourceAlreadyExistsException() throws IOException { + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(0); + listener.onFailure(new ResourceAlreadyExistsException("index created")); + return null; + }).when(anomalyDetectionIndices).initDetectionStateIndex(any()); + AnomalyDetector detector = randomDetector( + new DetectionDateRange(startTime, endTime), + ImmutableList.of(randomFeature(true)), + randomAlphaOfLength(5), + 1, + randomAlphaOfLength(5) + ); + + adTaskManager.createADTaskIndex(detector, randomUser(), listener); + verify(listener, never()).onFailure(any()); + } + + public void testCreateTaskIndexWithException() throws IOException { + String error = randomAlphaOfLength(5); + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(0); + listener.onFailure(new RuntimeException(error)); + return null; + }).when(anomalyDetectionIndices).initDetectionStateIndex(any()); + AnomalyDetector detector = randomDetector( + new DetectionDateRange(startTime, endTime), + ImmutableList.of(randomFeature(true)), + randomAlphaOfLength(5), + 1, + randomAlphaOfLength(5) + ); + + adTaskManager.createADTaskIndex(detector, randomUser(), listener); + verify(listener, times(1)).onFailure(exceptionCaptor.capture()); + assertEquals(error, exceptionCaptor.getValue().getMessage()); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportActionTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportActionTests.java new file mode 100644 index 00000000..9dc252d7 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/ADBatchAnomalyResultTransportActionTests.java @@ -0,0 +1,214 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import static com.amazon.opendistroforelasticsearch.ad.TestHelpers.randomFeature; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.BATCH_TASK_PIECE_INTERVAL_SECONDS; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE; +import static com.amazon.opendistroforelasticsearch.ad.settings.EnabledSetting.AD_PLUGIN_ENABLED; + +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.common.io.stream.NotSerializableExceptionWrapper; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESIntegTestCase; +import org.junit.Before; + +import com.amazon.opendistroforelasticsearch.ad.HistoricalDetectorIntegTestCase; +import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.common.exception.EndRunException; +import com.amazon.opendistroforelasticsearch.ad.common.exception.LimitExceededException; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskState; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 2) +public class ADBatchAnomalyResultTransportActionTests extends HistoricalDetectorIntegTestCase { + + private String testIndex; + private Instant startTime; + private Instant endTime; + private String type = "error"; + private int detectionIntervalInMinutes = 1; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + testIndex = "test_historical_data"; + startTime = Instant.now().minus(10, ChronoUnit.DAYS); + endTime = Instant.now(); + ingestTestData(testIndex, startTime, detectionIntervalInMinutes, type); + createDetectionStateIndex(); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings + .builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(BATCH_TASK_PIECE_INTERVAL_SECONDS.getKey(), 1) + .put(MAX_BATCH_TASK_PER_NODE.getKey(), 1) + .build(); + } + + public void testAnomalyDetectorWithNullDetector() throws IOException { + ADTask task = randomCreatedADTask(randomAlphaOfLength(5), null); + ADBatchAnomalyResultRequest request = new ADBatchAnomalyResultRequest(task); + ActionRequestValidationException exception = expectThrows( + ActionRequestValidationException.class, + () -> client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(30_000) + ); + assertTrue(exception.getMessage().contains("Detector can't be null")); + } + + public void testRealtimeAnomalyDetector() throws IOException { + AnomalyDetector detector = randomDetector(null, ImmutableList.of(randomFeature(true))); + ADTask task = randomCreatedADTask(randomAlphaOfLength(5), detector); + ADBatchAnomalyResultRequest request = new ADBatchAnomalyResultRequest(task); + ActionRequestValidationException exception = expectThrows( + ActionRequestValidationException.class, + () -> client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(30_000) + ); + assertTrue(exception.getMessage().contains("Can't run batch task for realtime detector")); + } + + public void testAnomalyDetectorWithNullTaskId() throws IOException { + AnomalyDetector detector = randomDetector(null, ImmutableList.of(randomFeature(true))); + ADTask task = randomCreatedADTask(null, detector); + ADBatchAnomalyResultRequest request = new ADBatchAnomalyResultRequest(task); + ActionRequestValidationException exception = expectThrows( + ActionRequestValidationException.class, + () -> client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(30_000) + ); + assertTrue(exception.getMessage().contains("Can't run batch task for realtime detector")); + assertTrue(exception.getMessage().contains("Task id can't be null")); + } + + public void testHistoricalDetectorWithFutureDateRange() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(endTime, endTime.plus(10, ChronoUnit.DAYS)); + testInvalidDetectionDateRange(dateRange); + } + + public void testHistoricalDetectorWithInvalidHistoricalDateRange() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(startTime.minus(10, ChronoUnit.DAYS), startTime); + testInvalidDetectionDateRange(dateRange); + } + + public void testHistoricalDetectorWithSmallHistoricalDateRange() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(startTime, startTime.plus(10, ChronoUnit.MINUTES)); + testInvalidDetectionDateRange(dateRange, "There is no enough data to train model"); + } + + public void testHistoricalDetectorWithValidDateRange() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest(dateRange); + client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(5000); + Thread.sleep(10000); + GetResponse doc = getDoc(CommonName.DETECTION_STATE_INDEX, request.getAdTask().getTaskId()); + assertEquals(ADTaskState.FINISHED.name(), doc.getSourceAsMap().get(ADTask.STATE_FIELD)); + } + + public void testHistoricalDetectorWithNonExistingIndex() throws IOException { + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest( + new DetectionDateRange(startTime, endTime), + randomAlphaOfLength(5) + ); + client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(5000); + } + + public void testHistoricalDetectorExceedsMaxRunningTaskLimit() throws IOException, InterruptedException { + updateTransientSettings(ImmutableMap.of(MAX_BATCH_TASK_PER_NODE.getKey(), 1)); + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + for (int i = 0; i < getDataNodes().size(); i++) { + client().execute(ADBatchAnomalyResultAction.INSTANCE, adBatchAnomalyResultRequest(dateRange)); + } + + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest(dateRange); + + RuntimeException exception = expectThrowsAnyOf( + ImmutableList.of(LimitExceededException.class, NotSerializableExceptionWrapper.class), + () -> client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(5000) + ); + assertTrue( + exception + .getMessage() + .contains("All nodes' executing historical detector count exceeds limitation. No eligible node to run detector") + ); + } + + public void testDisableADPlugin() throws IOException { + updateTransientSettings(ImmutableMap.of(AD_PLUGIN_ENABLED, false)); + + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest(new DetectionDateRange(startTime, endTime)); + RuntimeException exception = expectThrowsAnyOf( + ImmutableList.of(NotSerializableExceptionWrapper.class, EndRunException.class), + () -> client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(10000) + ); + assertTrue(exception.getMessage().contains("AD plugin is disabled")); + updateTransientSettings(ImmutableMap.of(AD_PLUGIN_ENABLED, true)); + } + + public void testMultipleTasks() throws IOException, InterruptedException { + updateTransientSettings(ImmutableMap.of(MAX_BATCH_TASK_PER_NODE.getKey(), 2)); + + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + for (int i = 0; i < getDataNodes().size(); i++) { + client().execute(ADBatchAnomalyResultAction.INSTANCE, adBatchAnomalyResultRequest(dateRange)); + } + + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest( + new DetectionDateRange(startTime, startTime.plus(2000, ChronoUnit.MINUTES)) + ); + client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(5000); + Thread.sleep(10000); + GetResponse doc = getDoc(CommonName.DETECTION_STATE_INDEX, request.getAdTask().getTaskId()); + assertEquals(ADTaskState.FINISHED.name(), doc.getSourceAsMap().get(ADTask.STATE_FIELD)); + updateTransientSettings(ImmutableMap.of(MAX_BATCH_TASK_PER_NODE.getKey(), 1)); + } + + private ADBatchAnomalyResultRequest adBatchAnomalyResultRequest(DetectionDateRange dateRange) throws IOException { + return adBatchAnomalyResultRequest(dateRange, testIndex); + } + + private ADBatchAnomalyResultRequest adBatchAnomalyResultRequest(DetectionDateRange dateRange, String indexName) throws IOException { + AnomalyDetector detector = TestHelpers + .randomDetector(dateRange, ImmutableList.of(maxValueFeature()), indexName, detectionIntervalInMinutes, timeField); + ADTask adTask = randomCreatedADTask(randomAlphaOfLength(5), detector); + adTask.setTaskId(createADTask(adTask)); + return new ADBatchAnomalyResultRequest(adTask); + } + + private void testInvalidDetectionDateRange(DetectionDateRange dateRange) throws IOException, InterruptedException { + testInvalidDetectionDateRange(dateRange, "There is no data in the detection date range"); + } + + private void testInvalidDetectionDateRange(DetectionDateRange dateRange, String error) throws IOException, InterruptedException { + ADBatchAnomalyResultRequest request = adBatchAnomalyResultRequest(dateRange); + client().execute(ADBatchAnomalyResultAction.INSTANCE, request).actionGet(5000); + Thread.sleep(5000); + GetResponse doc = getDoc(CommonName.DETECTION_STATE_INDEX, request.getAdTask().getTaskId()); + assertEquals(error, doc.getSourceAsMap().get(ADTask.ERROR_FIELD)); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobActionTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobActionTests.java index 9c519f8b..6a261bb6 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobActionTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobActionTests.java @@ -43,6 +43,7 @@ import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; import com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings; +import com.amazon.opendistroforelasticsearch.ad.task.ADTaskManager; import com.amazon.opendistroforelasticsearch.commons.ConfigConstants; public class AnomalyDetectorJobActionTests extends ESIntegTestCase { @@ -77,7 +78,8 @@ public void setUp() throws Exception { clusterService, indexSettings(), mock(AnomalyDetectionIndices.class), - xContentRegistry() + xContentRegistry(), + mock(ADTaskManager.class) ); task = mock(Task.class); request = new AnomalyDetectorJobRequest("1234", 4567, 7890, "_start"); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportActionTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportActionTests.java new file mode 100644 index 00000000..572b1ff3 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyDetectorJobTransportActionTests.java @@ -0,0 +1,257 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport; + +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.BATCH_TASK_PIECE_INTERVAL_SECONDS; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_BATCH_TASK_PER_NODE; +import static com.amazon.opendistroforelasticsearch.ad.settings.AnomalyDetectorSettings.MAX_OLD_AD_TASK_DOCS_PER_DETECTOR; +import static com.amazon.opendistroforelasticsearch.ad.util.RestHandlerUtils.START_JOB; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; + +import java.io.IOException; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.junit.Before; + +import com.amazon.opendistroforelasticsearch.ad.HistoricalDetectorIntegTestCase; +import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; +import com.amazon.opendistroforelasticsearch.ad.model.ADTask; +import com.amazon.opendistroforelasticsearch.ad.model.ADTaskState; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; +import com.amazon.opendistroforelasticsearch.ad.model.DetectionDateRange; +import com.amazon.opendistroforelasticsearch.ad.plugin.MockReindexPlugin; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 2) +public class AnomalyDetectorJobTransportActionTests extends HistoricalDetectorIntegTestCase { + private Instant startTime; + private Instant endTime; + private String type = "error"; + private int maxOldAdTaskDocsPerDetector = 2; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + startTime = Instant.now().minus(10, ChronoUnit.DAYS); + endTime = Instant.now(); + ingestTestData(testIndex, startTime, detectionIntervalInMinutes, type, 2000); + createDetectorIndex(); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings + .builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(BATCH_TASK_PIECE_INTERVAL_SECONDS.getKey(), 1) + .put(MAX_BATCH_TASK_PER_NODE.getKey(), 1) + .put(MAX_OLD_AD_TASK_DOCS_PER_DETECTOR.getKey(), maxOldAdTaskDocsPerDetector) + .build(); + } + + @Override + protected Collection> getMockPlugins() { + final ArrayList> plugins = new ArrayList<>(); + plugins.add(MockReindexPlugin.class); + plugins.addAll(super.getMockPlugins()); + return Collections.unmodifiableList(plugins); + } + + public void testDetectorIndexNotFound() { + deleteDetectorIndex(); + String detectorId = randomAlphaOfLength(5); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + IndexNotFoundException exception = expectThrows( + IndexNotFoundException.class, + () -> client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(3000) + ); + assertTrue(exception.getMessage().contains("no such index [.opendistro-anomaly-detectors]")); + } + + public void testDetectorNotFound() { + String detectorId = randomAlphaOfLength(5); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + ElasticsearchStatusException exception = expectThrows( + ElasticsearchStatusException.class, + () -> client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(3000) + ); + assertTrue(exception.getMessage().contains("AnomalyDetector is not found")); + } + + public void testValidHistoricalDetector() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + AnomalyDetector detector = TestHelpers + .randomDetector(dateRange, ImmutableList.of(maxValueFeature()), testIndex, detectionIntervalInMinutes, timeField); + String detectorId = createDetector(detector); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + AnomalyDetectorJobResponse response = client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(10000); + Thread.sleep(10000); + GetResponse doc = getDoc(CommonName.DETECTION_STATE_INDEX, response.getId()); + assertEquals(ADTaskState.FINISHED.name(), doc.getSourceAsMap().get(ADTask.STATE_FIELD)); + } + + public void testRunMultipleTasksForHistoricalDetector() throws IOException, InterruptedException { + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + AnomalyDetector detector = TestHelpers + .randomDetector(dateRange, ImmutableList.of(maxValueFeature()), testIndex, detectionIntervalInMinutes, timeField); + String detectorId = createDetector(detector); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + AnomalyDetectorJobResponse response = client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(10000); + assertNotNull(response.getId()); + ElasticsearchStatusException exception = expectThrows( + ElasticsearchStatusException.class, + () -> client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(10000) + ); + assertTrue(exception.getMessage().contains("Detector is already running")); + } + + public void testCleanOldTaskDocs() throws IOException, InterruptedException { + updateTransientSettings(ImmutableMap.of(BATCH_TASK_PIECE_INTERVAL_SECONDS.getKey(), 1)); + DetectionDateRange dateRange = new DetectionDateRange(startTime, endTime); + AnomalyDetector detector = TestHelpers + .randomDetector(dateRange, ImmutableList.of(maxValueFeature()), testIndex, detectionIntervalInMinutes, timeField); + String detectorId = createDetector(detector); + + createDetectionStateIndex(); + List states = ImmutableList.of(ADTaskState.FAILED, ADTaskState.FINISHED, ADTaskState.STOPPED); + for (ADTaskState state : states) { + ADTask task = randomADTask(randomAlphaOfLength(5), detector, detectorId, state); + createADTask(task); + } + long count = countDocs(CommonName.DETECTION_STATE_INDEX); + assertEquals(states.size(), count); + + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest(detectorId, randomLong(), randomLong(), START_JOB); + AtomicReference response = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + client().execute(AnomalyDetectorJobAction.INSTANCE, request, ActionListener.wrap(r -> { + latch.countDown(); + response.set(r); + }, e -> { latch.countDown(); })); + latch.await(); + Thread.sleep(10000); + count = countDocs(CommonName.DETECTION_STATE_INDEX); + // we have one latest task, so total count should add 1 + assertEquals(maxOldAdTaskDocsPerDetector + 1, count); + } + + public void testStartRealtimeDetector() throws IOException { + AnomalyDetector detector = TestHelpers + .randomDetector(null, ImmutableList.of(maxValueFeature()), testIndex, detectionIntervalInMinutes, timeField); + String detectorId = createDetector(detector); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + AnomalyDetectorJobResponse response = client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(10000); + assertEquals(detectorId, response.getId()); + GetResponse doc = getDoc(AnomalyDetectorJob.ANOMALY_DETECTOR_JOB_INDEX, detectorId); + AnomalyDetectorJob job = toADJob(doc); + assertTrue(job.isEnabled()); + assertEquals(detectorId, job.getName()); + } + + public void testRealtimeDetectorWithoutFeature() throws IOException { + AnomalyDetector detector = TestHelpers.randomDetector(null, ImmutableList.of(), testIndex, detectionIntervalInMinutes, timeField); + testInvalidDetector(detector, "Can't start detector job as no features configured"); + } + + public void testHistoricalDetectorWithoutFeature() throws IOException { + AnomalyDetector detector = TestHelpers + .randomDetector( + new DetectionDateRange(startTime, endTime), + ImmutableList.of(), + testIndex, + detectionIntervalInMinutes, + timeField + ); + testInvalidDetector(detector, "Can't start detector job as no features configured"); + } + + public void testRealtimeDetectorWithoutEnabledFeature() throws IOException { + AnomalyDetector detector = TestHelpers + .randomDetector(null, ImmutableList.of(TestHelpers.randomFeature(false)), testIndex, detectionIntervalInMinutes, timeField); + testInvalidDetector(detector, "Can't start detector job as no enabled features configured"); + } + + public void testHistoricalDetectorWithoutEnabledFeature() throws IOException { + AnomalyDetector detector = TestHelpers + .randomDetector( + new DetectionDateRange(startTime, endTime), + ImmutableList.of(TestHelpers.randomFeature(false)), + testIndex, + detectionIntervalInMinutes, + timeField + ); + testInvalidDetector(detector, "Can't start detector job as no enabled features configured"); + } + + private void testInvalidDetector(AnomalyDetector detector, String error) throws IOException { + String detectorId = createDetector(detector); + AnomalyDetectorJobRequest request = new AnomalyDetectorJobRequest( + detectorId, + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + START_JOB + ); + ElasticsearchStatusException exception = expectThrows( + ElasticsearchStatusException.class, + () -> client().execute(AnomalyDetectorJobAction.INSTANCE, request).actionGet(10000) + ); + assertEquals(error, exception.getMessage()); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTests.java index f31435da..7090ea8e 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTests.java @@ -284,14 +284,12 @@ public void setUp() throws Exception { GetRequest request = (GetRequest) args[0]; ActionListener listener = (ActionListener) args[1]; - if (request.index().equals(DetectorInternalState.DETECTOR_STATE_INDEX)) { + if (request.index().equals(CommonName.DETECTION_STATE_INDEX)) { DetectorInternalState.Builder result = new DetectorInternalState.Builder().lastUpdateTime(Instant.now()); listener - .onResponse( - TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), DetectorInternalState.DETECTOR_STATE_INDEX) - ); + .onResponse(TestHelpers.createGetResponse(result.build(), detector.getDetectorId(), CommonName.DETECTION_STATE_INDEX)); } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTransportActionTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTransportActionTests.java index 63d2355a..7328b8eb 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTransportActionTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/AnomalyResultTransportActionTests.java @@ -43,8 +43,6 @@ public class AnomalyResultTransportActionTests extends ADIntegTestCase { private Instant testDataTimeStamp; private long start; private long end; - private String timeField = "timestamp"; - private String categoryField = "type"; @Override @Before @@ -58,14 +56,7 @@ public void setUp() throws Exception { } private void ingestTestData() throws IOException { - String mappings = "{\"properties\":{\"" - + timeField - + "\":{\"type\":\"date\",\"format\":\"strict_date_time||epoch_millis\"}," - + "\"value\":{\"type\":\"double\"}, \"" - + categoryField - + "\":{\"type\":\"keyword\"}," - + "\"is_error\":{\"type\":\"boolean\"}, \"message\":{\"type\":\"text\"}}}"; - createIndex(testIndex, mappings); + createTestDataIndex(testIndex); double value = randomDouble(); String type = randomAlphaOfLength(5); boolean isError = randomBoolean(); @@ -205,7 +196,7 @@ private String createDetectorWithFeatureAgg(String aggQuery, boolean hcDetector) AnomalyDetector detector = hcDetector ? randomHCDetector(ImmutableList.of(testIndex), ImmutableList.of(feature)) : randomDetector(ImmutableList.of(testIndex), ImmutableList.of(feature)); - String adId = createDetectors(detector); + String adId = createDetector(detector); return adId; } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java new file mode 100644 index 00000000..eda6fadd --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/AnomalyResultBulkIndexHandlerTests.java @@ -0,0 +1,151 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.transport.handler; + +import static com.amazon.opendistroforelasticsearch.ad.constant.CommonName.ANOMALY_RESULT_INDEX_ALIAS; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.IOException; +import java.time.Clock; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.ThreadPool; + +import com.amazon.opendistroforelasticsearch.ad.ADUnitTestCase; +import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyResult; +import com.amazon.opendistroforelasticsearch.ad.util.ClientUtil; +import com.amazon.opendistroforelasticsearch.ad.util.IndexUtils; +import com.amazon.opendistroforelasticsearch.ad.util.Throttler; +import com.amazon.opendistroforelasticsearch.ad.util.ThrowingConsumerWrapper; +import com.google.common.collect.ImmutableList; + +public class AnomalyResultBulkIndexHandlerTests extends ADUnitTestCase { + + private AnomalyResultBulkIndexHandler bulkIndexHandler; + private Client client; + private IndexUtils indexUtils; + private ActionListener listener; + private AnomalyDetectionIndices anomalyDetectionIndices; + + @Override + public void setUp() throws Exception { + super.setUp(); + anomalyDetectionIndices = mock(AnomalyDetectionIndices.class); + client = mock(Client.class); + Settings settings = Settings.EMPTY; + Clock clock = mock(Clock.class); + Throttler throttler = new Throttler(clock); + ThreadPool threadpool = mock(ThreadPool.class); + ClientUtil clientUtil = new ClientUtil(Settings.EMPTY, client, throttler, threadpool); + indexUtils = mock(IndexUtils.class); + ClusterService clusterService = mock(ClusterService.class); + ThreadPool threadPool = mock(ThreadPool.class); + bulkIndexHandler = new AnomalyResultBulkIndexHandler( + client, + settings, + threadPool, + ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectionStateIndex), + anomalyDetectionIndices::doesDetectorStateIndexExist, + clientUtil, + indexUtils, + clusterService, + anomalyDetectionIndices + ); + listener = spy(new ActionListener() { + @Override + public void onResponse(BulkResponse bulkItemResponses) {} + + @Override + public void onFailure(Exception e) {} + }); + } + + public void testNullAnomalyResults() { + bulkIndexHandler.bulkIndexAnomalyResult(null, listener); + verify(listener, times(1)).onResponse(null); + verify(anomalyDetectionIndices, never()).doesAnomalyDetectorIndexExist(); + } + + public void testCreateADResultIndexNotAcknowledged() throws IOException { + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(0); + listener.onResponse(new CreateIndexResponse(false, false, ANOMALY_RESULT_INDEX_ALIAS)); + return null; + }).when(anomalyDetectionIndices).initAnomalyResultIndexDirectly(any()); + bulkIndexHandler.bulkIndexAnomalyResult(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()); + } + + public void testWrongAnomalyResult() { + BulkRequestBuilder bulkRequestBuilder = mock(BulkRequestBuilder.class); + doReturn(bulkRequestBuilder).when(client).prepareBulk(); + doReturn(true).when(anomalyDetectionIndices).doesAnomalyResultIndexExist(); + bulkIndexHandler.bulkIndexAnomalyResult(ImmutableList.of(wrongAnomalyResult(), TestHelpers.randomAnomalyDetectResult()), listener); + verify(listener, times(1)).onFailure(exceptionCaptor.capture()); + assertEquals("Failed to prepare request to bulk index anomaly results", exceptionCaptor.getValue().getMessage()); + } + + public void testBulkSaveException() { + BulkRequestBuilder bulkRequestBuilder = mock(BulkRequestBuilder.class); + doReturn(bulkRequestBuilder).when(client).prepareBulk(); + doReturn(true).when(anomalyDetectionIndices).doesAnomalyResultIndexExist(); + + String testError = randomAlphaOfLength(5); + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(1); + listener.onFailure(new RuntimeException(testError)); + return null; + }).when(client).bulk(any(), any()); + + bulkIndexHandler.bulkIndexAnomalyResult(ImmutableList.of(TestHelpers.randomAnomalyDetectResult()), listener); + verify(listener, times(1)).onFailure(exceptionCaptor.capture()); + assertEquals(testError, exceptionCaptor.getValue().getMessage()); + } + + private AnomalyResult wrongAnomalyResult() { + return new AnomalyResult( + randomAlphaOfLength(5), + randomDouble(), + randomDouble(), + randomDouble(), + null, + null, + null, + null, + null, + randomAlphaOfLength(5), + null, + null, + null + ); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectorStateHandlerTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectorStateHandlerTests.java index d710ec42..75ae5f93 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectorStateHandlerTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/transport/handler/DetectorStateHandlerTests.java @@ -38,6 +38,7 @@ import com.amazon.opendistroforelasticsearch.ad.NodeStateManager; import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.indices.AnomalyDetectionIndices; import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.transport.handler.DetectionStateHandler.ErrorStrategy; @@ -72,7 +73,7 @@ public void setUp() throws Exception { client, settings, threadPool, - ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectorStateIndex), + ThrowingConsumerWrapper.throwingConsumerWrapper(anomalyDetectionIndices::initDetectionStateIndex), anomalyDetectionIndices::doesDetectorStateIndexExist, clientUtil, indexUtils, @@ -119,7 +120,7 @@ public void testNoUpdateWitoutErrorChange() { @SuppressWarnings("unchecked") ActionListener listener = (ActionListener) args[1]; DetectorInternalState.Builder result = new DetectorInternalState.Builder().lastUpdateTime(Instant.now()).error(error); - listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, DetectorInternalState.DETECTOR_STATE_INDEX)); + listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, CommonName.DETECTION_STATE_INDEX)); return null; }).when(client).get(any(), any()); @@ -135,7 +136,7 @@ public void testUpdateWithErrorChange() { @SuppressWarnings("unchecked") ActionListener listener = (ActionListener) args[1]; DetectorInternalState.Builder result = new DetectorInternalState.Builder().lastUpdateTime(Instant.now()).error("blah"); - listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, DetectorInternalState.DETECTOR_STATE_INDEX)); + listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, CommonName.DETECTION_STATE_INDEX)); return null; }).when(client).get(any(), any()); @@ -153,7 +154,7 @@ public void testUpdateWithFirstChange() { DetectorInternalState.Builder result = new DetectorInternalState.Builder() .lastUpdateTime(Instant.ofEpochMilli(1)) .error("blah"); - listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, DetectorInternalState.DETECTOR_STATE_INDEX)); + listener.onResponse(TestHelpers.createGetResponse(result.build(), detectorId, CommonName.DETECTION_STATE_INDEX)); return null; }).when(client).get(any(), any()); diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtilsTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtilsTests.java new file mode 100644 index 00000000..ebba6820 --- /dev/null +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ExceptionUtilsTests.java @@ -0,0 +1,87 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazon.opendistroforelasticsearch.ad.util; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESTestCase; + +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; + +public class ExceptionUtilsTests extends ESTestCase { + + public void testGetShardsFailure() { + ShardId shardId = new ShardId(randomAlphaOfLength(5), randomAlphaOfLength(5), 1); + ReplicationResponse.ShardInfo.Failure failure = new ReplicationResponse.ShardInfo.Failure( + shardId, + randomAlphaOfLength(5), + new RuntimeException("test"), + RestStatus.BAD_REQUEST, + false + ); + ReplicationResponse.ShardInfo shardInfo = new ReplicationResponse.ShardInfo(2, 1, failure); + IndexResponse indexResponse = new IndexResponse( + shardId, + randomAlphaOfLength(5), + randomAlphaOfLength(5), + randomLong(), + randomLong(), + randomLong(), + randomBoolean() + ); + indexResponse.setShardInfo(shardInfo); + String shardsFailure = ExceptionUtil.getShardsFailure(indexResponse); + assertEquals("RuntimeException[test]", shardsFailure); + } + + public void testGetShardsFailureWithoutError() { + ShardId shardId = new ShardId(randomAlphaOfLength(5), randomAlphaOfLength(5), 1); + IndexResponse indexResponse = new IndexResponse( + shardId, + randomAlphaOfLength(5), + randomAlphaOfLength(5), + randomLong(), + randomLong(), + randomLong(), + randomBoolean() + ); + assertNull(ExceptionUtil.getShardsFailure(indexResponse)); + + ReplicationResponse.ShardInfo shardInfo = new ReplicationResponse.ShardInfo(2, 1, ReplicationResponse.EMPTY); + indexResponse.setShardInfo(shardInfo); + assertNull(ExceptionUtil.getShardsFailure(indexResponse)); + } + + public void testCountInStats() { + assertTrue(ExceptionUtil.countInStats(new AnomalyDetectionException("test"))); + assertFalse(ExceptionUtil.countInStats(new AnomalyDetectionException("test").countedInStats(false))); + assertTrue(ExceptionUtil.countInStats(new RuntimeException("test"))); + } + + public void testGetErrorMessage() { + assertEquals("test", ExceptionUtil.getErrorMessage(new AnomalyDetectionException("test"))); + assertEquals("test", ExceptionUtil.getErrorMessage(new IllegalArgumentException("test"))); + assertEquals("org.elasticsearch.ElasticsearchException: test", ExceptionUtil.getErrorMessage(new ElasticsearchException("test"))); + assertTrue( + ExceptionUtil + .getErrorMessage(new RuntimeException("test")) + .contains("at com.amazon.opendistroforelasticsearch.ad.util.ExceptionUtilsTests.testGetErrorMessage") + ); + } +} diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtilsTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtilsTests.java index 6e60cb4c..4953084d 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtilsTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/ParseUtilsTests.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.time.Instant; +import java.time.temporal.ChronoUnit; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -29,7 +30,9 @@ import org.elasticsearch.test.ESTestCase; import com.amazon.opendistroforelasticsearch.ad.TestHelpers; +import com.amazon.opendistroforelasticsearch.ad.common.exception.AnomalyDetectionException; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; +import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorType; import com.amazon.opendistroforelasticsearch.ad.model.Feature; import com.amazon.opendistroforelasticsearch.commons.authuser.User; import com.google.common.collect.ImmutableList; @@ -182,4 +185,98 @@ public void testAddUserRoleFilterWithNormalUserBackendRole() { searchSourceBuilder.toString() ); } + + public void testBatchFeatureQuery() throws IOException { + String index = randomAlphaOfLength(5); + Instant now = Instant.now().truncatedTo(ChronoUnit.SECONDS); + Feature feature1 = TestHelpers.randomFeature(true); + Feature feature2 = TestHelpers.randomFeature(false); + AnomalyDetector detector = TestHelpers + .randomAnomalyDetector( + ImmutableList.of(index), + ImmutableList.of(feature1, feature2), + null, + now, + AnomalyDetectorType.HISTORICAL_MULTI_ENTITY.name(), + 1, + TestHelpers.randomDetectionDateRange(), + false + ); + + long startTime = now.minus(10, ChronoUnit.DAYS).toEpochMilli(); + long endTime = now.plus(10, ChronoUnit.DAYS).toEpochMilli(); + SearchSourceBuilder searchSourceBuilder = ParseUtils + .batchFeatureQuery(detector, startTime, endTime, TestHelpers.xContentRegistry()); + assertEquals( + "{\"size\":0,\"query\":{\"bool\":{\"must\":[{\"range\":{\"" + + detector.getTimeField() + + "\":{\"from\":" + + startTime + + ",\"to\":" + + endTime + + ",\"include_lower\":true,\"include_upper\":false,\"format\":\"epoch_millis\",\"boost\"" + + ":1.0}}},{\"bool\":{\"must\":[{\"term\":{\"user\":{\"value\":\"kimchy\",\"boost\":1.0}}}],\"filter\":" + + "[{\"term\":{\"tag\":{\"value\":\"tech\",\"boost\":1.0}}}],\"must_not\":[{\"range\":{\"age\":{\"from\":10," + + "\"to\":20,\"include_lower\":true,\"include_upper\":true,\"boost\":1.0}}}],\"should\":[{\"term\":{\"tag\":" + + "{\"value\":\"wow\",\"boost\":1.0}}},{\"term\":{\"tag\":{\"value\":\"elasticsearch\",\"boost\":1.0}}}]," + + "\"adjust_pure_negative\":true,\"minimum_should_match\":\"1\",\"boost\":1.0}}],\"adjust_pure_negative" + + "\":true,\"boost\":1.0}},\"aggregations\":{\"feature_aggs\":{\"composite\":{\"size\":1000,\"sources\":" + + "[{\"date_histogram\":{\"date_histogram\":{\"field\":\"" + + detector.getTimeField() + + "\",\"missing_bucket\":false,\"order\":\"asc\"," + + "\"fixed_interval\":\"60s\"}}}]},\"aggregations\":{\"" + + feature1.getId() + + "\":{\"value_count\":{\"field\":\"ok\"}}}}}}", + searchSourceBuilder.toString() + ); + } + + public void testBatchFeatureQueryWithoutEnabledFeature() throws IOException { + String index = randomAlphaOfLength(5); + Instant now = Instant.now().truncatedTo(ChronoUnit.SECONDS); + AnomalyDetector detector = TestHelpers + .randomAnomalyDetector( + ImmutableList.of(index), + ImmutableList.of(TestHelpers.randomFeature(false)), + null, + now, + AnomalyDetectorType.HISTORICAL_MULTI_ENTITY.name(), + 1, + TestHelpers.randomDetectionDateRange(), + false + ); + + long startTime = now.minus(10, ChronoUnit.DAYS).toEpochMilli(); + long endTime = now.plus(10, ChronoUnit.DAYS).toEpochMilli(); + + AnomalyDetectionException exception = expectThrows( + AnomalyDetectionException.class, + () -> ParseUtils.batchFeatureQuery(detector, startTime, endTime, TestHelpers.xContentRegistry()) + ); + assertEquals("No enabled feature configured", exception.getMessage()); + } + + public void testBatchFeatureQueryWithoutFeature() throws IOException { + String index = randomAlphaOfLength(5); + Instant now = Instant.now().truncatedTo(ChronoUnit.SECONDS); + AnomalyDetector detector = TestHelpers + .randomAnomalyDetector( + ImmutableList.of(index), + ImmutableList.of(), + null, + now, + AnomalyDetectorType.HISTORICAL_MULTI_ENTITY.name(), + 1, + TestHelpers.randomDetectionDateRange(), + false + ); + + long startTime = now.minus(10, ChronoUnit.DAYS).toEpochMilli(); + long endTime = now.plus(10, ChronoUnit.DAYS).toEpochMilli(); + AnomalyDetectionException exception = expectThrows( + AnomalyDetectionException.class, + () -> ParseUtils.batchFeatureQuery(detector, startTime, endTime, TestHelpers.xContentRegistry()) + ); + assertEquals("No enabled feature configured", exception.getMessage()); + } } diff --git a/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/RestHandlerUtilsTests.java b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/RestHandlerUtilsTests.java index 1406963b..0b57ef50 100644 --- a/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/RestHandlerUtilsTests.java +++ b/src/test/java/com/amazon/opendistroforelasticsearch/ad/util/RestHandlerUtilsTests.java @@ -60,12 +60,6 @@ public void testCreateXContentParser() throws IOException { parser.close(); } - public void testValidateAnomalyDetectorWithNullFeatures() throws IOException { - AnomalyDetector detector = TestHelpers.randomAnomalyDetector(null); - String error = RestHandlerUtils.validateAnomalyDetector(detector, 1); - assertNull(error); - } - public void testValidateAnomalyDetectorWithTooManyFeatures() throws IOException { AnomalyDetector detector = TestHelpers.randomAnomalyDetector(ImmutableList.of(randomFeature(), randomFeature())); String error = RestHandlerUtils.validateAnomalyDetector(detector, 1); diff --git a/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityProfileTests.java b/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityProfileTests.java index 10d7e463..8b90c4f7 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityProfileTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/metrics/CardinalityProfileTests.java @@ -50,7 +50,6 @@ import com.amazon.opendistroforelasticsearch.ad.constant.CommonName; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetector; import com.amazon.opendistroforelasticsearch.ad.model.AnomalyDetectorJob; -import com.amazon.opendistroforelasticsearch.ad.model.DetectorInternalState; import com.amazon.opendistroforelasticsearch.ad.model.IntervalTimeConfiguration; import com.amazon.opendistroforelasticsearch.ad.transport.ProfileAction; import com.amazon.opendistroforelasticsearch.ad.transport.ProfileNodeResponse; @@ -109,7 +108,7 @@ private void setUpMultiEntityClientGet(DetectorStatus detectorStatus, JobStatus assertTrue("should not reach here", false); break; } - } else if (request.index().equals(DetectorInternalState.DETECTOR_STATE_INDEX)) { + } else if (request.index().equals(CommonName.DETECTION_STATE_INDEX)) { switch (errorResultStatus) { case NO_ERROR: listener.onResponse(null);