-
Notifications
You must be signed in to change notification settings - Fork 3
Commit
Typically time_series aggregation is wrapped by a date histogram aggregation. This commit explores idea around making things more efficient for time series agg if this is the case. This commit explores two main ideas: * With time series index searcher docs are emitted in tsid and timestamp order. Because of this within docs of the tsid, the date histogram buckets are also emitted in order to sub aggs. This allows time series aggregator to only keep track of the bucket belonging to the current tsid and bucket ordinal. The removes the need for using BytesKeyedBucketOrds, which in production is very heavy. Also given the fact the tsid is a high cardinality field. For each tsid and buck ordinal combination we keep track of doc count and delegate to sub agg. When the tsid / bucket ordinal combination changes the time series agg on the fly creates a new bucket. Sub aggs of time series agg, only ever contain buckets for a single parent bucket ordinal, this allows to always use a bucket ordinal of value 0. After each bucket has been created the sub agg is cleared. * If the bucket that date histogram creates are contained with the index boundaries of the backing index the shard the search is executed belongs to, then reduction/pipeline aggregation can happen locally only the fly when the time series buckets are created. In order to support this a TimestampBoundsAware interface was added. That can tell a sub agg of a date histogram whether the bounds of parent bucket are within the bounds of the backing index. In this experiment the terms aggregator was hard coded to use min bucket pipeline agg, which gets fed a time series bucket (with sub agg buckets) each time tsid / bucket ordinal combo changes. If buckets are outside backing index boundary then buckets are kept around and pipeline agg is executed in reduce method of InternalTimeSeries response class. This fundamentally changes the time series agg, since the response depends on the pipeline agg used. The `TimeSeriesAggregator3` contains both of these changes. Extra notes: * Date histogram could use `AggregationExecutionContext#getTimestamp()` as source for rounding values into buckets. * I think there is no need for doc count if pipeline aggs reduce on the fly the buckets created by time series agg. * Date agg's filter by filter optimization has been disabled when agg requires in order execution. The time series index searcher doesn't work with filter by filter optimization. Relates to elastic#74660
- Loading branch information
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,7 +17,10 @@ | |
import org.elasticsearch.search.aggregations.InternalAggregation; | ||
import org.elasticsearch.search.aggregations.InternalAggregations; | ||
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; | ||
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; | ||
import org.elasticsearch.search.aggregations.bucket.IteratorAndCurrent; | ||
import org.elasticsearch.search.aggregations.pipeline.BucketMetricsPipelineAggregationBuilder; | ||
import org.elasticsearch.search.aggregations.pipeline.BucketMetricsPipelineAggregator; | ||
import org.elasticsearch.xcontent.ObjectParser; | ||
import org.elasticsearch.xcontent.XContentBuilder; | ||
|
||
|
@@ -138,13 +141,28 @@ public int hashCode() { | |
} | ||
|
||
private final List<InternalTimeSeries.InternalBucket> buckets; | ||
private final BucketMetricsPipelineAggregationBuilder<?> pipelineAggregationBuilder; | ||
private final boolean keyed; | ||
// bucketMap gets lazily initialized from buckets in getBucketByKey() | ||
private transient Map<String, InternalTimeSeries.InternalBucket> bucketMap; | ||
|
||
public InternalTimeSeries(String name, List<InternalTimeSeries.InternalBucket> buckets, boolean keyed, Map<String, Object> metadata) { | ||
super(name, metadata); | ||
this.buckets = buckets; | ||
this.pipelineAggregationBuilder = null; | ||
this.buckets = Objects.requireNonNull(buckets); | ||
this.keyed = keyed; | ||
} | ||
|
||
public InternalTimeSeries( | ||
String name, | ||
BucketMetricsPipelineAggregationBuilder<?> pipelineAggregationBuilder, | ||
List<InternalTimeSeries.InternalBucket> buckets, | ||
boolean keyed, | ||
Map<String, Object> metadata | ||
) { | ||
super(name, metadata); | ||
this.pipelineAggregationBuilder = pipelineAggregationBuilder; | ||
this.buckets = Objects.requireNonNull(buckets); | ||
this.keyed = keyed; | ||
} | ||
|
||
|
@@ -153,6 +171,9 @@ public InternalTimeSeries(String name, List<InternalTimeSeries.InternalBucket> b | |
*/ | ||
public InternalTimeSeries(StreamInput in) throws IOException { | ||
super(in); | ||
pipelineAggregationBuilder = (BucketMetricsPipelineAggregationBuilder<?>) in.readOptionalNamedWriteable( | ||
PipelineAggregationBuilder.class | ||
); | ||
keyed = in.readBoolean(); | ||
int size = in.readVInt(); | ||
List<InternalTimeSeries.InternalBucket> buckets = new ArrayList<>(size); | ||
|
@@ -194,6 +215,17 @@ protected void doWriteTo(StreamOutput out) throws IOException { | |
|
||
@Override | ||
public InternalAggregation reduce(List<InternalAggregation> aggregations, AggregationReduceContext reduceContext) { | ||
if (pipelineAggregationBuilder != null) { | ||
// This agg response class is shared now between 3 versions of time series aggregator, | ||
// but if TimeSeriesAggregator3 was the only implementation the reduce method would only do what happens in the if block. | ||
BucketMetricsPipelineAggregator pipelineAggregator = (BucketMetricsPipelineAggregator) pipelineAggregationBuilder.create(); | ||
pipelineAggregator.start(); | ||
for (InternalBucket bucket : buckets) { | ||
pipelineAggregator.collect(this, bucket); | ||
} | ||
return pipelineAggregator.end(); | ||
} | ||
This comment has been minimized.
Sorry, something went wrong.
This comment has been minimized.
Sorry, something went wrong.
martijnvg
Author
Owner
|
||
|
||
// TODO: optimize single result case either by having a if check here and return aggregations.get(0) or | ||
// by overwriting the mustReduceOnSingleInternalAgg() method | ||
final int initialCapacity = aggregations.stream() | ||
|
@@ -244,7 +276,8 @@ protected boolean lessThan(IteratorAndCurrent<InternalBucket> a, IteratorAndCurr | |
reducedBucket = reduceBucket(bucketsWithSameKey, reduceContext); | ||
} | ||
BytesRef tsid = reducedBucket.key; | ||
assert prevTsid == null || tsid.compareTo(prevTsid) > 0; | ||
assert prevTsid == null || tsid.compareTo(prevTsid) > 0 | ||
: "prevTsid=" + TimeSeriesIdFieldMapper.decodeTsid(prevTsid) + " tsid=" + TimeSeriesIdFieldMapper.decodeTsid(tsid); | ||
reduced.buckets.add(reducedBucket); | ||
prevTsid = tsid; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -92,15 +92,29 @@ protected void doClose() { | |
protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException { | ||
return new LeafBucketCollectorBase(sub, null) { | ||
|
||
// This helps significantly reducing time spent attempting to add bucket + tsid combos that already were added. | ||
long currentTsidOrd = -1; | ||
long currentBucket = -1; | ||
long currentBucketOrdinal; | ||
|
||
@Override | ||
public void collect(int doc, long bucket) throws IOException { | ||
if (currentBucket == bucket && currentTsidOrd == aggCtx.getTsidOrd()) { | ||
collectExistingBucket(sub, doc, currentBucketOrdinal); | ||
return; | ||
} | ||
|
||
long bucketOrdinal = bucketOrds.add(bucket, aggCtx.getTsid()); | ||
if (bucketOrdinal < 0) { // already seen | ||
bucketOrdinal = -1 - bucketOrdinal; | ||
collectExistingBucket(sub, doc, bucketOrdinal); | ||
} else { | ||
collectBucket(sub, doc, bucketOrdinal); | ||
} | ||
|
||
currentBucketOrdinal = bucketOrdinal; | ||
currentTsidOrd = aggCtx.getTsidOrd(); | ||
currentBucket = bucket; | ||
} | ||
This comment has been minimized.
Sorry, something went wrong.
nik9000
|
||
}; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,127 @@ | ||
/* | ||
* 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.timeseries; | ||
|
||
import org.apache.lucene.util.BytesRef; | ||
import org.elasticsearch.index.TimestampBounds; | ||
import org.elasticsearch.search.aggregations.AggregationExecutionContext; | ||
import org.elasticsearch.search.aggregations.Aggregator; | ||
import org.elasticsearch.search.aggregations.AggregatorFactories; | ||
import org.elasticsearch.search.aggregations.CardinalityUpperBound; | ||
import org.elasticsearch.search.aggregations.InternalAggregation; | ||
import org.elasticsearch.search.aggregations.LeafBucketCollector; | ||
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; | ||
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; | ||
import org.elasticsearch.search.aggregations.bucket.histogram.TimestampBoundsAware; | ||
import org.elasticsearch.search.aggregations.support.AggregationContext; | ||
|
||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
public class TimeSeriesAggregator2 extends BucketsAggregator { | ||
|
||
private final TimestampBounds timestampBounds; | ||
private final TimestampBoundsAware parent; | ||
private final Map<Long, List<InternalTimeSeries.InternalBucket>> results; | ||
private final boolean keyed; | ||
|
||
public TimeSeriesAggregator2( | ||
String name, | ||
AggregatorFactories factories, | ||
boolean keyed, | ||
AggregationContext context, | ||
Aggregator parent, | ||
Map<String, Object> metadata | ||
) throws IOException { | ||
super(name, factories, context, parent, CardinalityUpperBound.ONE, metadata); | ||
this.keyed = keyed; | ||
this.timestampBounds = context.getIndexSettings().getTimestampBounds(); | ||
this.parent = (TimestampBoundsAware) parent; | ||
this.results = new HashMap<>(); | ||
} | ||
|
||
@Override | ||
public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException { | ||
// figure out running pipeline aggs here | ||
// context.pipelineTreeRoot().subTree(agg.getName()) | ||
completeBucket(); | ||
InternalAggregation[] result = new InternalAggregation[owningBucketOrds.length]; | ||
for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) { | ||
long owningOrdinal = owningBucketOrds[ordIdx]; | ||
List<InternalTimeSeries.InternalBucket> buckets = results.get(owningOrdinal); | ||
if (buckets == null) { | ||
continue; | ||
} | ||
result[ordIdx] = new InternalTimeSeries(name, buckets, keyed, metadata()); | ||
} | ||
return result; | ||
} | ||
|
||
@Override | ||
public InternalAggregation buildEmptyAggregation() { | ||
return new InternalTimeSeries(name, new ArrayList<>(), false, metadata()); | ||
} | ||
|
||
BytesRef currentTsid; | ||
int currentTsidOrd = -1; | ||
long currentParentBucket; | ||
long docCount; | ||
|
||
@Override | ||
protected LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, LeafBucketCollector sub) throws IOException { | ||
return new LeafBucketCollectorBase(sub, null) { | ||
|
||
@Override | ||
public void collect(int doc, long bucket) throws IOException { | ||
// System.out.println("bucketId=" + bucket); | ||
// System.out.println("tsid=" + TimeSeriesIdFieldMapper.decodeTsid(aggCtx.getTsid())); | ||
|
||
if (currentTsidOrd == aggCtx.getTsidOrd() && currentParentBucket == bucket) { | ||
docCount++; | ||
sub.collect(doc, 0L); | ||
return; | ||
} | ||
if (currentTsid != null) { | ||
completeBucket(); | ||
} | ||
if (currentTsidOrd != aggCtx.getTsidOrd()) { | ||
currentTsidOrd = aggCtx.getTsidOrd(); | ||
currentTsid = BytesRef.deepCopyOf(aggCtx.getTsid()); | ||
} | ||
if (currentParentBucket != bucket) { | ||
currentParentBucket = bucket; | ||
} | ||
|
||
sub.clear(); | ||
docCount = 1; | ||
sub.collect(doc, 0L); | ||
} | ||
}; | ||
} | ||
|
||
private void completeBucket() throws IOException { | ||
InternalTimeSeries.InternalBucket bucket = new InternalTimeSeries.InternalBucket( | ||
currentTsid, | ||
docCount, | ||
buildSubAggsForBuckets(new long[] { 0L })[0], | ||
keyed | ||
); | ||
// System.out.println("complete bucket=" + currentParentBucket); | ||
List<InternalTimeSeries.InternalBucket> result = results.get(currentParentBucket); | ||
if (result == null) { | ||
result = new ArrayList<>(); | ||
results.put(currentParentBucket, result); | ||
} | ||
result.add(bucket); | ||
} | ||
|
||
} |
reduceContext.builder
should contain your builder. The tests aren't super careful about that, but in production you have your builder. And you can take the pipeline builder out of that.