Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import org.elasticsearch.aggregations.bucket.adjacency.ParsedAdjacencyMatrix;
import org.elasticsearch.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder;
import org.elasticsearch.aggregations.bucket.histogram.ParsedAutoDateHistogram;
import org.elasticsearch.aggregations.bucket.timeseries.ParsedTimeSeries;
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
import org.elasticsearch.aggregations.pipeline.DerivativePipelineAggregationBuilder;
import org.elasticsearch.client.analytics.ParsedStringStats;
import org.elasticsearch.client.analytics.ParsedTopMetrics;
Expand Down Expand Up @@ -163,8 +165,6 @@
import org.elasticsearch.search.aggregations.pipeline.ParsedStatsBucket;
import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.StatsBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.timeseries.ParsedTimeSeries;
import org.elasticsearch.search.aggregations.timeseries.TimeSeriesAggregationBuilder;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0 and the Server Side Public License, v 1; you may not use this file except
* in compliance with, at your election, the Elastic License 2.0 or the Server
* Side Public License, v 1.
*/

package org.elasticsearch.aggregations.bucket;

import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.aggregations.AggregationsPlugin;
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.aggregations.metrics.ScriptedMetricAggregationBuilder;
import org.elasticsearch.test.AbstractSearchCancellationTestCase;

import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;

import static org.elasticsearch.index.IndexSettings.TIME_SERIES_END_TIME;
import static org.elasticsearch.index.IndexSettings.TIME_SERIES_START_TIME;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;

public class SearchCancellationIT extends AbstractSearchCancellationTestCase {

@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
List<Class<? extends Plugin>> plugins = new ArrayList<>(super.nodePlugins());
plugins.add(AggregationsPlugin.class);
return List.copyOf(plugins);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

probably fine to just return plugins?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test also needs a plugin from the super class. This is the test plugin that allows for testing the cancellation.

}

public void testCancellationDuringTimeSeriesAggregation() throws Exception {
List<ScriptedBlockPlugin> plugins = initBlockFactory();
int numberOfShards = between(2, 5);
long now = Instant.now().toEpochMilli();
int numberOfRefreshes = between(1, 5);
// After a few initial checks we check every 2048 - number of shards records so we need to ensure all
// shards have enough records to trigger a check
int numberOfDocsPerRefresh = numberOfShards * between(3000, 3500) / numberOfRefreshes;
assertAcked(
prepareCreate("test").setSettings(
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numberOfShards)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
.put(IndexSettings.MODE.getKey(), IndexMode.TIME_SERIES.name())
.put(IndexMetadata.INDEX_ROUTING_PATH.getKey(), "dim")
.put(TIME_SERIES_START_TIME.getKey(), now)
.put(TIME_SERIES_END_TIME.getKey(), now + (long) numberOfRefreshes * numberOfDocsPerRefresh + 1)
.build()
).setMapping("""
{
"properties": {
"@timestamp": {"type": "date", "format": "epoch_millis"},
"dim": {"type": "keyword", "time_series_dimension": true}
}
}
""")
);

for (int i = 0; i < numberOfRefreshes; i++) {
// Make sure we sometimes have a few segments
BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
for (int j = 0; j < numberOfDocsPerRefresh; j++) {
bulkRequestBuilder.add(
client().prepareIndex("test")
.setOpType(DocWriteRequest.OpType.CREATE)
.setSource(
"@timestamp",
now + (long) i * numberOfDocsPerRefresh + j,
"val",
(double) j,
"dim",
String.valueOf(j % 100)
)
);
}
assertNoFailures(bulkRequestBuilder.get());
}

logger.info("Executing search");
TimeSeriesAggregationBuilder timeSeriesAggregationBuilder = new TimeSeriesAggregationBuilder("test_agg");
ActionFuture<SearchResponse> searchResponse = client().prepareSearch("test")
.setQuery(matchAllQuery())
.addAggregation(
timeSeriesAggregationBuilder.subAggregation(
new ScriptedMetricAggregationBuilder("sub_agg").initScript(
new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.INIT_SCRIPT_NAME, Collections.emptyMap())
)
.mapScript(
new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.MAP_BLOCK_SCRIPT_NAME, Collections.emptyMap())
)
.combineScript(
new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.COMBINE_SCRIPT_NAME, Collections.emptyMap())
)
.reduceScript(
new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.REDUCE_FAIL_SCRIPT_NAME, Collections.emptyMap())
)
)
)
.execute();
awaitForBlock(plugins);
cancelSearch(SearchAction.NAME);
disableBlocks(plugins);

