|
|
@@ -18,7 +18,6 @@ import org.elasticsearch.client.internal.Client;
|
|
|
import org.elasticsearch.client.internal.ParentTaskAssigningClient;
|
|
|
import org.elasticsearch.client.internal.node.NodeClient;
|
|
|
import org.elasticsearch.common.inject.Inject;
|
|
|
-import org.elasticsearch.common.util.Maps;
|
|
|
import org.elasticsearch.index.IndexNotFoundException;
|
|
|
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
|
|
|
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
|
|
|
@@ -30,10 +29,11 @@ import org.elasticsearch.xcontent.ObjectPath;
|
|
|
|
|
|
import java.util.Arrays;
|
|
|
import java.util.HashMap;
|
|
|
-import java.util.HashSet;
|
|
|
import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
+import java.util.TreeMap;
|
|
|
+import java.util.TreeSet;
|
|
|
|
|
|
public class TransportGetProfilingAction extends HandledTransportAction<GetProfilingRequest, GetProfilingResponse> {
|
|
|
private static final Logger log = LogManager.getLogger(TransportGetProfilingAction.class);
|
|
|
@@ -108,7 +108,10 @@ public class TransportGetProfilingAction extends HandledTransportAction<GetProfi
|
|
|
long totalCount = Math.round(totalCountAgg.value());
|
|
|
Resampler resampler = new Resampler(request, eventsIndex.getSampleRate(), totalCount);
|
|
|
StringTerms stacktraces = searchResponse.getAggregations().get("group_by");
|
|
|
- Map<String, Integer> stackTraceEvents = Maps.newHashMapWithExpectedSize(stacktraces.getBuckets().size());
|
|
|
+ // sort items lexicographically to access Lucene's term dictionary more efficiently when issuing an mget request.
|
|
|
+ // The term dictionary is lexicographically sorted and using the same order reduces the number of page faults
|
|
|
+ // needed to load it.
|
|
|
+ Map<String, Integer> stackTraceEvents = new TreeMap<>();
|
|
|
for (StringTerms.Bucket bucket : stacktraces.getBuckets()) {
|
|
|
Sum count = bucket.getAggregations().get("count");
|
|
|
int finalCount = resampler.adjustSampleCount((int) count.value());
|
|
|
@@ -143,8 +146,11 @@ public class TransportGetProfilingAction extends HandledTransportAction<GetProfi
|
|
|
@Override
|
|
|
public void onResponse(MultiGetResponse multiGetItemResponses) {
|
|
|
Map<String, StackTrace> stackTracePerId = new HashMap<>();
|
|
|
- Set<String> stackFrameIds = new HashSet<>();
|
|
|
- Set<String> executableIds = new HashSet<>();
|
|
|
+ // sort items lexicographically to access Lucene's term dictionary more efficiently when issuing an mget request.
|
|
|
+ // The term dictionary is lexicographically sorted and using the same order reduces the number of page faults
|
|
|
+ // needed to load it.
|
|
|
+ Set<String> stackFrameIds = new TreeSet<>();
|
|
|
+ Set<String> executableIds = new TreeSet<>();
|
|
|
int totalFrames = 0;
|
|
|
for (MultiGetItemResponse trace : multiGetItemResponses) {
|
|
|
if (trace.isFailed() == false && trace.getResponse().isExists()) {
|