Skip to content

Commit 3ece828

Browse files
authored
Change internal representation of bucket key of time_series agg (#91407)
Currently, the key is a map, which can make reducing large responses more memory intense then it should be also. Also the map used during the reduce to detect duplicate buckets is not taken into account by circuit breaker. This map can become very large when reducing large shard level responses. This commit changes how the key is represented internally. By using BytesRef instead of Map. This commit doesn't change how the key is represented in the response. The reduce is also changed to merge the shard responses without creating intermediate data structures for detected duplicated buckets. This is possible because the buckets in the shard level responses are sorted by tsid. Relates to #74660
1 parent 9d0b0ba commit 3ece828

File tree

3 files changed

+151
-42
lines changed

3 files changed

+151
-42
lines changed

modules/aggregations/src/main/java/org/elasticsearch/aggregations/bucket/timeseries/InternalTimeSeries.java

+62-32
Original file line numberDiff line numberDiff line change
@@ -8,12 +8,16 @@
88

99
package org.elasticsearch.aggregations.bucket.timeseries;
1010

11+
import org.apache.lucene.util.BytesRef;
12+
import org.apache.lucene.util.PriorityQueue;
1113
import org.elasticsearch.common.io.stream.StreamInput;
1214
import org.elasticsearch.common.io.stream.StreamOutput;
15+
import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
1316
import org.elasticsearch.search.aggregations.AggregationReduceContext;
1417
import org.elasticsearch.search.aggregations.InternalAggregation;
1518
import org.elasticsearch.search.aggregations.InternalAggregations;
1619
import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation;
20+
import org.elasticsearch.search.aggregations.bucket.IteratorAndCurrent;
1721
import org.elasticsearch.xcontent.ObjectParser;
1822
import org.elasticsearch.xcontent.XContentBuilder;
1923

@@ -47,11 +51,12 @@ public class InternalTimeSeries extends InternalMultiBucketAggregation<InternalT
4751
public static class InternalBucket extends InternalMultiBucketAggregation.InternalBucket {
4852
protected long bucketOrd;
4953
protected final boolean keyed;
50-
protected final Map<String, Object> key;
54+
protected final BytesRef key;
55+
// TODO: make computing docCount optional
5156
protected long docCount;
5257
protected InternalAggregations aggregations;
5358

54-
public InternalBucket(Map<String, Object> key, long docCount, InternalAggregations aggregations, boolean keyed) {
59+
public InternalBucket(BytesRef key, long docCount, InternalAggregations aggregations, boolean keyed) {
5560
this.key = key;
5661
this.docCount = docCount;
5762
this.aggregations = aggregations;
@@ -63,26 +68,26 @@ public InternalBucket(Map<String, Object> key, long docCount, InternalAggregatio
6368
*/
6469
public InternalBucket(StreamInput in, boolean keyed) throws IOException {
6570
this.keyed = keyed;
66-
key = in.readOrderedMap(StreamInput::readString, StreamInput::readGenericValue);
71+
key = in.readBytesRef();
6772
docCount = in.readVLong();
6873
aggregations = InternalAggregations.readFrom(in);
6974
}
7075

7176
@Override
7277
public void writeTo(StreamOutput out) throws IOException {
73-
out.writeMap(key, StreamOutput::writeString, StreamOutput::writeGenericValue);
78+
out.writeBytesRef(key);
7479
out.writeVLong(docCount);
7580
aggregations.writeTo(out);
7681
}
7782

7883
@Override
7984
public Map<String, Object> getKey() {
80-
return key;
85+
return TimeSeriesIdFieldMapper.decodeTsid(key);
8186
}
8287

8388
@Override
8489
public String getKeyAsString() {
85-
return key.toString();
90+
return getKey().toString();
8691
}
8792

8893
@Override
@@ -97,8 +102,10 @@ public InternalAggregations getAggregations() {
97102

98103
@Override
99104
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
105+
// Use map key in the xcontent response:
106+
var key = getKey();
100107
if (keyed) {
101-
builder.startObject(getKeyAsString());
108+
builder.startObject(key.toString());
102109
} else {
103110
builder.startObject();
104111
}
@@ -187,38 +194,61 @@ protected void doWriteTo(StreamOutput out) throws IOException {
187194

188195
@Override
189196
public InternalAggregation reduce(List<InternalAggregation> aggregations, AggregationReduceContext reduceContext) {
190-
// We still need to reduce in case we got the same time series in 2 different indices, but we should be able to optimize
191-
// that in the future
192-
Map<Map<String, Object>, List<InternalBucket>> bucketsList = null;
197+
// TODO: optimize single result case either by having a if check here and return aggregations.get(0) or
198+
// by overwriting the mustReduceOnSingleInternalAgg() method
199+
final int initialCapacity = aggregations.stream()
200+
.map(value -> (InternalTimeSeries) value)
201+
.mapToInt(value -> value.getBuckets().size())
202+
.max()
203+
.getAsInt();
204+
205+
final PriorityQueue<IteratorAndCurrent<InternalBucket>> pq = new PriorityQueue<>(aggregations.size()) {
206+
@Override
207+
protected boolean lessThan(IteratorAndCurrent<InternalBucket> a, IteratorAndCurrent<InternalBucket> b) {
208+
return a.current().key.compareTo(b.current().key) < 0;
209+
}
210+
};
193211
for (InternalAggregation aggregation : aggregations) {
194212
InternalTimeSeries timeSeries = (InternalTimeSeries) aggregation;
195-
if (bucketsList != null) {
196-
for (InternalBucket bucket : timeSeries.buckets) {
197-
bucketsList.compute(bucket.key, (map, list) -> {
198-
if (list == null) {
199-
list = new ArrayList<>();
200-
}
201-
list.add(bucket);
202-
return list;
203-
});
204-
}
205-
} else {
206-
bucketsList = new HashMap<>(timeSeries.buckets.size());
207-
for (InternalTimeSeries.InternalBucket bucket : timeSeries.buckets) {
208-
List<InternalBucket> bucketList = new ArrayList<>();
209-
bucketList.add(bucket);
210-
bucketsList.put(bucket.key, bucketList);
211-
}
213+
if (timeSeries.buckets.isEmpty() == false) {
214+
IteratorAndCurrent<InternalBucket> iterator = new IteratorAndCurrent<>(timeSeries.buckets.iterator());
215+
pq.add(iterator);
212216
}
213217
}
214218

215-
reduceContext.consumeBucketsAndMaybeBreak(bucketsList.size());
216-
InternalTimeSeries reduced = new InternalTimeSeries(name, new ArrayList<>(bucketsList.size()), keyed, getMetadata());
217-
for (Map.Entry<Map<String, Object>, List<InternalBucket>> bucketEntry : bucketsList.entrySet()) {
218-
reduced.buckets.add(reduceBucket(bucketEntry.getValue(), reduceContext));
219+
InternalTimeSeries reduced = new InternalTimeSeries(name, new ArrayList<>(initialCapacity), keyed, getMetadata());
220+
List<InternalBucket> bucketsWithSameKey = new ArrayList<>(aggregations.size());
221+
BytesRef prevTsid = null;
222+
while (pq.size() > 0) {
223+
reduceContext.consumeBucketsAndMaybeBreak(1);
224+
bucketsWithSameKey.clear();
225+
226+
while (bucketsWithSameKey.isEmpty() || bucketsWithSameKey.get(0).key.equals(pq.top().current().key)) {
227+
IteratorAndCurrent<InternalBucket> iterator = pq.top();
228+
bucketsWithSameKey.add(iterator.current());
229+
if (iterator.hasNext()) {
230+
iterator.next();
231+
pq.updateTop();
232+
} else {
233+
pq.pop();
234+
if (pq.size() == 0) {
235+
break;
236+
}
237+
}
238+
}
239+
240+
InternalBucket reducedBucket;
241+
if (bucketsWithSameKey.size() == 1) {
242+
reducedBucket = bucketsWithSameKey.get(0);
243+
} else {
244+
reducedBucket = reduceBucket(bucketsWithSameKey, reduceContext);
245+
}
246+
BytesRef tsid = reducedBucket.key;
247+
assert prevTsid == null || tsid.compareTo(prevTsid) > 0;
248+
reduced.buckets.add(reducedBucket);
249+
prevTsid = tsid;
219250
}
220251
return reduced;
221-
222252
}
223253

224254
@Override

modules/aggregations/src/main/java/org/elasticsearch/aggregations/bucket/timeseries/TimeSeriesAggregator.java

+6-5
Original file line numberDiff line numberDiff line change
@@ -10,7 +10,6 @@
1010

1111
import org.apache.lucene.util.BytesRef;
1212
import org.elasticsearch.core.Releasables;
13-
import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
1413
import org.elasticsearch.search.aggregations.AggregationExecutionContext;
1514
import org.elasticsearch.search.aggregations.Aggregator;
1615
import org.elasticsearch.search.aggregations.AggregatorFactories;
@@ -32,7 +31,6 @@ public class TimeSeriesAggregator extends BucketsAggregator {
3231
protected final BytesKeyedBucketOrds bucketOrds;
3332
private final boolean keyed;
3433

35-
@SuppressWarnings("unchecked")
3634
public TimeSeriesAggregator(
3735
String name,
3836
AggregatorFactories factories,
@@ -49,16 +47,19 @@ public TimeSeriesAggregator(
4947

5048
@Override
5149
public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException {
50+
BytesRef spare = new BytesRef();
5251
InternalTimeSeries.InternalBucket[][] allBucketsPerOrd = new InternalTimeSeries.InternalBucket[owningBucketOrds.length][];
5352
for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) {
54-
BytesRef spareKey = new BytesRef();
5553
BytesKeyedBucketOrds.BucketOrdsEnum ordsEnum = bucketOrds.ordsEnum(owningBucketOrds[ordIdx]);
5654
List<InternalTimeSeries.InternalBucket> buckets = new ArrayList<>();
55+
BytesRef prev = null;
5756
while (ordsEnum.next()) {
5857
long docCount = bucketDocCount(ordsEnum.ord());
59-
ordsEnum.readValue(spareKey);
58+
ordsEnum.readValue(spare);
59+
assert prev == null || spare.compareTo(prev) > 0
60+
: "key [" + spare.utf8ToString() + "] is smaller than previous key [" + prev.utf8ToString() + "]";
6061
InternalTimeSeries.InternalBucket bucket = new InternalTimeSeries.InternalBucket(
61-
TimeSeriesIdFieldMapper.decodeTsid(spareKey),
62+
prev = BytesRef.deepCopyOf(spare), // Closing bucketOrds will corrupt the bytes ref, so need to make a deep copy here.
6263
docCount,
6364
null,
6465
keyed

modules/aggregations/src/test/java/org/elasticsearch/aggregations/bucket/timeseries/InternalTimeSeriesTests.java

+83-5
Original file line numberDiff line numberDiff line change
@@ -8,19 +8,32 @@
88

99
package org.elasticsearch.aggregations.bucket.timeseries;
1010

11+
import org.apache.lucene.util.BytesRef;
1112
import org.elasticsearch.aggregations.bucket.AggregationMultiBucketAggregationTestCase;
13+
import org.elasticsearch.aggregations.bucket.timeseries.InternalTimeSeries.InternalBucket;
14+
import org.elasticsearch.common.settings.Settings;
15+
import org.elasticsearch.common.util.MockBigArrays;
16+
import org.elasticsearch.common.util.MockPageCacheRecycler;
17+
import org.elasticsearch.index.mapper.TimeSeriesIdFieldMapper;
18+
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
1219
import org.elasticsearch.search.aggregations.Aggregation;
20+
import org.elasticsearch.search.aggregations.AggregationReduceContext;
1321
import org.elasticsearch.search.aggregations.InternalAggregations;
22+
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
1423
import org.elasticsearch.xcontent.ContextParser;
1524

25+
import java.io.IOException;
26+
import java.io.UncheckedIOException;
1627
import java.util.ArrayList;
28+
import java.util.Comparator;
1729
import java.util.HashMap;
1830
import java.util.List;
1931
import java.util.Map;
2032
import java.util.TreeMap;
2133
import java.util.function.Predicate;
2234

2335
import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
36+
import static org.hamcrest.Matchers.equalTo;
2437

2538
public class InternalTimeSeriesTests extends AggregationMultiBucketAggregationTestCase<InternalTimeSeries> {
2639

@@ -29,14 +42,25 @@ protected Map.Entry<String, ContextParser<Object, Aggregation>> getParser() {
2942
return Map.entry(TimeSeriesAggregationBuilder.NAME, (p, c) -> ParsedTimeSeries.fromXContent(p, (String) c));
3043
}
3144

32-
private List<InternalTimeSeries.InternalBucket> randomBuckets(boolean keyed, InternalAggregations aggregations) {
45+
private List<InternalBucket> randomBuckets(boolean keyed, InternalAggregations aggregations) {
3346
int numberOfBuckets = randomNumberOfBuckets();
34-
List<InternalTimeSeries.InternalBucket> bucketList = new ArrayList<>(numberOfBuckets);
47+
List<InternalBucket> bucketList = new ArrayList<>(numberOfBuckets);
3548
List<Map<String, Object>> keys = randomKeys(bucketKeys(randomIntBetween(1, 4)), numberOfBuckets);
3649
for (int j = 0; j < numberOfBuckets; j++) {
3750
long docCount = randomLongBetween(0, Long.MAX_VALUE / (20L * numberOfBuckets));
38-
bucketList.add(new InternalTimeSeries.InternalBucket(keys.get(j), docCount, aggregations, keyed));
51+
var builder = new TimeSeriesIdFieldMapper.TimeSeriesIdBuilder(null);
52+
for (var entry : keys.get(j).entrySet()) {
53+
builder.addString(entry.getKey(), (String) entry.getValue());
54+
}
55+
try {
56+
var key = builder.build().toBytesRef();
57+
bucketList.add(new InternalBucket(key, docCount, aggregations, keyed));
58+
} catch (IOException e) {
59+
throw new UncheckedIOException(e);
60+
}
3961
}
62+
// The interal time series' reduce method expects for each shard level response that the buckets are sorted by tsid:
63+
bucketList.sort(Comparator.comparing(o -> o.key));
4064
return bucketList;
4165
}
4266

@@ -68,7 +92,7 @@ protected InternalTimeSeries createTestInstance(String name, Map<String, Object>
6892
protected void assertReduced(InternalTimeSeries reduced, List<InternalTimeSeries> inputs) {
6993
Map<Map<String, Object>, Long> keys = new HashMap<>();
7094
for (InternalTimeSeries in : inputs) {
71-
for (InternalTimeSeries.InternalBucket bucket : in.getBuckets()) {
95+
for (InternalBucket bucket : in.getBuckets()) {
7296
keys.compute(bucket.getKey(), (k, v) -> {
7397
if (v == null) {
7498
return bucket.docCount;
@@ -79,7 +103,7 @@ protected void assertReduced(InternalTimeSeries reduced, List<InternalTimeSeries
79103
}
80104
}
81105
assertThat(
82-
reduced.getBuckets().stream().map(InternalTimeSeries.InternalBucket::getKey).toArray(Object[]::new),
106+
reduced.getBuckets().stream().map(InternalBucket::getKey).toArray(Object[]::new),
83107
arrayContainingInAnyOrder(keys.keySet().toArray(Object[]::new))
84108
);
85109
}
@@ -93,4 +117,58 @@ protected Class<ParsedTimeSeries> implementationClass() {
93117
protected Predicate<String> excludePathsFromXContentInsertion() {
94118
return s -> s.endsWith(".key");
95119
}
120+
121+
public void testReduceSimple() {
122+
// a simple test, to easily spot easy mistakes in the merge logic in InternalTimeSeries#reduce(...) method.
123+
InternalTimeSeries first = new InternalTimeSeries(
124+
"ts",
125+
List.of(
126+
new InternalBucket(new BytesRef("1"), 3, InternalAggregations.EMPTY, false),
127+
new InternalBucket(new BytesRef("10"), 6, InternalAggregations.EMPTY, false),
128+
new InternalBucket(new BytesRef("2"), 2, InternalAggregations.EMPTY, false),
129+
new InternalBucket(new BytesRef("9"), 5, InternalAggregations.EMPTY, false)
130+
),
131+
false,
132+
Map.of()
133+
);
134+
InternalTimeSeries second = new InternalTimeSeries(
135+
"ts",
136+
List.of(
137+
new InternalBucket(new BytesRef("2"), 1, InternalAggregations.EMPTY, false),
138+
new InternalBucket(new BytesRef("3"), 3, InternalAggregations.EMPTY, false)
139+
),
140+
false,
141+
Map.of()
142+
);
143+
InternalTimeSeries third = new InternalTimeSeries(
144+
"ts",
145+
List.of(
146+
new InternalBucket(new BytesRef("1"), 2, InternalAggregations.EMPTY, false),
147+
new InternalBucket(new BytesRef("3"), 4, InternalAggregations.EMPTY, false),
148+
new InternalBucket(new BytesRef("9"), 4, InternalAggregations.EMPTY, false)
149+
),
150+
false,
151+
Map.of()
152+
);
153+
AggregationReduceContext context = new AggregationReduceContext.ForFinal(
154+
new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()),
155+
mockScriptService(),
156+
() -> false,
157+
new TimeSeriesAggregationBuilder("ts"),
158+
value -> {},
159+
PipelineAggregator.PipelineTree.EMPTY
160+
);
161+
162+
InternalTimeSeries result = (InternalTimeSeries) first.reduce(List.of(first, second, third), context);
163+
assertThat(result.getBuckets().get(0).key.utf8ToString(), equalTo("1"));
164+
assertThat(result.getBuckets().get(0).getDocCount(), equalTo(5L));
165+
assertThat(result.getBuckets().get(1).key.utf8ToString(), equalTo("10"));
166+
assertThat(result.getBuckets().get(1).getDocCount(), equalTo(6L));
167+
assertThat(result.getBuckets().get(2).key.utf8ToString(), equalTo("2"));
168+
assertThat(result.getBuckets().get(2).getDocCount(), equalTo(3L));
169+
assertThat(result.getBuckets().get(3).key.utf8ToString(), equalTo("3"));
170+
assertThat(result.getBuckets().get(3).getDocCount(), equalTo(7L));
171+
assertThat(result.getBuckets().get(4).key.utf8ToString(), equalTo("9"));
172+
assertThat(result.getBuckets().get(4).getDocCount(), equalTo(9L));
173+
}
96174
}

0 commit comments

Comments
 (0)