SearchPhaseExecutionException ex = expectThrows(SearchPhaseExecutionException.class, searchResponse::actionGet);
assertThat(ExceptionsHelper.status(ex), equalTo(RestStatus.BAD_REQUEST));
logger.info("All shards failed with", ex);
if (lowLevelCancellation) {
// Ensure that we cancelled in TimeSeriesIndexSearcher and not in reduce phase
assertThat(ExceptionsHelper.stackTrace(ex), containsString("TimeSeriesIndexSearcher"));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,19 +6,25 @@
* Side Public License, v 1.
*/

package org.elasticsearch.search.aggregations;
package org.elasticsearch.aggregations.bucket;

import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.aggregations.AggregationIntegTestCase;
import org.elasticsearch.aggregations.bucket.timeseries.InternalTimeSeries;
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.RangeQueryBuilder;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.PipelineAggregatorBuilders;
import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation;
import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval;
Expand All @@ -28,7 +34,6 @@
import org.elasticsearch.search.aggregations.metrics.Stats;
import org.elasticsearch.search.aggregations.metrics.Sum;
import org.elasticsearch.search.aggregations.pipeline.SimpleValue;
import org.elasticsearch.search.aggregations.timeseries.TimeSeries;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.xcontent.XContentBuilder;
Expand All @@ -49,7 +54,6 @@
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.search.aggregations.AggregationBuilders.timeSeries;
import static org.elasticsearch.search.aggregations.AggregationBuilders.topHits;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
Expand All @@ -61,7 +65,7 @@
import static org.hamcrest.Matchers.lessThanOrEqualTo;

@ESIntegTestCase.SuiteScopeTestCase
public class TimeSeriesAggregationsIT extends ESIntegTestCase {
public class TimeSeriesAggregationsIT extends AggregationIntegTestCase {

private static final Map<Map<String, String>, Map<Long, Map<String, Double>>> data = new HashMap<>();
private static int numberOfDimensions;
Expand Down Expand Up @@ -175,12 +179,12 @@ public void testStandAloneTimeSeriesAgg() {
assertSearchResponse(response);
Aggregations aggregations = response.getAggregations();
assertNotNull(aggregations);
TimeSeries timeSeries = aggregations.get("by_ts");
InternalTimeSeries timeSeries = aggregations.get("by_ts");
assertThat(
timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()),
equalTo(data.keySet())
);
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
assertThat((long) data.get(key).size(), equalTo(bucket.getDocCount()));
Expand All @@ -204,8 +208,8 @@ public void testTimeSeriesGroupedByADimension() {
Terms terms = aggregations.get("by_dim");
Set<Map<String, String>> keys = new HashSet<>();
for (Terms.Bucket term : terms.getBuckets()) {
TimeSeries timeSeries = term.getAggregations().get("by_ts");
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
InternalTimeSeries timeSeries = term.getAggregations().get("by_ts");
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
assertThat((long) data.get(key).size(), equalTo(bucket.getDocCount()));
Expand Down Expand Up @@ -234,8 +238,8 @@ public void testTimeSeriesGroupedByDateHistogram() {
for (Histogram.Bucket interval : histogram.getBuckets()) {
long intervalStart = ((ZonedDateTime) interval.getKey()).toEpochSecond() * 1000;
long intervalEnd = intervalStart + fixedInterval.estimateMillis();
TimeSeries timeSeries = interval.getAggregations().get("by_ts");
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
InternalTimeSeries timeSeries = interval.getAggregations().get("by_ts");
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
keys.compute(key, (k, v) -> (v == null ? 0 : v) + bucket.getDocCount());
Expand Down Expand Up @@ -269,13 +273,13 @@ public void testStandAloneTimeSeriesAggWithDimFilter() {
assertSearchResponse(response);
Aggregations aggregations = response.getAggregations();
assertNotNull(aggregations);
TimeSeries timeSeries = aggregations.get("by_ts");
InternalTimeSeries timeSeries = aggregations.get("by_ts");
Map<Map<String, String>, Map<Long, Map<String, Double>>> filteredData = dataFilteredByDimension("dim_" + dim, val, include);
assertThat(
timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()),
equalTo(filteredData.keySet())
);
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
assertThat(bucket.getDocCount(), equalTo((long) filteredData.get(key).size()));
Expand All @@ -301,13 +305,13 @@ public void testStandAloneTimeSeriesAggWithGlobalAggregation() {
assertSearchResponse(response);
Aggregations aggregations = response.getAggregations();
assertNotNull(aggregations);
TimeSeries timeSeries = aggregations.get("by_ts");
InternalTimeSeries timeSeries = aggregations.get("by_ts");
Map<Map<String, String>, Map<Long, Map<String, Double>>> filteredData = dataFilteredByDimension("dim_" + dim, val, include);
assertThat(
timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()),
equalTo(filteredData.keySet())
);
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
assertThat(bucket.getDocCount(), equalTo((long) filteredData.get(key).size()));
Expand Down Expand Up @@ -348,13 +352,13 @@ public void testStandAloneTimeSeriesAggWithMetricFilter() {
assertSearchResponse(response);
Aggregations aggregations = response.getAggregations();
assertNotNull(aggregations);
TimeSeries timeSeries = aggregations.get("by_ts");
InternalTimeSeries timeSeries = aggregations.get("by_ts");
Map<Map<String, String>, Map<Long, Map<String, Double>>> filteredData = dataFilteredByMetric(data, "metric_" + metric, val, above);
assertThat(
timeSeries.getBuckets().stream().map(MultiBucketsAggregation.Bucket::getKey).collect(Collectors.toSet()),
equalTo(filteredData.keySet())
);
for (TimeSeries.Bucket bucket : timeSeries.getBuckets()) {
for (InternalTimeSeries.Bucket bucket : timeSeries.getBuckets()) {
@SuppressWarnings("unchecked")
Map<String, String> key = (Map<String, String>) bucket.getKey();
assertThat(bucket.getDocCount(), equalTo((long) filteredData.get(key).size()));
Expand Down Expand Up @@ -521,4 +525,8 @@ public void testGetHitsFailure() throws Exception {

}

public static TimeSeriesAggregationBuilder timeSeries(String name) {
return new TimeSeriesAggregationBuilder(name);
}

}
1 change: 1 addition & 0 deletions modules/aggregations/src/main/java/module-info.java
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

exports org.elasticsearch.aggregations.bucket.histogram;
exports org.elasticsearch.aggregations.bucket.adjacency;
exports org.elasticsearch.aggregations.bucket.timeseries;
exports org.elasticsearch.aggregations.pipeline;

opens org.elasticsearch.aggregations to org.elasticsearch.painless.spi; // whitelist resource access
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,34 +12,51 @@
import org.elasticsearch.aggregations.bucket.adjacency.InternalAdjacencyMatrix;
import org.elasticsearch.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder;
import org.elasticsearch.aggregations.bucket.histogram.InternalAutoDateHistogram;
import org.elasticsearch.aggregations.bucket.timeseries.InternalTimeSeries;
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
import org.elasticsearch.aggregations.pipeline.BucketSortPipelineAggregationBuilder;
import org.elasticsearch.aggregations.pipeline.Derivative;
import org.elasticsearch.aggregations.pipeline.DerivativePipelineAggregationBuilder;
import org.elasticsearch.aggregations.pipeline.MovFnPipelineAggregationBuilder;
import org.elasticsearch.aggregations.pipeline.MovingFunctionScript;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.ScriptPlugin;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.script.ScriptContext;

import java.util.ArrayList;
import java.util.List;

public class AggregationsPlugin extends Plugin implements SearchPlugin, ScriptPlugin {
@Override
public List<AggregationSpec> getAggregations() {
return List.of(
List<AggregationSpec> specs = new ArrayList<>();
specs.add(
new AggregationSpec(
AdjacencyMatrixAggregationBuilder.NAME,
AdjacencyMatrixAggregationBuilder::new,
AdjacencyMatrixAggregationBuilder::parse
).addResultReader(InternalAdjacencyMatrix::new),
).addResultReader(InternalAdjacencyMatrix::new)
);
specs.add(
new AggregationSpec(
AutoDateHistogramAggregationBuilder.NAME,
AutoDateHistogramAggregationBuilder::new,
AutoDateHistogramAggregationBuilder.PARSER
).addResultReader(InternalAutoDateHistogram::new)
.setAggregatorRegistrar(AutoDateHistogramAggregationBuilder::registerAggregators)
);
if (IndexSettings.isTimeSeriesModeEnabled()) {
specs.add(
new AggregationSpec(
TimeSeriesAggregationBuilder.NAME,
TimeSeriesAggregationBuilder::new,
TimeSeriesAggregationBuilder.PARSER
).addResultReader(InternalTimeSeries::new)
);
}
return List.copyOf(specs);
}

@Override
Expand Down
Loading