Skip to content

Commit

Permalink
[Transform] Optimize composite agg execution using ordered groupings (#…
Browse files Browse the repository at this point in the history
…75518)

Automatically reorder group_by for composite aggs, ensuring date histogram
group by comes first. The order is only changed for execution, the provided
config remains unchanged.

In case of 2 group_by's of the same order type, the configuration order is
respected. Script and runtime field based group_by's are penalized.
  • Loading branch information
Hendrik Muhs committed Jul 20, 2021
1 parent 957e093 commit 45d1e24
Show file tree
Hide file tree
Showing 12 changed files with 329 additions and 124 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@
package org.elasticsearch.xpack.core.transform.transforms.pivot;

import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
Expand All @@ -18,16 +17,15 @@
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.search.aggregations.MultiBucketConsumerService;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.xpack.core.transform.TransformField;
import org.elasticsearch.xpack.core.transform.utils.ExceptionsHelper;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map.Entry;
import java.util.Objects;

import static org.elasticsearch.action.ValidateActions.addValidationError;
Expand Down Expand Up @@ -111,23 +109,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
return builder;
}

public void toCompositeAggXContent(XContentBuilder builder) throws IOException {
builder.startObject();
builder.field(CompositeAggregationBuilder.SOURCES_FIELD_NAME.getPreferredName());
builder.startArray();

for (Entry<String, SingleGroupSource> groupBy : groups.getGroups().entrySet()) {
builder.startObject();
builder.startObject(groupBy.getKey());
builder.field(groupBy.getValue().getType().value(), groupBy.getValue());
builder.endObject();
builder.endObject();
}

builder.endArray();
builder.endObject(); // sources
}

@Override
public void writeTo(StreamOutput out) throws IOException {
groups.writeTo(out);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,16 +11,14 @@
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.common.xcontent.DeprecationHandler;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.composite.CompositeValuesSourceBuilder;
import org.elasticsearch.core.TimeValue;
import org.elasticsearch.xpack.core.common.validation.SourceDestValidator.RemoteClusterMinimumVersionValidation;
import org.elasticsearch.xpack.core.common.validation.SourceDestValidator.SourceDestValidation;
import org.elasticsearch.xpack.core.transform.AbstractSerializingTransformTestCase;
Expand All @@ -36,19 +34,17 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.TestMatchers.matchesPattern;
import static org.elasticsearch.xpack.core.transform.transforms.DestConfigTests.randomDestConfig;
import static org.elasticsearch.xpack.core.transform.transforms.SourceConfigTests.randomInvalidSourceConfig;
import static org.elasticsearch.xpack.core.transform.transforms.SourceConfigTests.randomSourceConfig;
import static org.hamcrest.Matchers.contains;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.contains;

public class TransformConfigTests extends AbstractSerializingTransformTestCase<TransformConfig> {

Expand Down Expand Up @@ -598,7 +594,9 @@ public void testGetAdditionalSourceDestValidations_WithRuntimeMappings() throws

public void testGroupByStayInOrder() throws IOException {
String json = "{"
+ " \"id\" : \"" + transformId +"\","
+ " \"id\" : \""
+ transformId
+ "\","
+ " \"source\" : {"
+ " \"index\":\"src\""
+ "},"
Expand All @@ -625,20 +623,12 @@ public void testGroupByStayInOrder() throws IOException {
+ "} } } } }";
TransformConfig transformConfig = createTransformConfigFromString(json, transformId, true);
List<String> originalGroups = new ArrayList<>(transformConfig.getPivotConfig().getGroupConfig().getGroups().keySet());
assertThat(
originalGroups,
contains("time", "alert", "id")
);
assertThat(originalGroups, contains("time", "alert", "id"));
for (int runs = 0; runs < NUMBER_OF_TEST_RUNS; runs++) {
// Wire serialization order guarantees
TransformConfig serialized = this.copyInstance(transformConfig);
List<String> serializedGroups = new ArrayList<>(serialized.getPivotConfig().getGroupConfig().getGroups().keySet());
assertThat(serializedGroups, equalTo(originalGroups));
CompositeAggregationBuilder compositeAggregationBuilder = createCompositeAggregationSources(serialized.getPivotConfig());
assertThat(
compositeAggregationBuilder.sources().stream().map(CompositeValuesSourceBuilder::name).collect(Collectors.toList()),
equalTo(originalGroups)
);

// Now test xcontent serialization and parsing on wire serialized object
XContentType xContentType = randomFrom(XContentType.values());
Expand All @@ -647,11 +637,6 @@ public void testGroupByStayInOrder() throws IOException {
TransformConfig parsed = doParseInstance(parser);
List<String> parsedGroups = new ArrayList<>(parsed.getPivotConfig().getGroupConfig().getGroups().keySet());
assertThat(parsedGroups, equalTo(originalGroups));
compositeAggregationBuilder = createCompositeAggregationSources(parsed.getPivotConfig());
assertThat(
compositeAggregationBuilder.sources().stream().map(CompositeValuesSourceBuilder::name).collect(Collectors.toList()),
equalTo(originalGroups)
);
}
}

Expand All @@ -665,21 +650,4 @@ private TransformConfig createTransformConfigFromString(String json, String id,
return TransformConfig.fromXContent(parser, id, lenient);
}

private CompositeAggregationBuilder createCompositeAggregationSources(PivotConfig config) throws IOException {
CompositeAggregationBuilder compositeAggregation;

try (XContentBuilder builder = jsonBuilder()) {
config.toCompositeAggXContent(builder);
XContentParser parser = builder.generator()
.contentType()
.xContent()
.createParser(
xContentRegistry(),
DeprecationHandler.THROW_UNSUPPORTED_OPERATION,
BytesReference.bytes(builder).streamInput()
);
compositeAggregation = CompositeAggregationBuilder.PARSER.parse(parser, "composite_agg");
}
return compositeAggregation;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,20 @@ public static DateHistogramGroupSource randomDateHistogramGroupSource() {
return randomDateHistogramGroupSource(Version.CURRENT);
}

public static DateHistogramGroupSource randomDateHistogramGroupSourceNoScript() {
return randomDateHistogramGroupSource(Version.CURRENT, false);
}

public static DateHistogramGroupSource randomDateHistogramGroupSource(Version version) {
return randomDateHistogramGroupSource(version, randomBoolean());
}

public static DateHistogramGroupSource randomDateHistogramGroupSource(Version version, boolean withScript) {
ScriptConfig scriptConfig = null;
String field;

// either a field or a script must be specified, it's possible to have both, but disallowed to have none
if (version.onOrAfter(Version.V_7_7_0) && randomBoolean()) {
if (version.onOrAfter(Version.V_7_7_0) && withScript) {
scriptConfig = ScriptConfigTests.randomScriptConfig();
field = randomBoolean() ? null : randomAlphaOfLengthBetween(1, 20);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,20 @@ public static HistogramGroupSource randomHistogramGroupSource() {
return randomHistogramGroupSource(Version.CURRENT);
}

public static HistogramGroupSource randomHistogramGroupSourceNoScript() {
return randomHistogramGroupSource(Version.CURRENT, false);
}

public static HistogramGroupSource randomHistogramGroupSource(Version version) {
return randomHistogramGroupSource(version, randomBoolean());
}

public static HistogramGroupSource randomHistogramGroupSource(Version version, boolean withScript) {
ScriptConfig scriptConfig = null;
String field;

// either a field or a script must be specified, it's possible to have both, but disallowed to have none
if (version.onOrAfter(Version.V_7_7_0) && randomBoolean()) {
if (version.onOrAfter(Version.V_7_7_0) && withScript) {
scriptConfig = ScriptConfigTests.randomScriptConfig();
field = randomBoolean() ? null : randomAlphaOfLengthBetween(1, 20);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,20 @@ public static TermsGroupSource randomTermsGroupSource() {
return randomTermsGroupSource(Version.CURRENT);
}

public static TermsGroupSource randomTermsGroupSourceNoScript() {
return randomTermsGroupSource(Version.CURRENT, false);
}

public static TermsGroupSource randomTermsGroupSource(Version version) {
return randomTermsGroupSource(Version.CURRENT, randomBoolean());
}

public static TermsGroupSource randomTermsGroupSource(Version version, boolean withScript) {
ScriptConfig scriptConfig = null;
String field;

// either a field or a script must be specified, it's possible to have both, but disallowed to have none
if (version.onOrAfter(Version.V_7_7_0) && randomBoolean()) {
if (version.onOrAfter(Version.V_7_7_0) && withScript) {
scriptConfig = ScriptConfigTests.randomScriptConfig();
field = randomBoolean() ? null : randomAlphaOfLengthBetween(1, 20);
} else {
Expand All @@ -43,11 +51,6 @@ public static TermsGroupSource randomTermsGroupSource(Version version) {
return new TermsGroupSource(field, scriptConfig, missingBucket);
}

public static TermsGroupSource randomTermsGroupSourceNoScript() {
String field = randomAlphaOfLengthBetween(1, 20);
return new TermsGroupSource(field, null, randomBoolean());
}

@Override
protected TermsGroupSource doParseInstance(XContentParser parser) throws IOException {
return TermsGroupSource.fromXContent(parser, false);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ public void assertGetProgress(int userWithMissingBuckets) throws Exception {
null
);

Pivot pivot = new Pivot(pivotConfig, new SettingsConfig(), Version.CURRENT);
Pivot pivot = new Pivot(pivotConfig, new SettingsConfig(), Version.CURRENT, Collections.emptySet());

TransformProgress progress = getProgress(pivot, getProgressQuery(pivot, config.getSource().getIndex(), null));

Expand Down Expand Up @@ -196,7 +196,7 @@ public void assertGetProgress(int userWithMissingBuckets) throws Exception {
Collections.singletonMap("every_50", new HistogramGroupSource("missing_field", null, missingBucket, 50.0))
);
pivotConfig = new PivotConfig(histgramGroupConfig, aggregationConfig, null);
pivot = new Pivot(pivotConfig, new SettingsConfig(), Version.CURRENT);
pivot = new Pivot(pivotConfig, new SettingsConfig(), Version.CURRENT, Collections.emptySet());

progress = getProgress(
pivot,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,12 @@ private FunctionFactory() {}
*/
public static Function create(TransformConfig config) {
if (config.getPivotConfig() != null) {
return new Pivot(config.getPivotConfig(), config.getSettings(), config.getVersion());
return new Pivot(
config.getPivotConfig(),
config.getSettings(),
config.getVersion(),
config.getSource().getScriptBasedRuntimeMappings().keySet()
);
} else if (config.getLatestConfig() != null) {
return new Latest(config.getLatestConfig());
} else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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; you may not use this file except in compliance with the Elastic License
* 2.0.
*/

package org.elasticsearch.xpack.transform.transforms.pivot;

import org.elasticsearch.core.Tuple;
import org.elasticsearch.xpack.core.transform.transforms.pivot.SingleGroupSource;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.stream.Collectors;

public final class GroupByOptimizer {

private GroupByOptimizer() {}

/**
* Returns an ordered collection of group by fields in order to get better performance.
*
* The decision is based on the type and whether the input field is a indexed/runtime/script field
*
* TODO: take index sorting into account
*
* @param groups group by as defined by the user
* @param runtimeFields set of runtime fields
* @return collection in order of priority
*/
static Collection<Entry<String, SingleGroupSource>> reorderGroups(Map<String, SingleGroupSource> groups, Set<String> runtimeFields) {
if (groups.size() == 1) {
return groups.entrySet();
}

List<Tuple<Entry<String, SingleGroupSource>, Integer>> prioritizedGroups = new ArrayList<>(groups.size());

// respect the order in the configuration by giving every entry a base priority
int basePriority = groups.size();

for (Entry<String, SingleGroupSource> groupBy : groups.entrySet()) {
// prefer indexed fields over runtime fields over scripts
int priority = basePriority-- + (groupBy.getValue().getScriptConfig() == null
? runtimeFields.contains(groupBy.getValue().getField()) ? 250 : 500
: 0);

switch (groupBy.getValue().getType()) {
case DATE_HISTOGRAM:
priority += 4000;
break;
case HISTOGRAM:
priority += 3000;
break;
case TERMS:
priority += 2000;
break;
case GEOTILE_GRID:
priority += 1000;
break;
default:
assert false : "new group source type misses priority definition";
}

prioritizedGroups.add(new Tuple<>(groupBy, priority));
}

prioritizedGroups.sort(Comparator.comparing(Tuple<Entry<String, SingleGroupSource>, Integer>::v2).reversed());

return prioritizedGroups.stream().map(x -> x.v1()).collect(Collectors.toList());
}
}
Loading

0 comments on commit 45d1e24

Please sign in to comment.