From db64aceda644b041111cd898f7d7e746c3527ac1 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 2 Mar 2020 10:34:30 -0800
Subject: [PATCH 01/23] insilico
---
.../support/MultiValuesSource.java | 36 +++++
.../GeoLineAggregationBuilder.java | 106 ++++++++++++
.../aggregations/GeoLineAggregator.java | 151 ++++++++++++++++++
.../GeoLineAggregatorFactory.java | 48 ++++++
.../search/aggregations/InternalGeoLine.java | 129 +++++++++++++++
.../search/aggregations/PathArraySorter.java | 47 ++++++
.../aggregations/GeoLineAggregatorTests.java | 133 +++++++++++++++
7 files changed, 650 insertions(+)
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
create mode 100644 x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
index 3d1d444c17ca1..e670813253c66 100644
--- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
+++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
@@ -19,6 +19,7 @@
package org.elasticsearch.search.aggregations.support;
import org.apache.lucene.index.LeafReaderContext;
+import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
@@ -55,6 +56,41 @@ public SortedNumericDoubleValues getField(String fieldName, LeafReaderContext ct
}
}
+ public static class AnyMultiValuesSource extends MultiValuesSource {
+ public AnyMultiValuesSource(Map> valuesSourceConfigs,
+ QueryShardContext context) {
+ values = new HashMap<>(valuesSourceConfigs.size());
+ for (Map.Entry> entry : valuesSourceConfigs.entrySet()) {
+ values.put(entry.getKey(), entry.getValue().toValuesSource(context));
+ }
+ }
+
+ private ValuesSource getField(String fieldName) {
+ ValuesSource valuesSource = values.get(fieldName);
+ if (valuesSource == null) {
+ throw new IllegalArgumentException("Could not find field name [" + fieldName + "] in multiValuesSource");
+ }
+ return valuesSource;
+ }
+
+ public SortedNumericDoubleValues getNumericField(String fieldName, LeafReaderContext ctx) throws IOException {
+ ValuesSource valuesSource = getField(fieldName);
+ if (valuesSource instanceof ValuesSource.Numeric) {
+ return ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+ }
+ throw new IllegalArgumentException("field [" + fieldName + "] is not a numeric type");
+ }
+
+ public MultiGeoPointValues getGeoPointField(String fieldName, LeafReaderContext ctx) {
+ ValuesSource valuesSource = getField(fieldName);
+ if (valuesSource instanceof ValuesSource.GeoPoint) {
+ return ((ValuesSource.GeoPoint) valuesSource).geoPointValues(ctx);
+ }
+ throw new IllegalArgumentException("field [" + fieldName + "] is not a geo_point type");
+ }
+
+ }
+
public boolean needsScores() {
return values.values().stream().anyMatch(ValuesSource::needsScores);
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
new file mode 100644
index 0000000000000..bc352eed08431
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.ObjectParser;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.index.query.QueryShardContext;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationBuilder;
+import org.elasticsearch.search.aggregations.AggregatorFactories;
+import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregationBuilder;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceParseHelper;
+import org.elasticsearch.search.aggregations.support.ValueType;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+public class GeoLineAggregationBuilder
+ extends MultiValuesSourceAggregationBuilder {
+
+ static final ParseField GEO_POINT_FIELD = new ParseField("geo_point");
+ static final ParseField SORT_FIELD = new ParseField("sort");
+
+ static final String NAME = "geo_line";
+
+ private static final ObjectParser PARSER;
+ static {
+ PARSER = new ObjectParser<>(NAME);
+ MultiValuesSourceParseHelper.declareCommon(PARSER, true, ValueType.NUMERIC);
+ MultiValuesSourceParseHelper.declareField(GEO_POINT_FIELD.getPreferredName(), PARSER, true, false);
+ MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false);
+ }
+
+ GeoLineAggregationBuilder(String name) {
+ super(name, null);
+ }
+
+ private GeoLineAggregationBuilder(GeoLineAggregationBuilder clone,
+ AggregatorFactories.Builder factoriesBuilder, Map metaData) {
+ super(clone, factoriesBuilder, metaData);
+ }
+
+ /**
+ * Read from a stream.
+ */
+ GeoLineAggregationBuilder(StreamInput in) throws IOException {
+ super(in, null);
+ }
+
+ static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
+ return PARSER.parse(parser, new GeoLineAggregationBuilder(aggregationName), null);
+ }
+
+ @Override
+ protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBuilder, Map metaData) {
+ return new GeoLineAggregationBuilder(this, factoriesBuilder, metaData);
+ }
+
+ @Override
+ protected void innerWriteTo(StreamOutput out) {
+ // Do nothing, no extra state to write to stream
+ }
+
+ @Override
+ protected MultiValuesSourceAggregatorFactory innerBuild(QueryShardContext queryShardContext, Map> configs, DocValueFormat format, AggregatorFactory parent,
+ AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
+ return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metaData);
+ }
+
+ public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
+ valueConfig = Objects.requireNonNull(valueConfig, "Configuration for field [" + GEO_POINT_FIELD + "] cannot be null");
+ field(GEO_POINT_FIELD.getPreferredName(), valueConfig);
+ return this;
+ }
+
+ public GeoLineAggregationBuilder sort(MultiValuesSourceFieldConfig sortConfig) {
+ sortConfig = Objects.requireNonNull(sortConfig, "Configuration for field [" + SORT_FIELD + "] cannot be null");
+ field(SORT_FIELD.getPreferredName(), sortConfig);
+ return this;
+ }
+
+ @Override
+ public XContentBuilder doXContentBody(XContentBuilder builder, ToXContent.Params params) {
+ return builder;
+ }
+
+ @Override
+ public String getType() {
+ return NAME;
+ }
+}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
new file mode 100644
index 0000000000000..a50e8997cd139
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -0,0 +1,151 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.NumericUtils;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.IntArray;
+import org.elasticsearch.common.util.ObjectArray;
+import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.aggregations.AggregationExecutionException;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.LeafBucketCollector;
+import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
+import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.GEO_POINT_FIELD;
+import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
+
+/**
+ * Metric Aggregation for computing the pearson product correlation coefficient between multiple fields
+ **/
+final class GeoLineAggregator extends MetricsAggregator {
+ /** Multiple ValuesSource with field names */
+ private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+
+ private ObjectArray paths;
+ private ObjectArray sortValues;
+ private IntArray idxs;
+
+ GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
+ Aggregator parent, List pipelineAggregators,
+ Map metaData) throws IOException {
+ super(name, context, parent, pipelineAggregators, metaData);
+ this.valuesSources = valuesSources;
+ if (valuesSources != null) {
+ paths = context.bigArrays().newObjectArray(1);
+ sortValues = context.bigArrays().newObjectArray(1);
+ idxs = context.bigArrays().newIntArray(1);
+ }
+ }
+
+ @Override
+ public ScoreMode scoreMode() {
+ if (valuesSources != null && valuesSources.needsScores()) {
+ return ScoreMode.COMPLETE;
+ }
+ return super.scoreMode();
+ }
+
+ @Override
+ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
+ final LeafBucketCollector sub) throws IOException {
+ if (valuesSources == null) {
+ return LeafBucketCollector.NO_OP_COLLECTOR;
+ }
+ final BigArrays bigArrays = context.bigArrays();
+ MultiGeoPointValues docGeoPointValues = valuesSources.getGeoPointField(GEO_POINT_FIELD.getPreferredName(), ctx);
+ SortedNumericDoubleValues docSortValues = valuesSources.getNumericField(SORT_FIELD.getPreferredName(), ctx);
+
+ return new LeafBucketCollectorBase(sub, docGeoPointValues) {
+ @Override
+ public void collect(int doc, long bucket) throws IOException {
+ paths = bigArrays.grow(paths, bucket + 1);
+ if (docGeoPointValues.advanceExact(doc) && docSortValues.advanceExact(doc)) {
+ if (docSortValues.docValueCount() > 1) {
+ throw new AggregationExecutionException("Encountered more than one sort value for a " +
+ "single document. Use a script to combine multiple sort-values-per-doc into a single value.");
+ }
+ if (docGeoPointValues.docValueCount() > 1) {
+ throw new AggregationExecutionException("Encountered more than one geo_point value for a " +
+ "single document. Use a script to combine multiple geo_point-values-per-doc into a single value.");
+ }
+
+ // There should always be one weight if advanceExact lands us here, either
+ // a real weight or a `missing` weight
+ assert docSortValues.docValueCount() == 1;
+ assert docGeoPointValues.docValueCount() == 1;
+ final double sort = docSortValues.nextValue();
+ final GeoPoint point = docGeoPointValues.nextValue();
+
+ int idx = idxs.get(bucket);
+ long[] bucketLine = paths.get(bucket);
+ double[] sortVals = sortValues.get(bucket);
+ if (bucketLine == null) {
+ bucketLine = new long[10];
+ } else {
+ bucketLine = ArrayUtil.grow(bucketLine, idx + 1);
+ }
+
+
+ if (sortVals == null) {
+ sortVals = new double[10];
+ } else {
+ sortVals = ArrayUtil.grow(sortVals, idx + 1);
+ }
+
+ int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
+ int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
+ long lonLat = (((long) encodedLon) << 32) | (encodedLat & 0xffffffffL);
+
+ sortVals[idx] = sort;
+ bucketLine[idx] = lonLat;
+
+ paths.set(bucket, bucketLine);
+ sortValues.set(bucket, sortVals);
+ idxs.set(bucket, idx + 1);
+ }
+ }
+ };
+ }
+
+ @Override
+ public InternalAggregation buildAggregation(long bucket) {
+ if (valuesSources == null) {
+ return buildEmptyAggregation();
+ }
+ long[] bucketLine = paths.get(bucket);
+ double[] sortVals = sortValues.get(bucket);
+ int length = idxs.get(bucket);
+ new PathArraySorter(bucketLine, sortVals, length).sort();
+ return new InternalGeoLine(name, bucketLine, sortVals, length, pipelineAggregators(), metaData());
+ }
+
+ @Override
+ public InternalAggregation buildEmptyAggregation() {
+ return new InternalGeoLine(name, null, null, 0, pipelineAggregators(), metaData());
+ }
+
+ @Override
+ public void doClose() {
+ Releasables.close(paths, idxs, sortValues);
+ }
+}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
new file mode 100644
index 0000000000000..b934e147882c6
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.index.query.QueryShardContext;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.AggregatorFactories;
+import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory {
+
+ GeoLineAggregatorFactory(String name,
+ Map> configs,
+ DocValueFormat format, QueryShardContext queryShardContext, AggregatorFactory parent,
+ AggregatorFactories.Builder subFactoriesBuilder,
+ Map metaData) throws IOException {
+ super(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metaData);
+ }
+
+ @Override
+ protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent, List pipelineAggregators,
+ Map metaData) throws IOException {
+ return new GeoLineAggregator(name, null, searchContext, parent, pipelineAggregators, metaData);
+ }
+
+ @Override
+ protected Aggregator doCreateInternal(SearchContext searchContext, Map> configs,
+ DocValueFormat format, Aggregator parent, boolean collectsFromSingleBucket,
+ List pipelineAggregators, Map metaData) throws IOException {
+ MultiValuesSource.AnyMultiValuesSource valuesSources = new MultiValuesSource
+ .AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
+ return new GeoLineAggregator(name, valuesSources, searchContext, parent, pipelineAggregators, metaData);
+ }
+}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
new file mode 100644
index 0000000000000..b8b787335e93d
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.elasticsearch.common.Strings;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.geometry.Line;
+import org.elasticsearch.geometry.utils.WellKnownText;
+import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.lucene.util.ArrayUtil.grow;
+
+/**
+ * A single line string representing a sorted sequence of geo-points
+ */
+public class InternalGeoLine extends InternalAggregation {
+ private static final Logger logger = LogManager.getLogger(InternalGeoLine.class);
+
+ private long[] line;
+ private double[] sortVals;
+ private int length;
+
+ InternalGeoLine(String name, long[] line, double[] sortVals, int length, List pipelineAggregators, Map metaData) {
+ super(name, pipelineAggregators, metaData);
+ this.line = line;
+ this.sortVals = sortVals;
+ this.length = length;
+ }
+
+ /**
+ * Read from a stream.
+ */
+ public InternalGeoLine(StreamInput in) throws IOException {
+ super(in);
+ this.line = in.readLongArray();
+ this.length = in.readVInt();
+ }
+
+ @Override
+ protected void doWriteTo(StreamOutput out) throws IOException {
+ out.writeLongArray(line);
+ out.writeVInt(length);
+ }
+
+ @Override
+ public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) {
+ int mergedSize = 0;
+ for (InternalAggregation aggregation : aggregations) {
+ InternalGeoLine geoLine = (InternalGeoLine) aggregation;
+ mergedSize += geoLine.length;
+ }
+
+ long[] finalList = new long[mergedSize];
+ double[] finalSortVals = new double[mergedSize];
+ int idx = 0;
+ for (InternalAggregation aggregation : aggregations) {
+ InternalGeoLine geoLine = (InternalGeoLine) aggregation;
+ for (int i = 0; i < geoLine.length; i++) {
+ finalSortVals[idx] = geoLine.sortVals[i];
+ finalList[idx++] = geoLine.line[i];
+ }
+ }
+
+ new PathArraySorter(finalList, finalSortVals, length).sort();
+
+ // sort the final list
+ return new InternalGeoLine(name, finalList, finalSortVals, mergedSize, pipelineAggregators(), getMetaData());
+ }
+
+ @Override
+ public String getWriteableName() {
+ return GeoLineAggregationBuilder.NAME;
+ }
+
+ public long[] line() {
+ return line;
+ }
+
+ public int length() {
+ return length;
+ }
+
+ @Override
+ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
+ builder.field("type", "LineString");
+ final List coordinates = new ArrayList<>();
+ for (int i = 0; i < length; i++) {
+ int x = (int) line[i] >> 32;
+ int y = (int) line[i];
+ coordinates.add(new double[] { GeoEncodingUtils.decodeLongitude(x), GeoEncodingUtils.decodeLatitude(y) });
+ }
+
+ builder.array("coordinates", coordinates.toArray());
+ builder.array("sorts", sortVals);
+ return builder;
+ }
+
+ @Override
+ public String toString() {
+ return Strings.toString(this);
+ }
+
+ @Override
+ public Object getProperty(List path) {
+ logger.error("what in the world");
+ if (path.isEmpty()) {
+ return this;
+ } else if (path.size() == 1 && "value".equals(path.get(0))) {
+ return line;
+ } else {
+ throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path);
+ }
+ }
+}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
new file mode 100644
index 0000000000000..d7ed0bda2b970
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.apache.lucene.util.IntroSorter;
+
+final class PathArraySorter extends IntroSorter {
+
+ private final long[] points;
+ private final double[] sortValues;
+ private double sortValuePivot;
+ private int length;
+
+ public PathArraySorter(long[] points, double[] sortValues, int length) {
+ this.points = points;
+ this.sortValues = sortValues;
+ this.sortValuePivot = 0;
+ this.length = length;
+ }
+
+ public final void sort() {
+ sort(0, length);
+ }
+
+ @Override
+ protected void swap(int i, int j) {
+ final long tmpPoint = points[i];
+ points[i] = points[j];
+ points[j] = tmpPoint;
+ final double tmpSortValue = sortValues[i];
+ sortValues[i] = sortValues[j];
+ sortValues[j] = tmpSortValue;
+ }
+
+ @Override
+ protected void setPivot(int i) {
+ sortValuePivot = sortValues[i];
+ }
+
+ @Override
+ protected int comparePivot(int j) {
+ return Double.compare(sortValuePivot, sortValues[j]);
+ }
+}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
new file mode 100644
index 0000000000000..9db71d4400c4c
--- /dev/null
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.apache.lucene.document.LatLonDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.NumericUtils;
+import org.elasticsearch.common.CheckedConsumer;
+import org.elasticsearch.geo.GeometryTestUtils;
+import org.elasticsearch.geometry.Point;
+import org.elasticsearch.index.mapper.GeoPointFieldMapper;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
+import org.mockito.internal.matchers.ArrayEquals;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.function.Consumer;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+
+public class GeoLineAggregatorTests extends AggregatorTestCase {
+
+ public void testSomething() throws IOException {
+ MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName("value_field")
+ .build();
+ MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("sort_field").build();
+ GeoLineAggregationBuilder aggregationBuilder = new GeoLineAggregationBuilder("_name")
+ .value(valueConfig)
+ .sort(sortConfig);
+
+ int numPoints = randomIntBetween(1, 10000);
+ int arrayLength = randomIntBetween(numPoints, numPoints + 1000);
+ long[] points = new long[arrayLength];
+ double[] sortValues = new double[arrayLength];
+ for (int i = 0; i < numPoints; i++) {
+ Point point = GeometryTestUtils.randomPoint(false);
+ int encodedLat = GeoEncodingUtils.encodeLatitude(point.getLat());
+ int encodedLon = GeoEncodingUtils.encodeLongitude(point.getLon());
+ long lonLat = (((long) encodedLon) << 32) | (encodedLat & 0xffffffffL);
+ points[i] = lonLat;
+ sortValues[i] = i;
+ }
+
+ InternalGeoLine geoLine = new InternalGeoLine("_name",
+ Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength),
+ numPoints, null, null);
+
+ for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
+ int idx1 = randomIntBetween(0, numPoints);
+ int idx2 = randomIntBetween(0, numPoints);
+ final long tmpPoint = points[idx1];
+ points[idx1] = points[idx2];
+ points[idx2] = tmpPoint;
+ final double tmpSortValue = sortValues[idx1];
+ sortValues[idx1] = sortValues[idx2];
+ sortValues[idx2] = tmpSortValue;
+ }
+
+
+ testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> {
+ for (int i = 0; i < numPoints; i++) {
+ int x = (int) points[i] >> 32;
+ int y = (int) points[i];
+ iw.addDocument(Arrays.asList(new LatLonDocValuesField("value_field",
+ GeoEncodingUtils.decodeLatitude(y),
+ GeoEncodingUtils.decodeLongitude(x)),
+ new SortedNumericDocValuesField("sort_field", NumericUtils.doubleToSortableLong(sortValues[i]))));
+ }
+ }, actualGeoLine -> {
+ assertThat(actualGeoLine.length(), equalTo(geoLine.length()));
+ for (int i = 0; i < geoLine.length(); i++) {
+ assertThat(GeoEncodingUtils.decodeLongitude((int) actualGeoLine.line()[i]),
+ equalTo(GeoEncodingUtils.decodeLongitude((int) geoLine.line()[i])));
+ assertThat(GeoEncodingUtils.decodeLatitude((int) actualGeoLine.line()[i] << 32),
+ equalTo(GeoEncodingUtils.decodeLatitude((int) geoLine.line()[i] << 32)));
+ }
+ });
+ }
+
+ private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
+ CheckedConsumer buildIndex,
+ Consumer verify) throws IOException {
+ testCase(query, aggregationBuilder, buildIndex, verify, NumberFieldMapper.NumberType.LONG);
+ }
+
+ private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
+ CheckedConsumer buildIndex,
+ Consumer verify,
+ NumberFieldMapper.NumberType fieldNumberType) throws IOException {
+
+ Directory directory = newDirectory();
+ RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory);
+ buildIndex.accept(indexWriter);
+ indexWriter.close();
+ IndexReader indexReader = DirectoryReader.open(directory);
+ IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
+
+ try {
+ MappedFieldType fieldType = new GeoPointFieldMapper.GeoPointFieldType();
+ fieldType.setName("value_field");
+ fieldType.setHasDocValues(true);
+
+ MappedFieldType fieldType2 = new NumberFieldMapper.NumberFieldType(fieldNumberType);
+ fieldType2.setName("sort_field");
+ fieldType2.setHasDocValues(true);
+
+ GeoLineAggregator aggregator = createAggregator(aggregationBuilder, indexSearcher, fieldType, fieldType2);
+ aggregator.preCollection();
+ indexSearcher.search(query, aggregator);
+ aggregator.postCollection();
+ verify.accept((InternalGeoLine) aggregator.buildAggregation(0L));
+ } finally {
+ indexReader.close();
+ directory.close();
+ }
+ }
+}
From 065701263d1b0312f1670803e54f4e87df34eb3b Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 24 Aug 2020 17:17:37 -0700
Subject: [PATCH 02/23] update code to work
---
.../support/MultiValuesSource.java | 13 +++--
.../xpack/spatial/SpatialPlugin.java | 4 +-
.../GeoLineAggregationBuilder.java | 53 ++++++++++++-------
.../aggregations/GeoLineAggregator.java | 12 ++---
.../GeoLineAggregatorFactory.java | 33 +++++++-----
.../search/aggregations/InternalGeoLine.java | 12 ++---
.../search/aggregations/PathArraySorter.java | 4 +-
.../aggregations/GeoLineAggregatorTests.java | 22 ++++----
8 files changed, 87 insertions(+), 66 deletions(-)
diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
index e670813253c66..6411fecde65de 100644
--- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
+++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
@@ -57,11 +57,16 @@ public SortedNumericDoubleValues getField(String fieldName, LeafReaderContext ct
}
public static class AnyMultiValuesSource extends MultiValuesSource {
- public AnyMultiValuesSource(Map> valuesSourceConfigs,
- QueryShardContext context) {
+ public AnyMultiValuesSource(Map valuesSourceConfigs, QueryShardContext context) {
values = new HashMap<>(valuesSourceConfigs.size());
- for (Map.Entry> entry : valuesSourceConfigs.entrySet()) {
- values.put(entry.getKey(), entry.getValue().toValuesSource(context));
+ for (Map.Entry entry : valuesSourceConfigs.entrySet()) {
+ final ValuesSource valuesSource = entry.getValue().getValuesSource();
+ if (valuesSource instanceof ValuesSource.Numeric == false
+ && valuesSource instanceof ValuesSource.GeoPoint == false) {
+ throw new AggregationExecutionException("ValuesSource type " + valuesSource.toString() +
+ "is not supported for multi-valued aggregation");
+ }
+ values.put(entry.getKey(), valuesSource);
}
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
index 940afdc458400..7731f47ec3476 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
@@ -38,6 +38,7 @@
import org.elasticsearch.xpack.spatial.index.mapper.ShapeFieldMapper;
import org.elasticsearch.xpack.spatial.index.query.ShapeQueryBuilder;
import org.elasticsearch.xpack.spatial.ingest.CircleProcessor;
+import org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.BoundedGeoHashGridTiler;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.BoundedGeoTileGridTiler;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.GeoGridTiler;
@@ -95,7 +96,8 @@ public List> getAggregationExtentions() {
this::registerGeoShapeGridAggregators,
SpatialPlugin::registerGeoShapeBoundsAggregator,
SpatialPlugin::registerValueCountAggregator,
- SpatialPlugin::registerCardinalityAggregator
+ SpatialPlugin::registerCardinalityAggregator,
+ GeoLineAggregationBuilder::registerUsage
);
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index bc352eed08431..93155b7d396e9 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -11,42 +11,48 @@
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceParseHelper;
import org.elasticsearch.search.aggregations.support.ValueType;
-import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
+import org.elasticsearch.search.aggregations.support.ValuesSourceType;
import java.io.IOException;
import java.util.Map;
import java.util.Objects;
public class GeoLineAggregationBuilder
- extends MultiValuesSourceAggregationBuilder {
+ extends MultiValuesSourceAggregationBuilder.LeafOnly {
static final ParseField GEO_POINT_FIELD = new ParseField("geo_point");
static final ParseField SORT_FIELD = new ParseField("sort");
static final String NAME = "geo_line";
- private static final ObjectParser PARSER;
+ private static final ObjectParser PARSER =
+ ObjectParser.fromBuilder(NAME, GeoLineAggregationBuilder::new);
static {
- PARSER = new ObjectParser<>(NAME);
MultiValuesSourceParseHelper.declareCommon(PARSER, true, ValueType.NUMERIC);
- MultiValuesSourceParseHelper.declareField(GEO_POINT_FIELD.getPreferredName(), PARSER, true, false);
- MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false);
+ MultiValuesSourceParseHelper.declareField(GEO_POINT_FIELD.getPreferredName(), PARSER, true, false, false);
+ MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false, false);
}
- GeoLineAggregationBuilder(String name) {
- super(name, null);
+ public static void registerUsage(ValuesSourceRegistry.Builder builder) {
+ builder.registerUsage(NAME, CoreValuesSourceType.GEOPOINT);
+ }
+
+ public GeoLineAggregationBuilder(String name) {
+ super(name);
}
private GeoLineAggregationBuilder(GeoLineAggregationBuilder clone,
@@ -57,12 +63,8 @@ private GeoLineAggregationBuilder(GeoLineAggregationBuilder clone,
/**
* Read from a stream.
*/
- GeoLineAggregationBuilder(StreamInput in) throws IOException {
- super(in, null);
- }
-
- static AggregationBuilder parse(String aggregationName, XContentParser parser) throws IOException {
- return PARSER.parse(parser, new GeoLineAggregationBuilder(aggregationName), null);
+ public GeoLineAggregationBuilder(StreamInput in) throws IOException {
+ super(in);
}
@Override
@@ -70,16 +72,29 @@ protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBu
return new GeoLineAggregationBuilder(this, factoriesBuilder, metaData);
}
+ @Override
+ public BucketCardinality bucketCardinality() {
+ return BucketCardinality.NONE;
+ }
+
@Override
protected void innerWriteTo(StreamOutput out) {
// Do nothing, no extra state to write to stream
}
@Override
- protected MultiValuesSourceAggregatorFactory innerBuild(QueryShardContext queryShardContext, Map> configs, DocValueFormat format, AggregatorFactory parent,
- AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
- return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metaData);
+ protected ValuesSourceType defaultValueSourceType() {
+ return CoreValuesSourceType.NUMERIC;
+ }
+
+ @Override
+ protected MultiValuesSourceAggregatorFactory innerBuild(QueryShardContext queryShardContext,
+ Map configs,
+ Map filters,
+ DocValueFormat format,
+ AggregatorFactory parent,
+ AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
+ return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metadata);
}
public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index a50e8997cd139..332dddde8c8c9 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -9,7 +9,6 @@
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ScoreMode;
import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
@@ -23,12 +22,10 @@
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.MultiValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
-import java.util.List;
import java.util.Map;
import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.GEO_POINT_FIELD;
@@ -46,9 +43,8 @@ final class GeoLineAggregator extends MetricsAggregator {
private IntArray idxs;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
- Aggregator parent, List pipelineAggregators,
- Map metaData) throws IOException {
- super(name, context, parent, pipelineAggregators, metaData);
+ Aggregator parent, Map metaData) throws IOException {
+ super(name, context, parent, metaData);
this.valuesSources = valuesSources;
if (valuesSources != null) {
paths = context.bigArrays().newObjectArray(1);
@@ -136,12 +132,12 @@ public InternalAggregation buildAggregation(long bucket) {
double[] sortVals = sortValues.get(bucket);
int length = idxs.get(bucket);
new PathArraySorter(bucketLine, sortVals, length).sort();
- return new InternalGeoLine(name, bucketLine, sortVals, length, pipelineAggregators(), metaData());
+ return new InternalGeoLine(name, bucketLine, sortVals, length, metadata());
}
@Override
public InternalAggregation buildEmptyAggregation() {
- return new InternalGeoLine(name, null, null, 0, pipelineAggregators(), metaData());
+ return new InternalGeoLine(name, null, null, 0, metadata());
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
index b934e147882c6..5dce865023e1b 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -10,21 +10,19 @@
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.CardinalityUpperBound;
import org.elasticsearch.search.aggregations.support.MultiValuesSource;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
-import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
-import java.util.List;
import java.util.Map;
-final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory {
+final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory {
GeoLineAggregatorFactory(String name,
- Map> configs,
+ Map configs,
DocValueFormat format, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
Map metaData) throws IOException {
@@ -32,17 +30,26 @@ final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory<
}
@Override
- protected Aggregator createUnmapped(SearchContext searchContext, Aggregator parent, List pipelineAggregators,
+ protected Aggregator createUnmapped(SearchContext searchContext,
+ Aggregator parent,
Map metaData) throws IOException {
- return new GeoLineAggregator(name, null, searchContext, parent, pipelineAggregators, metaData);
+ return new GeoLineAggregator(name, null, searchContext, parent, metaData);
}
@Override
- protected Aggregator doCreateInternal(SearchContext searchContext, Map> configs,
- DocValueFormat format, Aggregator parent, boolean collectsFromSingleBucket,
- List pipelineAggregators, Map metaData) throws IOException {
- MultiValuesSource.AnyMultiValuesSource valuesSources = new MultiValuesSource
- .AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
- return new GeoLineAggregator(name, valuesSources, searchContext, parent, pipelineAggregators, metaData);
+ protected Aggregator doCreateInternal(SearchContext searchContext,
+ Map configs,
+ DocValueFormat format,
+ Aggregator parent,
+ CardinalityUpperBound cardinality,
+ Map metaData) throws IOException {
+ MultiValuesSource.AnyMultiValuesSource valuesSources =
+ new MultiValuesSource.AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
+ return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData);
+ }
+
+ @Override
+ public String getStatsSubtype() {
+ return configs.get(GeoLineAggregationBuilder.GEO_POINT_FIELD.getPreferredName()).valueSourceType().typeName();
}
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index b8b787335e93d..620baf7f8458e 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -12,19 +12,13 @@
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.geometry.Line;
-import org.elasticsearch.geometry.utils.WellKnownText;
import org.elasticsearch.search.aggregations.InternalAggregation;
-import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import java.util.Map;
-import static org.apache.lucene.util.ArrayUtil.grow;
-
/**
* A single line string representing a sorted sequence of geo-points
*/
@@ -35,8 +29,8 @@ public class InternalGeoLine extends InternalAggregation {
private double[] sortVals;
private int length;
- InternalGeoLine(String name, long[] line, double[] sortVals, int length, List pipelineAggregators, Map metaData) {
- super(name, pipelineAggregators, metaData);
+ InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata) {
+ super(name, metadata);
this.line = line;
this.sortVals = sortVals;
this.length = length;
@@ -79,7 +73,7 @@ public InternalAggregation reduce(List aggregations, Reduce
new PathArraySorter(finalList, finalSortVals, length).sort();
// sort the final list
- return new InternalGeoLine(name, finalList, finalSortVals, mergedSize, pipelineAggregators(), getMetaData());
+ return new InternalGeoLine(name, finalList, finalSortVals, mergedSize, getMetadata());
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
index d7ed0bda2b970..813eb8d10f89f 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
@@ -14,14 +14,14 @@ final class PathArraySorter extends IntroSorter {
private double sortValuePivot;
private int length;
- public PathArraySorter(long[] points, double[] sortValues, int length) {
+ PathArraySorter(long[] points, double[] sortValues, int length) {
this.points = points;
this.sortValues = sortValues;
this.sortValuePivot = 0;
this.length = length;
}
- public final void sort() {
+ public void sort() {
sort(0, length);
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 9db71d4400c4c..66e84546337af 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -15,7 +15,6 @@
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.geo.GeometryTestUtils;
@@ -23,18 +22,26 @@
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
-import org.mockito.internal.matchers.ArrayEquals;
+import org.elasticsearch.xpack.spatial.SpatialPlugin;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
import java.util.function.Consumer;
import static org.hamcrest.CoreMatchers.equalTo;
public class GeoLineAggregatorTests extends AggregatorTestCase {
+ @Override
+ protected List getSearchPlugins() {
+ return Collections.singletonList(new SpatialPlugin());
+ }
+
public void testSomething() throws IOException {
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder()
.setFieldName("value_field")
@@ -58,8 +65,7 @@ public void testSomething() throws IOException {
}
InternalGeoLine geoLine = new InternalGeoLine("_name",
- Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength),
- numPoints, null, null);
+ Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null);
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints);
@@ -112,13 +118,9 @@ private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
IndexSearcher indexSearcher = newSearcher(indexReader, true, true);
try {
- MappedFieldType fieldType = new GeoPointFieldMapper.GeoPointFieldType();
- fieldType.setName("value_field");
- fieldType.setHasDocValues(true);
+ MappedFieldType fieldType = new GeoPointFieldMapper.GeoPointFieldType("value_field");
- MappedFieldType fieldType2 = new NumberFieldMapper.NumberFieldType(fieldNumberType);
- fieldType2.setName("sort_field");
- fieldType2.setHasDocValues(true);
+ MappedFieldType fieldType2 = new NumberFieldMapper.NumberFieldType("sort_field", fieldNumberType);
GeoLineAggregator aggregator = createAggregator(aggregationBuilder, indexSearcher, fieldType, fieldType2);
aggregator.preCollection();
From 3089cc28466a8f82283776b5bb24a22c6180052c Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Wed, 26 Aug 2020 14:56:27 -0700
Subject: [PATCH 03/23] fix more issues
---
x-pack/plugin/spatial/build.gradle | 2 +-
.../xpack/spatial/SpatialPlugin.java | 26 ++++++++-
.../GeoLineAggregationBuilder.java | 4 +-
.../aggregations/GeoLineAggregator.java | 2 +-
.../search/aggregations/InternalGeoLine.java | 2 +-
.../rest-api-spec/test/50_geoline.yml | 56 +++++++++++++++++++
6 files changed, 85 insertions(+), 7 deletions(-)
create mode 100644 x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
diff --git a/x-pack/plugin/spatial/build.gradle b/x-pack/plugin/spatial/build.gradle
index d4eba55ef2448..44d41ca66290d 100644
--- a/x-pack/plugin/spatial/build.gradle
+++ b/x-pack/plugin/spatial/build.gradle
@@ -29,6 +29,6 @@ restResources {
testClusters.all {
setting 'xpack.license.self_generated.type', 'trial'
- setting 'indices.breaker.request.limit', '25kb'
+ //TODO(talevy): setting 'indices.breaker.request.limit', '25kb'
testDistribution = 'DEFAULT'
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
index 7731f47ec3476..58fe963184df5 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
@@ -7,6 +7,7 @@
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse;
+import org.elasticsearch.common.xcontent.ContextParser;
import org.elasticsearch.geo.GeoPlugin;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.ingest.Processor;
@@ -25,6 +26,7 @@
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
+import org.elasticsearch.xpack.core.XPackField;
import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction;
import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
@@ -39,6 +41,7 @@
import org.elasticsearch.xpack.spatial.index.query.ShapeQueryBuilder;
import org.elasticsearch.xpack.spatial.ingest.CircleProcessor;
import org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder;
+import org.elasticsearch.xpack.spatial.search.aggregations.InternalGeoLine;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.BoundedGeoHashGridTiler;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.BoundedGeoTileGridTiler;
import org.elasticsearch.xpack.spatial.search.aggregations.bucket.geogrid.GeoGridTiler;
@@ -96,11 +99,21 @@ public List> getAggregationExtentions() {
this::registerGeoShapeGridAggregators,
SpatialPlugin::registerGeoShapeBoundsAggregator,
SpatialPlugin::registerValueCountAggregator,
- SpatialPlugin::registerCardinalityAggregator,
- GeoLineAggregationBuilder::registerUsage
+ SpatialPlugin::registerCardinalityAggregator
);
}
+ @Override
+ public List getAggregations() {
+ return List.of(
+ new AggregationSpec(
+ GeoLineAggregationBuilder.NAME,
+ GeoLineAggregationBuilder::new,
+ checkLicense(GeoLineAggregationBuilder.PARSER))
+ .addResultReader(InternalGeoLine::new)
+ .setAggregatorRegistrar(GeoLineAggregationBuilder::registerUsage));
+ }
+
@Override
public Map getProcessors(Processor.Parameters parameters) {
return Map.of(CircleProcessor.TYPE, new CircleProcessor.Factory());
@@ -181,4 +194,13 @@ private static void registerValueCountAggregator(ValuesSourceRegistry.Builder bu
private static void registerCardinalityAggregator(ValuesSourceRegistry.Builder builder) {
builder.register(CardinalityAggregationBuilder.REGISTRY_KEY, GeoShapeValuesSourceType.instance(), CardinalityAggregator::new, true);
}
+
+ private ContextParser checkLicense(ContextParser realParser) {
+ return (parser, name) -> {
+ if (getLicenseState().checkFeature(XPackLicenseState.Feature.SPATIAL) == false) {
+ throw LicenseUtils.newComplianceException(XPackField.SPATIAL);
+ }
+ return realParser.parse(parser, name);
+ };
+ }
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index 93155b7d396e9..ec49bb41f6558 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -37,9 +37,9 @@ public class GeoLineAggregationBuilder
static final ParseField GEO_POINT_FIELD = new ParseField("geo_point");
static final ParseField SORT_FIELD = new ParseField("sort");
- static final String NAME = "geo_line";
+ public static final String NAME = "geo_line";
- private static final ObjectParser PARSER =
+ public static final ObjectParser PARSER =
ObjectParser.fromBuilder(NAME, GeoLineAggregationBuilder::new);
static {
MultiValuesSourceParseHelper.declareCommon(PARSER, true, ValueType.NUMERIC);
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 332dddde8c8c9..b3f9542c2ec47 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -110,7 +110,7 @@ public void collect(int doc, long bucket) throws IOException {
int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
- long lonLat = (((long) encodedLon) << 32) | (encodedLat & 0xffffffffL);
+ long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
sortVals[idx] = sort;
bucketLine[idx] = lonLat;
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 620baf7f8458e..34ac1af80a0c6 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -94,7 +94,7 @@ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) th
builder.field("type", "LineString");
final List coordinates = new ArrayList<>();
for (int i = 0; i < length; i++) {
- int x = (int) line[i] >> 32;
+ int x = (int) (line[i] >> 32);
int y = (int) line[i];
coordinates.add(new double[] { GeoEncodingUtils.decodeLongitude(x), GeoEncodingUtils.decodeLatitude(y) });
}
diff --git a/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml b/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
new file mode 100644
index 0000000000000..69057e23a6bc8
--- /dev/null
+++ b/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
@@ -0,0 +1,56 @@
+---
+"Test geoline agg":
+ - do:
+ indices.create:
+ index: locations
+ body:
+ mappings:
+ properties:
+ location:
+ type: geo_point
+ rank:
+ type: double
+
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - index:
+ _index: locations
+ _id: 1
+ - '{"location": [13.37, 47.82], "rank": 2.0 }'
+ - index:
+ _index: locations
+ _id: 2
+ - '{"location": [13.37, 47.88], "rank": 0.0 }'
+ - index:
+ _index: locations
+ _id: 3
+ - '{"location": [13.37, 48.20], "rank": 1.2 }'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: locations
+ size: 0
+ body:
+ aggs:
+ path:
+ geo_line:
+ geo_point:
+ field: location
+ sort:
+ field: rank
+ - match: { hits.total: 3 }
+ - match: { aggregations.path.type: "LineString" }
+ - length: { aggregations.path.coordinates: 3 }
+ - match: { aggregations.path.coordinates.0.0: 13.369999984279275 }
+ - match: { aggregations.path.coordinates.0.1: 47.879999969154596 }
+ - match: { aggregations.path.coordinates.1.0: 13.369999984279275 }
+ - match: { aggregations.path.coordinates.1.1: 48.19999999366701 }
+ - match: { aggregations.path.coordinates.2.0: 13.369999984279275 }
+ - match: { aggregations.path.coordinates.2.1: 47.81999995931983 }
+ - length: { aggregations.path.sorts: 3 }
+ - match: { aggregations.path.sorts.0: 0.0 }
+ - match: { aggregations.path.sorts.1: 1.2 }
+ - match: { aggregations.path.sorts.2: 2.0 }
From d03a4b4ceed040de6296da44a5c6c05149f42ebb Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Thu, 27 Aug 2020 10:55:32 -0700
Subject: [PATCH 04/23] fix more tests
---
.../aggregations/GeoLineAggregator.java | 2 +
.../aggregations/GeoLineAggregatorTests.java | 121 ++++++++++--------
2 files changed, 72 insertions(+), 51 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index b3f9542c2ec47..b3ed5d99a2a83 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -75,6 +75,8 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
@Override
public void collect(int doc, long bucket) throws IOException {
paths = bigArrays.grow(paths, bucket + 1);
+ sortValues = bigArrays.grow(sortValues, bucket + 1);
+ idxs = bigArrays.grow(idxs, bucket + 1);
if (docGeoPointValues.advanceExact(doc) && docSortValues.advanceExact(doc)) {
if (docSortValues.docValueCount() > 1) {
throw new AggregationExecutionException("Encountered more than one sort value for a " +
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 66e84546337af..97d46febe13fb 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -6,6 +6,7 @@
package org.elasticsearch.xpack.spatial.search.aggregations;
import org.apache.lucene.document.LatLonDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.geo.GeoEncodingUtils;
import org.apache.lucene.index.DirectoryReader;
@@ -15,22 +16,28 @@
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.CheckedConsumer;
import org.elasticsearch.geo.GeometryTestUtils;
import org.elasticsearch.geometry.Point;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
+import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.bucket.terms.Terms;
+import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
import org.elasticsearch.xpack.spatial.SpatialPlugin;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.function.Consumer;
import static org.hamcrest.CoreMatchers.equalTo;
@@ -47,67 +54,81 @@ public void testSomething() throws IOException {
.setFieldName("value_field")
.build();
MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("sort_field").build();
- GeoLineAggregationBuilder aggregationBuilder = new GeoLineAggregationBuilder("_name")
+ GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
.value(valueConfig)
.sort(sortConfig);
-
- int numPoints = randomIntBetween(1, 10000);
- int arrayLength = randomIntBetween(numPoints, numPoints + 1000);
- long[] points = new long[arrayLength];
- double[] sortValues = new double[arrayLength];
- for (int i = 0; i < numPoints; i++) {
- Point point = GeometryTestUtils.randomPoint(false);
- int encodedLat = GeoEncodingUtils.encodeLatitude(point.getLat());
- int encodedLon = GeoEncodingUtils.encodeLongitude(point.getLon());
- long lonLat = (((long) encodedLon) << 32) | (encodedLat & 0xffffffffL);
- points[i] = lonLat;
- sortValues[i] = i;
- }
-
- InternalGeoLine geoLine = new InternalGeoLine("_name",
- Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null);
-
- for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
- int idx1 = randomIntBetween(0, numPoints);
- int idx2 = randomIntBetween(0, numPoints);
- final long tmpPoint = points[idx1];
- points[idx1] = points[idx2];
- points[idx2] = tmpPoint;
- final double tmpSortValue = sortValues[idx1];
- sortValues[idx1] = sortValues[idx2];
- sortValues[idx2] = tmpSortValue;
+ TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name")
+ .field("group_id")
+ .subAggregation(lineAggregationBuilder);
+
+ int numGroups = randomIntBetween(1, 3);
+ Map lines = new HashMap<>(numGroups);
+ Map indexedPoints = new HashMap<>(numGroups);
+ Map indexedSortValues = new HashMap<>(numGroups);
+ for (int groupOrd = 0; groupOrd < numGroups; groupOrd++) {
+ int numPoints = randomIntBetween(2, 10);
+ int arrayLength = randomIntBetween(numPoints, numPoints);
+ long[] points = new long[arrayLength];
+ double[] sortValues = new double[arrayLength];
+ for (int i = 0; i < numPoints; i++) {
+ Point point = GeometryTestUtils.randomPoint(false);
+ int encodedLat = GeoEncodingUtils.encodeLatitude(point.getLat());
+ int encodedLon = GeoEncodingUtils.encodeLongitude(point.getLon());
+ long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
+ points[i] = lonLat;
+ sortValues[i] = i;
+ }
+ lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
+ Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null));
+
+ for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
+ int idx1 = randomIntBetween(0, numPoints - 1);
+ int idx2 = randomIntBetween(0, numPoints - 1);
+ final long tmpPoint = points[idx1];
+ points[idx1] = points[idx2];
+ points[idx2] = tmpPoint;
+ final double tmpSortValue = sortValues[idx1];
+ sortValues[idx1] = sortValues[idx2];
+ sortValues[idx2] = tmpSortValue;
+ }
+ indexedPoints.put(groupOrd, points);
+ indexedSortValues.put(groupOrd, sortValues);
}
testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> {
- for (int i = 0; i < numPoints; i++) {
- int x = (int) points[i] >> 32;
- int y = (int) points[i];
- iw.addDocument(Arrays.asList(new LatLonDocValuesField("value_field",
- GeoEncodingUtils.decodeLatitude(y),
- GeoEncodingUtils.decodeLongitude(x)),
- new SortedNumericDocValuesField("sort_field", NumericUtils.doubleToSortableLong(sortValues[i]))));
+ for (int group = 0; group < numGroups; group++) {
+ long[] points = indexedPoints.get(group);
+ double[] sortValues = indexedSortValues.get(group);
+ for (int i = 0; i < points.length; i++) {
+ int x = (int) (points[i] >> 32);
+ int y = (int) points[i];
+ iw.addDocument(Arrays.asList(new LatLonDocValuesField("value_field",
+ GeoEncodingUtils.decodeLatitude(y),
+ GeoEncodingUtils.decodeLongitude(x)),
+ new SortedNumericDocValuesField("sort_field", NumericUtils.doubleToSortableLong(sortValues[i])),
+ new SortedDocValuesField("group_id", new BytesRef(String.valueOf(group)))));
+ }
}
- }, actualGeoLine -> {
- assertThat(actualGeoLine.length(), equalTo(geoLine.length()));
- for (int i = 0; i < geoLine.length(); i++) {
- assertThat(GeoEncodingUtils.decodeLongitude((int) actualGeoLine.line()[i]),
- equalTo(GeoEncodingUtils.decodeLongitude((int) geoLine.line()[i])));
- assertThat(GeoEncodingUtils.decodeLatitude((int) actualGeoLine.line()[i] << 32),
- equalTo(GeoEncodingUtils.decodeLatitude((int) geoLine.line()[i] << 32)));
+ }, terms -> {
+ for (Terms.Bucket bucket : terms.getBuckets()) {
+ InternalGeoLine expectedGeoLine = lines.get(bucket.getKeyAsString());
+ assertThat(bucket.getDocCount(), equalTo((long) expectedGeoLine.length()));
+ InternalGeoLine geoLine = bucket.getAggregations().get("_name");
+ assertArrayEquals(expectedGeoLine.line(), geoLine.line());
}
});
}
- private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
+ private void testCase(Query query, TermsAggregationBuilder aggregationBuilder,
CheckedConsumer buildIndex,
- Consumer verify) throws IOException {
+ Consumer verify) throws IOException {
testCase(query, aggregationBuilder, buildIndex, verify, NumberFieldMapper.NumberType.LONG);
}
- private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
+ private void testCase(Query query, TermsAggregationBuilder aggregationBuilder,
CheckedConsumer buildIndex,
- Consumer verify,
+ Consumer verify,
NumberFieldMapper.NumberType fieldNumberType) throws IOException {
Directory directory = newDirectory();
@@ -119,14 +140,12 @@ private void testCase(Query query, GeoLineAggregationBuilder aggregationBuilder,
try {
MappedFieldType fieldType = new GeoPointFieldMapper.GeoPointFieldType("value_field");
-
+ MappedFieldType groupFieldType = new KeywordFieldMapper.KeywordFieldType("group_id");
MappedFieldType fieldType2 = new NumberFieldMapper.NumberFieldType("sort_field", fieldNumberType);
- GeoLineAggregator aggregator = createAggregator(aggregationBuilder, indexSearcher, fieldType, fieldType2);
- aggregator.preCollection();
- indexSearcher.search(query, aggregator);
- aggregator.postCollection();
- verify.accept((InternalGeoLine) aggregator.buildAggregation(0L));
+ Terms terms = searchAndReduce(indexSearcher, new MatchAllDocsQuery(), aggregationBuilder,
+ fieldType, fieldType2, groupFieldType);
+ verify.accept(terms);
} finally {
indexReader.close();
directory.close();
From 7de7ce367162902b0b5ef0a7c222282233c72dba Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Thu, 27 Aug 2020 14:33:21 -0700
Subject: [PATCH 05/23] insilico
---
.../aggregations/GeoLineAggregator.java | 24 ++++++++++---------
.../search/aggregations/InternalGeoLine.java | 21 +++++++++++++---
.../aggregations/GeoLineAggregatorTests.java | 8 ++++---
3 files changed, 36 insertions(+), 17 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index b3ed5d99a2a83..715f213242513 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -98,27 +98,29 @@ public void collect(int doc, long bucket) throws IOException {
long[] bucketLine = paths.get(bucket);
double[] sortVals = sortValues.get(bucket);
if (bucketLine == null) {
- bucketLine = new long[10];
+ bucketLine = new long[10000];
} else {
- bucketLine = ArrayUtil.grow(bucketLine, idx + 1);
+ //bucketLine = ArrayUtil.grow(bucketLine, idx + 1);
}
if (sortVals == null) {
- sortVals = new double[10];
+ sortVals = new double[10000];
} else {
- sortVals = ArrayUtil.grow(sortVals, idx + 1);
+ //sortVals = ArrayUtil.grow(sortVals, idx + 1);
}
int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
- sortVals[idx] = sort;
- bucketLine[idx] = lonLat;
+ if (idx < 10000) {
+ sortVals[idx] = sort;
+ bucketLine[idx] = lonLat;
- paths.set(bucket, bucketLine);
- sortValues.set(bucket, sortVals);
+ paths.set(bucket, bucketLine);
+ sortValues.set(bucket, sortVals);
+ }
idxs.set(bucket, idx + 1);
}
}
@@ -132,14 +134,14 @@ public InternalAggregation buildAggregation(long bucket) {
}
long[] bucketLine = paths.get(bucket);
double[] sortVals = sortValues.get(bucket);
- int length = idxs.get(bucket);
+ int length = Math.min(10000, idxs.get(bucket));
new PathArraySorter(bucketLine, sortVals, length).sort();
- return new InternalGeoLine(name, bucketLine, sortVals, length, metadata());
+ return new InternalGeoLine(name, bucketLine, sortVals, length, metadata(), idxs.get(bucket) < 10000);
}
@Override
public InternalAggregation buildEmptyAggregation() {
- return new InternalGeoLine(name, null, null, 0, metadata());
+ return new InternalGeoLine(name, null, null, 0, metadata(), true);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 34ac1af80a0c6..777c76bfb7a4d 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -16,6 +16,7 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
import java.util.Map;
@@ -28,12 +29,14 @@ public class InternalGeoLine extends InternalAggregation {
private long[] line;
private double[] sortVals;
private int length;
+ private boolean complete;
- InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata) {
+ InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata, boolean complete) {
super(name, metadata);
this.line = line;
this.sortVals = sortVals;
this.length = length;
+ this.complete = complete;
}
/**
@@ -43,22 +46,28 @@ public InternalGeoLine(StreamInput in) throws IOException {
super(in);
this.line = in.readLongArray();
this.length = in.readVInt();
+ this.complete = in.readBoolean();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeLongArray(line);
out.writeVInt(length);
+ out.writeBoolean(complete);
}
@Override
public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) {
int mergedSize = 0;
+ boolean complete = true;
for (InternalAggregation aggregation : aggregations) {
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
mergedSize += geoLine.length;
+ complete &= geoLine.complete;
}
+ complete &= mergedSize <= 10000;
+
long[] finalList = new long[mergedSize];
double[] finalSortVals = new double[mergedSize];
int idx = 0;
@@ -70,10 +79,12 @@ public InternalAggregation reduce(List aggregations, Reduce
}
}
- new PathArraySorter(finalList, finalSortVals, length).sort();
+ new PathArraySorter(finalList, finalSortVals, mergedSize).sort();
+ long[] finalCappedList = Arrays.copyOf(finalList, Math.min(10000, mergedSize));
+ double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(10000, mergedSize));
// sort the final list
- return new InternalGeoLine(name, finalList, finalSortVals, mergedSize, getMetadata());
+ return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, mergedSize, getMetadata(), complete);
}
@Override
@@ -89,6 +100,10 @@ public int length() {
return length;
}
+ public boolean isComplete() {
+ return complete;
+ }
+
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.field("type", "LineString");
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 97d46febe13fb..a6d9ab72ea86d 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -66,7 +66,8 @@ public void testSomething() throws IOException {
Map indexedPoints = new HashMap<>(numGroups);
Map indexedSortValues = new HashMap<>(numGroups);
for (int groupOrd = 0; groupOrd < numGroups; groupOrd++) {
- int numPoints = randomIntBetween(2, 10);
+ int numPoints = randomIntBetween(2, 20000);
+ boolean complete = numPoints <= 10000;
int arrayLength = randomIntBetween(numPoints, numPoints);
long[] points = new long[arrayLength];
double[] sortValues = new double[arrayLength];
@@ -79,7 +80,7 @@ public void testSomething() throws IOException {
sortValues[i] = i;
}
lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
- Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null));
+ Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null, complete));
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints - 1);
@@ -115,7 +116,8 @@ public void testSomething() throws IOException {
InternalGeoLine expectedGeoLine = lines.get(bucket.getKeyAsString());
assertThat(bucket.getDocCount(), equalTo((long) expectedGeoLine.length()));
InternalGeoLine geoLine = bucket.getAggregations().get("_name");
- assertArrayEquals(expectedGeoLine.line(), geoLine.line());
+ assertThat(geoLine.isComplete(), equalTo(expectedGeoLine.isComplete()));
+ //assertArrayEquals(expectedGeoLine.line(), geoLine.line());
}
});
}
From 1f5ee27506d5888e08e1951d390fc50b1d732ad2 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 31 Aug 2020 14:18:16 -0700
Subject: [PATCH 06/23] remove resizing logic
---
.../xpack/spatial/search/aggregations/GeoLineAggregator.java | 4 ----
.../xpack/spatial/search/aggregations/InternalGeoLine.java | 3 ++-
2 files changed, 2 insertions(+), 5 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 715f213242513..52c983b4c6c28 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -99,15 +99,11 @@ public void collect(int doc, long bucket) throws IOException {
double[] sortVals = sortValues.get(bucket);
if (bucketLine == null) {
bucketLine = new long[10000];
- } else {
- //bucketLine = ArrayUtil.grow(bucketLine, idx + 1);
}
if (sortVals == null) {
sortVals = new double[10000];
- } else {
- //sortVals = ArrayUtil.grow(sortVals, idx + 1);
}
int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 777c76bfb7a4d..1fdc9a9487def 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -75,7 +75,8 @@ public InternalAggregation reduce(List aggregations, Reduce
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
for (int i = 0; i < geoLine.length; i++) {
finalSortVals[idx] = geoLine.sortVals[i];
- finalList[idx++] = geoLine.line[i];
+ finalList[idx] = geoLine.line[i];
+ idx += 1;
}
}
From d4c4550b071ce272c80f13b9744fad35a37a2b34 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Tue, 1 Sep 2020 14:19:41 -0700
Subject: [PATCH 07/23] add more
---
.../GeoLineAggregationBuilder.java | 11 ++++-
.../aggregations/GeoLineAggregator.java | 9 ++--
.../GeoLineAggregatorFactory.java | 9 ++--
.../search/aggregations/InternalGeoLine.java | 39 +++++++++++----
.../aggregations/GeoLineAggregatorTests.java | 2 +-
.../aggregations/InternalGeoLineTests.java | 48 +++++++++++++++++++
.../rest-api-spec/test/50_geoline.yml | 33 +++++++------
7 files changed, 118 insertions(+), 33 deletions(-)
create mode 100644 x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index ec49bb41f6558..32f5a466d1d83 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -36,6 +36,7 @@ public class GeoLineAggregationBuilder
static final ParseField GEO_POINT_FIELD = new ParseField("geo_point");
static final ParseField SORT_FIELD = new ParseField("sort");
+ static final ParseField INCLUDE_SORT_FIELD = new ParseField("include_sort");
public static final String NAME = "geo_line";
@@ -45,8 +46,11 @@ public class GeoLineAggregationBuilder
MultiValuesSourceParseHelper.declareCommon(PARSER, true, ValueType.NUMERIC);
MultiValuesSourceParseHelper.declareField(GEO_POINT_FIELD.getPreferredName(), PARSER, true, false, false);
MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false, false);
+ PARSER.declareBoolean(GeoLineAggregationBuilder::includeSort, INCLUDE_SORT_FIELD);
}
+ private boolean includeSort;
+
public static void registerUsage(ValuesSourceRegistry.Builder builder) {
builder.registerUsage(NAME, CoreValuesSourceType.GEOPOINT);
}
@@ -67,6 +71,11 @@ public GeoLineAggregationBuilder(StreamInput in) throws IOException {
super(in);
}
+ public GeoLineAggregationBuilder includeSort(boolean includeSort) {
+ this.includeSort = includeSort;
+ return this;
+ }
+
@Override
protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBuilder, Map metaData) {
return new GeoLineAggregationBuilder(this, factoriesBuilder, metaData);
@@ -94,7 +103,7 @@ protected MultiValuesSourceAggregatorFactory innerBuild(QueryShardContext queryS
DocValueFormat format,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
- return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metadata);
+ return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metadata, includeSort);
}
public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 52c983b4c6c28..45c39d49c3368 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -8,7 +8,6 @@
import org.apache.lucene.geo.GeoEncodingUtils;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ScoreMode;
-import org.apache.lucene.util.ArrayUtil;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
@@ -41,9 +40,10 @@ final class GeoLineAggregator extends MetricsAggregator {
private ObjectArray paths;
private ObjectArray sortValues;
private IntArray idxs;
+ private boolean includeSorts;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
- Aggregator parent, Map metaData) throws IOException {
+ Aggregator parent, Map metaData, boolean includeSorts) throws IOException {
super(name, context, parent, metaData);
this.valuesSources = valuesSources;
if (valuesSources != null) {
@@ -51,6 +51,7 @@ final class GeoLineAggregator extends MetricsAggregator {
sortValues = context.bigArrays().newObjectArray(1);
idxs = context.bigArrays().newIntArray(1);
}
+ this.includeSorts = includeSorts;
}
@Override
@@ -132,12 +133,12 @@ public InternalAggregation buildAggregation(long bucket) {
double[] sortVals = sortValues.get(bucket);
int length = Math.min(10000, idxs.get(bucket));
new PathArraySorter(bucketLine, sortVals, length).sort();
- return new InternalGeoLine(name, bucketLine, sortVals, length, metadata(), idxs.get(bucket) < 10000);
+ return new InternalGeoLine(name, bucketLine, sortVals, length, metadata(), idxs.get(bucket) < 10000, includeSorts);
}
@Override
public InternalAggregation buildEmptyAggregation() {
- return new InternalGeoLine(name, null, null, 0, metadata(), true);
+ return new InternalGeoLine(name, null, null, 0, metadata(), true, includeSorts);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
index 5dce865023e1b..4f0b897154b46 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -21,19 +21,22 @@
final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory {
+ private boolean includeSort;
+
GeoLineAggregatorFactory(String name,
Map configs,
DocValueFormat format, QueryShardContext queryShardContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
- Map metaData) throws IOException {
+ Map metaData, boolean includeSort) throws IOException {
super(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metaData);
+ this.includeSort = includeSort;
}
@Override
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
Map metaData) throws IOException {
- return new GeoLineAggregator(name, null, searchContext, parent, metaData);
+ return new GeoLineAggregator(name, null, searchContext, parent, metaData, includeSort);
}
@Override
@@ -45,7 +48,7 @@ protected Aggregator doCreateInternal(SearchContext searchContext,
Map metaData) throws IOException {
MultiValuesSource.AnyMultiValuesSource valuesSources =
new MultiValuesSource.AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
- return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData);
+ return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 1fdc9a9487def..255b915598e47 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -25,18 +25,22 @@
*/
public class InternalGeoLine extends InternalAggregation {
private static final Logger logger = LogManager.getLogger(InternalGeoLine.class);
+ private static final double SCALE = Math.pow(10, 6);
private long[] line;
private double[] sortVals;
private int length;
private boolean complete;
+ private boolean includeSorts;
- InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata, boolean complete) {
+ InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata, boolean complete,
+ boolean includeSorts) {
super(name, metadata);
this.line = line;
this.sortVals = sortVals;
this.length = length;
this.complete = complete;
+ this.includeSorts = includeSorts;
}
/**
@@ -47,6 +51,7 @@ public InternalGeoLine(StreamInput in) throws IOException {
this.line = in.readLongArray();
this.length = in.readVInt();
this.complete = in.readBoolean();
+ this.includeSorts = in.readBoolean();
}
@Override
@@ -54,16 +59,19 @@ protected void doWriteTo(StreamOutput out) throws IOException {
out.writeLongArray(line);
out.writeVInt(length);
out.writeBoolean(complete);
+ out.writeBoolean(includeSorts);
}
@Override
public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) {
int mergedSize = 0;
boolean complete = true;
+ boolean includeSorts = true;
for (InternalAggregation aggregation : aggregations) {
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
mergedSize += geoLine.length;
complete &= geoLine.complete;
+ includeSorts &= geoLine.includeSorts;
}
complete &= mergedSize <= 10000;
@@ -83,9 +91,7 @@ public InternalAggregation reduce(List aggregations, Reduce
new PathArraySorter(finalList, finalSortVals, mergedSize).sort();
long[] finalCappedList = Arrays.copyOf(finalList, Math.min(10000, mergedSize));
double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(10000, mergedSize));
-
- // sort the final list
- return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, mergedSize, getMetadata(), complete);
+ return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, mergedSize, getMetadata(), complete, includeSorts);
}
@Override
@@ -107,19 +113,34 @@ public boolean isComplete() {
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
- builder.field("type", "LineString");
final List coordinates = new ArrayList<>();
for (int i = 0; i < length; i++) {
int x = (int) (line[i] >> 32);
int y = (int) line[i];
- coordinates.add(new double[] { GeoEncodingUtils.decodeLongitude(x), GeoEncodingUtils.decodeLatitude(y) });
+ coordinates.add(new double[] {
+ roundDegrees(GeoEncodingUtils.decodeLongitude(x)),
+ roundDegrees(GeoEncodingUtils.decodeLatitude(y))
+ });
}
-
- builder.array("coordinates", coordinates.toArray());
- builder.array("sorts", sortVals);
+ builder
+ .field("type", "Feature")
+ .startObject("geometry")
+ .field("type", "LineString")
+ .array("coordinates", coordinates.toArray())
+ .endObject()
+ .startObject("properties")
+ .field("complete", isComplete());
+ if (includeSorts) {
+ builder.field("sort_values", sortVals);
+ }
+ builder.endObject();
return builder;
}
+ private double roundDegrees(double degree) {
+ return Math.round(degree * SCALE) / SCALE;
+ }
+
@Override
public String toString() {
return Strings.toString(this);
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index a6d9ab72ea86d..02d6b8053c7a6 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -80,7 +80,7 @@ public void testSomething() throws IOException {
sortValues[i] = i;
}
lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
- Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null, complete));
+ Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null, complete, true));
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints - 1);
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
new file mode 100644
index 0000000000000..107582be55da2
--- /dev/null
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.plugins.SearchPlugin;
+import org.elasticsearch.search.aggregations.ParsedAggregation;
+import org.elasticsearch.test.InternalAggregationTestCase;
+import org.elasticsearch.xpack.spatial.SpatialPlugin;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class InternalGeoLineTests extends InternalAggregationTestCase {
+
+ @Override
+ protected SearchPlugin registerPlugin() {
+ return new SpatialPlugin();
+ }
+
+ @Override
+ protected InternalGeoLine createTestInstance(String name, Map metadata) {
+ return null;
+ }
+
+ @Override
+ protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
+ return null;
+ }
+
+ @Override
+ protected List randomResultsToReduce(String name, int size) {
+ return Collections.emptyList();
+ }
+
+ @Override
+ protected void assertReduced(InternalGeoLine reduced, List inputs) {
+ }
+
+ @Override
+ protected void assertFromXContent(InternalGeoLine aggregation, ParsedAggregation parsedAggregation) throws IOException {
+ // There is no ParsedGeoLine yet so we cannot test it here
+ }
+}
diff --git a/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml b/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
index 69057e23a6bc8..b2593f92290d4 100644
--- a/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
+++ b/x-pack/plugin/spatial/src/test/resources/rest-api-spec/test/50_geoline.yml
@@ -18,15 +18,15 @@
- index:
_index: locations
_id: 1
- - '{"location": [13.37, 47.82], "rank": 2.0 }'
+ - '{"location": [13.37139831, 47.82930284], "rank": 2.0 }'
- index:
_index: locations
_id: 2
- - '{"location": [13.37, 47.88], "rank": 0.0 }'
+ - '{"location": [13.3784208402, 47.88832084022], "rank": 0.0 }'
- index:
_index: locations
_id: 3
- - '{"location": [13.37, 48.20], "rank": 1.2 }'
+ - '{"location": [13.371830148701, 48.2084200148], "rank": 1.2 }'
- do:
search:
@@ -37,20 +37,23 @@
aggs:
path:
geo_line:
+ include_sort: true
geo_point:
field: location
sort:
field: rank
- match: { hits.total: 3 }
- - match: { aggregations.path.type: "LineString" }
- - length: { aggregations.path.coordinates: 3 }
- - match: { aggregations.path.coordinates.0.0: 13.369999984279275 }
- - match: { aggregations.path.coordinates.0.1: 47.879999969154596 }
- - match: { aggregations.path.coordinates.1.0: 13.369999984279275 }
- - match: { aggregations.path.coordinates.1.1: 48.19999999366701 }
- - match: { aggregations.path.coordinates.2.0: 13.369999984279275 }
- - match: { aggregations.path.coordinates.2.1: 47.81999995931983 }
- - length: { aggregations.path.sorts: 3 }
- - match: { aggregations.path.sorts.0: 0.0 }
- - match: { aggregations.path.sorts.1: 1.2 }
- - match: { aggregations.path.sorts.2: 2.0 }
+ - match: { aggregations.path.type: "Feature" }
+ - match: { aggregations.path.geometry.type: "LineString" }
+ - length: { aggregations.path.geometry.coordinates: 3 }
+ - match: { aggregations.path.geometry.coordinates.0.0: 13.378421 }
+ - match: { aggregations.path.geometry.coordinates.0.1: 47.888321 }
+ - match: { aggregations.path.geometry.coordinates.1.0: 13.37183 }
+ - match: { aggregations.path.geometry.coordinates.1.1: 48.20842 }
+ - match: { aggregations.path.geometry.coordinates.2.0: 13.371398 }
+ - match: { aggregations.path.geometry.coordinates.2.1: 47.829303 }
+ - is_true: aggregations.path.properties.complete
+ - length: { aggregations.path.properties.sort_values: 3 }
+ - match: { aggregations.path.properties.sort_values.0: 0.0 }
+ - match: { aggregations.path.properties.sort_values.1: 1.2 }
+ - match: { aggregations.path.properties.sort_values.2: 2.0 }
From 4bbc175b28ccf55389ff3b6eee0b734bc2806b7e Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 14 Sep 2020 16:57:59 -0600
Subject: [PATCH 08/23] add circuit breaker
---
.../aggregations/GeoLineAggregator.java | 19 ++++++++++++-------
1 file changed, 12 insertions(+), 7 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 45c39d49c3368..4c86aa55b9dda 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -31,16 +31,17 @@
import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
/**
- * Metric Aggregation for computing the pearson product correlation coefficient between multiple fields
+ * Metric Aggregation for joining sorted geo_point values into a single path
**/
final class GeoLineAggregator extends MetricsAggregator {
/** Multiple ValuesSource with field names */
private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+ private static final int PATH_ARRAY_SIZE = 10000;
private ObjectArray paths;
private ObjectArray sortValues;
private IntArray idxs;
- private boolean includeSorts;
+ private final boolean includeSorts;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
Aggregator parent, Map metaData, boolean includeSorts) throws IOException {
@@ -50,6 +51,10 @@ final class GeoLineAggregator extends MetricsAggregator {
paths = context.bigArrays().newObjectArray(1);
sortValues = context.bigArrays().newObjectArray(1);
idxs = context.bigArrays().newIntArray(1);
+ } else {
+ paths = null;
+ sortValues = null;
+ idxs = null;
}
this.includeSorts = includeSorts;
}
@@ -99,19 +104,19 @@ public void collect(int doc, long bucket) throws IOException {
long[] bucketLine = paths.get(bucket);
double[] sortVals = sortValues.get(bucket);
if (bucketLine == null) {
- bucketLine = new long[10000];
+ bucketLine = new long[PATH_ARRAY_SIZE];
+ addRequestCircuitBreakerBytes(Long.BYTES * PATH_ARRAY_SIZE);
}
-
-
if (sortVals == null) {
- sortVals = new double[10000];
+ sortVals = new double[PATH_ARRAY_SIZE];
+ addRequestCircuitBreakerBytes(Long.BYTES * PATH_ARRAY_SIZE);
}
int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
- if (idx < 10000) {
+ if (idx < PATH_ARRAY_SIZE) {
sortVals[idx] = sort;
bucketLine[idx] = lonLat;
From 8afb5f6a0999e58e8893e658b6236da5e827e876 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 14 Sep 2020 17:04:56 -0600
Subject: [PATCH 09/23] add telemetry to geo_line
---
.../xpack/core/spatial/action/SpatialStatsAction.java | 1 +
.../org/elasticsearch/xpack/spatial/SpatialPlugin.java | 7 ++++---
2 files changed, 5 insertions(+), 3 deletions(-)
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
index 41d6314f74267..6b68d91220949 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
@@ -38,6 +38,7 @@ private SpatialStatsAction() {
* Items to track. Serialized by ordinals. Append only, don't remove or change order of items in this list.
*/
public enum Item {
+ GEOLINE
}
public static class Request extends BaseNodesRequest implements ToXContentObject {
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
index 58fe963184df5..7d3d59b2b8efb 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
@@ -64,6 +64,7 @@
import static java.util.Collections.singletonList;
public class SpatialPlugin extends GeoPlugin implements ActionPlugin, MapperPlugin, SearchPlugin, IngestPlugin {
+ private final SpatialUsage usage = new SpatialUsage();
// to be overriden by tests
protected XPackLicenseState getLicenseState() {
@@ -107,9 +108,9 @@ public List> getAggregationExtentions() {
public List getAggregations() {
return List.of(
new AggregationSpec(
- GeoLineAggregationBuilder.NAME,
- GeoLineAggregationBuilder::new,
- checkLicense(GeoLineAggregationBuilder.PARSER))
+ GeoLineAggregationBuilder.NAME,
+ GeoLineAggregationBuilder::new,
+ usage.track(SpatialStatsAction.Item.GEOLINE, checkLicense(GeoLineAggregationBuilder.PARSER)))
.addResultReader(InternalGeoLine::new)
.setAggregatorRegistrar(GeoLineAggregationBuilder::registerUsage));
}
From 6d70ccd8088e845d5c8cd2ee9ff749e5478bebd2 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Wed, 14 Oct 2020 12:01:49 -0700
Subject: [PATCH 10/23] Refactor to leverage BucketedSort
these changes include usage of BucketedSort and ability to order
the lines by both ascending and descending time/sort order.
---
.../support/MultiValuesSource.java | 1 +
.../search/sort/BucketedSort.java | 24 +--
.../search/sort/BucketedSortTestCase.java | 2 +-
.../topmetrics/TopMetricsAggregator.java | 59 +------
.../search/aggregations/MissingHelper.java | 70 ++++++++
.../spatial/action/SpatialStatsAction.java | 12 +-
.../GeoLineAggregationBuilder.java | 31 +++-
.../aggregations/GeoLineAggregator.java | 101 +++--------
.../GeoLineAggregatorFactory.java | 17 +-
.../aggregations/GeoLineBucketedSort.java | 161 ++++++++++++++++++
.../search/aggregations/InternalGeoLine.java | 86 +++++++---
.../search/aggregations/PathArraySorter.java | 15 +-
.../aggregations/GeoLineAggregatorTests.java | 32 ++--
.../aggregations/InternalGeoLineTests.java | 77 ++++++++-
14 files changed, 476 insertions(+), 212 deletions(-)
create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/search/aggregations/MissingHelper.java
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
index 6411fecde65de..8a92f5bf54672 100644
--- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
+++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
@@ -21,6 +21,7 @@
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import java.io.IOException;
diff --git a/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java b/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java
index 5e0e8b3d48963..26ac1b4d06027 100644
--- a/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java
+++ b/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java
@@ -66,7 +66,7 @@
* worst case. Critically, it is a very fast {@code O(1)} to check if a value
* is competitive at all which, so long as buckets aren't hit in reverse
* order, they mostly won't be. Extracting results in sorted order is still
- * {@code O(n * log n)}.
+ * {@code O(n * log n)}.
*
*
* When we first collect a bucket we make sure that we've allocated enough
@@ -90,7 +90,7 @@ public interface ExtraData {
*
* Both parameters will have previously been loaded by
* {@link Loader#loadFromDoc(long, int)} so the implementer shouldn't
- * need to grow the underlying storage to implement this.
+ * need to grow the underlying storage to implement this.
*
*/
void swap(long lhs, long rhs);
@@ -128,7 +128,7 @@ public Loader loader(LeafReaderContext ctx) throws IOException {
private final SortOrder order;
private final DocValueFormat format;
private final int bucketSize;
- private final ExtraData extra;
+ protected final ExtraData extra;
/**
* {@code true} if the bucket is in heap mode, {@code false} if
* it is still gathering.
@@ -206,9 +206,9 @@ public final List getValues(long bucket) {
}
/**
- * Is this bucket a min heap {@code true} or in gathering mode {@code false}?
+ * Is this bucket a min heap {@code true} or in gathering mode {@code false}?
*/
- private boolean inHeapMode(long bucket) {
+ public boolean inHeapMode(long bucket) {
return heapMode.get(bucket);
}
@@ -254,7 +254,7 @@ private boolean inHeapMode(long bucket) {
/**
* {@code true} if the entry at index {@code lhs} is "better" than
* the entry at {@code rhs}. "Better" in this means "lower" for
- * {@link SortOrder#ASC} and "higher" for {@link SortOrder#DESC}.
+ * {@link SortOrder#ASC} and "higher" for {@link SortOrder#DESC}.
*/
protected abstract boolean betterThan(long lhs, long rhs);
@@ -283,7 +283,7 @@ protected final String debugFormat() {
/**
* Initialize the gather offsets after setting up values. Subclasses
- * should call this once, after setting up their {@link #values()}.
+ * should call this once, after setting up their {@link #values()}.
*/
protected final void initGatherOffsets() {
setNextGatherOffsets(0);
@@ -325,12 +325,12 @@ private void setNextGatherOffsets(long startingAt) {
* case.
*
*
- * @param rootIndex the index the start of the bucket
+ * @param rootIndex the index the start of the bucket
*/
private void heapify(long rootIndex) {
int maxParent = bucketSize / 2 - 1;
@@ -344,7 +344,7 @@ private void heapify(long rootIndex) {
* runs in {@code O(log n)} time.
* @param rootIndex index of the start of the bucket
* @param parent Index within the bucket of the parent to check.
- * For example, 0 is the "root".
+ * For example, 0 is the "root".
*/
private void downHeap(long rootIndex, int parent) {
while (true) {
@@ -443,7 +443,7 @@ public final void collect(int doc, long bucket) throws IOException {
/**
* {@code true} if the sort value for the doc is "better" than the
* entry at {@code index}. "Better" in means is "lower" for
- * {@link SortOrder#ASC} and "higher" for {@link SortOrder#DESC}.
+ * {@link SortOrder#ASC} and "higher" for {@link SortOrder#DESC}.
*/
protected abstract boolean docBetterThan(long index);
@@ -545,7 +545,7 @@ public abstract static class ForFloats extends BucketedSort {
* The maximum size of buckets this can store. This is because we
* store the next offset to write to in a float and floats only have
* {@code 23} bits of mantissa so they can't accurate store values
- * higher than {@code 2 ^ 24}.
+ * higher than {@code 2 ^ 24}.
*/
public static final int MAX_BUCKET_SIZE = (int) Math.pow(2, 24);
diff --git a/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java b/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
index 43e648929e9a2..c4459b798135a 100644
--- a/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
+++ b/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
@@ -212,7 +212,7 @@ public void testTwoHitsDesc() throws IOException {
assertThat(sort.getValues(0, extra.valueBuilder()), contains(extraValue(3000, 3), extraValue(200, 2)));
}
}
-
+
public void testTwoHitsAsc() throws IOException {
try (T sort = build(SortOrder.ASC, 2, new double[] {1, 2, 3})) {
BucketedSort.Leaf leaf = sort.forLeaf(null);
diff --git a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java
index b42154d46a83e..16436a798e5c0 100644
--- a/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java
+++ b/x-pack/plugin/analytics/src/main/java/org/elasticsearch/xpack/analytics/topmetrics/TopMetricsAggregator.java
@@ -28,6 +28,7 @@
import org.elasticsearch.search.sort.BucketedSort;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.sort.SortValue;
+import org.elasticsearch.xpack.core.common.search.aggregations.MissingHelper;
import org.elasticsearch.xpack.analytics.topmetrics.InternalTopMetrics.MetricValue;
import java.io.IOException;
@@ -427,62 +428,4 @@ public Loader loader(LeafReaderContext ctx) throws IOException {
public void close() {}
}
- /**
- * Helps {@link LongMetricValues} track "empty" slots. It attempts to have
- * very low CPU overhead and no memory overhead when there *aren't* empty
- * values.
- */
- private static class MissingHelper implements Releasable {
- private final BigArrays bigArrays;
- private BitArray tracker;
-
- MissingHelper(BigArrays bigArrays) {
- this.bigArrays = bigArrays;
- }
-
- void markMissing(long index) {
- if (tracker == null) {
- tracker = new BitArray(index, bigArrays);
- }
- tracker.set(index);
- }
-
- void markNotMissing(long index) {
- if (tracker == null) {
- return;
- }
- tracker.clear(index);
- }
-
- void swap(long lhs, long rhs) {
- if (tracker == null) {
- return;
- }
- boolean backup = tracker.get(lhs);
- if (tracker.get(rhs)) {
- tracker.set(lhs);
- } else {
- tracker.clear(lhs);
- }
- if (backup) {
- tracker.set(rhs);
- } else {
- tracker.clear(rhs);
- }
- }
-
- boolean isEmpty(long index) {
- if (tracker == null) {
- return false;
- }
- return tracker.get(index);
- }
-
- @Override
- public void close() {
- if (tracker != null) {
- tracker.close();
- }
- }
- }
}
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/search/aggregations/MissingHelper.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/search/aggregations/MissingHelper.java
new file mode 100644
index 0000000000000..6120d11b06809
--- /dev/null
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/common/search/aggregations/MissingHelper.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+
+package org.elasticsearch.xpack.core.common.search.aggregations;
+
+import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.BitArray;
+
+/**
+ * Helps long-valued {@link org.elasticsearch.search.sort.BucketedSort.ExtraData} track "empty" slots. It attempts to have
+ * very low CPU overhead and no memory overhead when there *aren't* empty
+ * values.
+ */
+public class MissingHelper implements Releasable {
+ private final BigArrays bigArrays;
+ private BitArray tracker;
+
+ public MissingHelper(BigArrays bigArrays) {
+ this.bigArrays = bigArrays;
+ }
+
+ public void markMissing(long index) {
+ if (tracker == null) {
+ tracker = new BitArray(index, bigArrays);
+ }
+ tracker.set(index);
+ }
+
+ public void markNotMissing(long index) {
+ if (tracker == null) {
+ return;
+ }
+ tracker.clear(index);
+ }
+
+ public void swap(long lhs, long rhs) {
+ if (tracker == null) {
+ return;
+ }
+ boolean backup = tracker.get(lhs);
+ if (tracker.get(rhs)) {
+ tracker.set(lhs);
+ } else {
+ tracker.clear(lhs);
+ }
+ if (backup) {
+ tracker.set(rhs);
+ } else {
+ tracker.clear(rhs);
+ }
+ }
+
+ public boolean isEmpty(long index) {
+ if (tracker == null) {
+ return false;
+ }
+ return tracker.get(index);
+ }
+
+ @Override
+ public void close() {
+ if (tracker != null) {
+ tracker.close();
+ }
+ }
+}
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
index 6b68d91220949..2681053fdd5ed 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/spatial/action/SpatialStatsAction.java
@@ -116,9 +116,15 @@ public EnumCounters- getStats() {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
EnumCounters
- stats = getStats();
- builder.startObject("stats");
- for (Item item : Item.values()) {
- builder.field(item.name().toLowerCase(Locale.ROOT) + "_usage", stats.get(item));
+ builder.startObject();
+ {
+ builder.startObject("stats");
+ {
+ for (Item item : Item.values()) {
+ builder.field(item.name().toLowerCase(Locale.ROOT) + "_usage", stats.get(item));
+ }
+ }
+ builder.endObject();
}
builder.endObject();
return builder;
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index 32f5a466d1d83..5f71a3f1e61f8 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -12,11 +12,11 @@
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilder;
-import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.CoreValuesSourceType;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
@@ -26,6 +26,7 @@
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
+import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
import java.util.Map;
@@ -34,8 +35,9 @@
public class GeoLineAggregationBuilder
extends MultiValuesSourceAggregationBuilder.LeafOnly {
- static final ParseField GEO_POINT_FIELD = new ParseField("geo_point");
+ static final ParseField POINT_FIELD = new ParseField("point");
static final ParseField SORT_FIELD = new ParseField("sort");
+ static final ParseField ORDER_FIELD = new ParseField("sort_order");
static final ParseField INCLUDE_SORT_FIELD = new ParseField("include_sort");
public static final String NAME = "geo_line";
@@ -44,12 +46,14 @@ public class GeoLineAggregationBuilder
ObjectParser.fromBuilder(NAME, GeoLineAggregationBuilder::new);
static {
MultiValuesSourceParseHelper.declareCommon(PARSER, true, ValueType.NUMERIC);
- MultiValuesSourceParseHelper.declareField(GEO_POINT_FIELD.getPreferredName(), PARSER, true, false, false);
+ MultiValuesSourceParseHelper.declareField(POINT_FIELD.getPreferredName(), PARSER, true, false, false);
MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false, false);
+ PARSER.declareString((builder, order) -> builder.sortOrder(SortOrder.fromString(order)), ORDER_FIELD);
PARSER.declareBoolean(GeoLineAggregationBuilder::includeSort, INCLUDE_SORT_FIELD);
}
private boolean includeSort;
+ private SortOrder sortOrder = SortOrder.ASC;
public static void registerUsage(ValuesSourceRegistry.Builder builder) {
builder.registerUsage(NAME, CoreValuesSourceType.GEOPOINT);
@@ -69,6 +73,8 @@ private GeoLineAggregationBuilder(GeoLineAggregationBuilder clone,
*/
public GeoLineAggregationBuilder(StreamInput in) throws IOException {
super(in);
+ sortOrder = SortOrder.readFromStream(in);
+ includeSort = in.readBoolean();
}
public GeoLineAggregationBuilder includeSort(boolean includeSort) {
@@ -76,6 +82,11 @@ public GeoLineAggregationBuilder includeSort(boolean includeSort) {
return this;
}
+ public GeoLineAggregationBuilder sortOrder(SortOrder sortOrder) {
+ this.sortOrder = sortOrder;
+ return this;
+ }
+
@Override
protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBuilder, Map metaData) {
return new GeoLineAggregationBuilder(this, factoriesBuilder, metaData);
@@ -87,8 +98,9 @@ public BucketCardinality bucketCardinality() {
}
@Override
- protected void innerWriteTo(StreamOutput out) {
- // Do nothing, no extra state to write to stream
+ protected void innerWriteTo(StreamOutput out) throws IOException {
+ sortOrder.writeTo(out);
+ out.writeBoolean(includeSort);
}
@Override
@@ -97,18 +109,19 @@ protected ValuesSourceType defaultValueSourceType() {
}
@Override
- protected MultiValuesSourceAggregatorFactory innerBuild(QueryShardContext queryShardContext,
+ protected MultiValuesSourceAggregatorFactory innerBuild(AggregationContext aggregationContext,
Map configs,
Map filters,
DocValueFormat format,
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
- return new GeoLineAggregatorFactory(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metadata, includeSort);
+ return new GeoLineAggregatorFactory(name, configs, format, aggregationContext, parent, subFactoriesBuilder, metadata,
+ includeSort, sortOrder);
}
public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
- valueConfig = Objects.requireNonNull(valueConfig, "Configuration for field [" + GEO_POINT_FIELD + "] cannot be null");
- field(GEO_POINT_FIELD.getPreferredName(), valueConfig);
+ valueConfig = Objects.requireNonNull(valueConfig, "Configuration for field [" + POINT_FIELD + "] cannot be null");
+ field(POINT_FIELD.getPreferredName(), valueConfig);
return this;
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 4c86aa55b9dda..a409dbb84947d 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -5,58 +5,47 @@
*/
package org.elasticsearch.xpack.spatial.search.aggregations;
-import org.apache.lucene.geo.GeoEncodingUtils;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ScoreMode;
-import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.lease.Releasables;
-import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.common.util.IntArray;
-import org.elasticsearch.common.util.ObjectArray;
-import org.elasticsearch.index.fielddata.MultiGeoPointValues;
-import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
-import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
-import org.elasticsearch.search.aggregations.LeafBucketCollectorBase;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
import org.elasticsearch.search.aggregations.support.MultiValuesSource;
import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.sort.BucketedSort;
+import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
import java.util.Map;
-import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.GEO_POINT_FIELD;
-import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
-
/**
* Metric Aggregation for joining sorted geo_point values into a single path
**/
final class GeoLineAggregator extends MetricsAggregator {
/** Multiple ValuesSource with field names */
private final MultiValuesSource.AnyMultiValuesSource valuesSources;
- private static final int PATH_ARRAY_SIZE = 10000;
+ static final int MAX_PATH_SIZE = 10000;
- private ObjectArray paths;
- private ObjectArray sortValues;
- private IntArray idxs;
+ private final GeoLineBucketedSort sort;
+ private final GeoLineBucketedSort.Extra extra;
private final boolean includeSorts;
+ private final SortOrder sortOrder;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
- Aggregator parent, Map metaData, boolean includeSorts) throws IOException {
+ Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder) throws IOException {
super(name, context, parent, metaData);
this.valuesSources = valuesSources;
if (valuesSources != null) {
- paths = context.bigArrays().newObjectArray(1);
- sortValues = context.bigArrays().newObjectArray(1);
- idxs = context.bigArrays().newIntArray(1);
+ this.extra = new GeoLineBucketedSort.Extra(context.bigArrays(), valuesSources);
+ this.sort = new GeoLineBucketedSort(context.bigArrays(), sortOrder, null, MAX_PATH_SIZE, valuesSources, extra);
} else {
- paths = null;
- sortValues = null;
- idxs = null;
+ this.extra = null;
+ this.sort = null;
}
this.includeSorts = includeSorts;
+ this.sortOrder = sortOrder;
}
@Override
@@ -73,58 +62,12 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx,
if (valuesSources == null) {
return LeafBucketCollector.NO_OP_COLLECTOR;
}
- final BigArrays bigArrays = context.bigArrays();
- MultiGeoPointValues docGeoPointValues = valuesSources.getGeoPointField(GEO_POINT_FIELD.getPreferredName(), ctx);
- SortedNumericDoubleValues docSortValues = valuesSources.getNumericField(SORT_FIELD.getPreferredName(), ctx);
+ BucketedSort.Leaf leafSort = sort.forLeaf(ctx);
- return new LeafBucketCollectorBase(sub, docGeoPointValues) {
+ return new LeafBucketCollector(){
@Override
public void collect(int doc, long bucket) throws IOException {
- paths = bigArrays.grow(paths, bucket + 1);
- sortValues = bigArrays.grow(sortValues, bucket + 1);
- idxs = bigArrays.grow(idxs, bucket + 1);
- if (docGeoPointValues.advanceExact(doc) && docSortValues.advanceExact(doc)) {
- if (docSortValues.docValueCount() > 1) {
- throw new AggregationExecutionException("Encountered more than one sort value for a " +
- "single document. Use a script to combine multiple sort-values-per-doc into a single value.");
- }
- if (docGeoPointValues.docValueCount() > 1) {
- throw new AggregationExecutionException("Encountered more than one geo_point value for a " +
- "single document. Use a script to combine multiple geo_point-values-per-doc into a single value.");
- }
-
- // There should always be one weight if advanceExact lands us here, either
- // a real weight or a `missing` weight
- assert docSortValues.docValueCount() == 1;
- assert docGeoPointValues.docValueCount() == 1;
- final double sort = docSortValues.nextValue();
- final GeoPoint point = docGeoPointValues.nextValue();
-
- int idx = idxs.get(bucket);
- long[] bucketLine = paths.get(bucket);
- double[] sortVals = sortValues.get(bucket);
- if (bucketLine == null) {
- bucketLine = new long[PATH_ARRAY_SIZE];
- addRequestCircuitBreakerBytes(Long.BYTES * PATH_ARRAY_SIZE);
- }
- if (sortVals == null) {
- sortVals = new double[PATH_ARRAY_SIZE];
- addRequestCircuitBreakerBytes(Long.BYTES * PATH_ARRAY_SIZE);
- }
-
- int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
- int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
- long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
-
- if (idx < PATH_ARRAY_SIZE) {
- sortVals[idx] = sort;
- bucketLine[idx] = lonLat;
-
- paths.set(bucket, bucketLine);
- sortValues.set(bucket, sortVals);
- }
- idxs.set(bucket, idx + 1);
- }
+ leafSort.collect(doc, bucket);
}
};
}
@@ -134,20 +77,20 @@ public InternalAggregation buildAggregation(long bucket) {
if (valuesSources == null) {
return buildEmptyAggregation();
}
- long[] bucketLine = paths.get(bucket);
- double[] sortVals = sortValues.get(bucket);
- int length = Math.min(10000, idxs.get(bucket));
- new PathArraySorter(bucketLine, sortVals, length).sort();
- return new InternalGeoLine(name, bucketLine, sortVals, length, metadata(), idxs.get(bucket) < 10000, includeSorts);
+ boolean complete = sort.inHeapMode(bucket) == false;
+ double[] sortVals = sort.getSortValues(bucket);
+ long[] bucketLine = sort.getPoints(bucket);
+ new PathArraySorter(bucketLine, sortVals, sortOrder).sort();
+ return new InternalGeoLine(name, bucketLine, sortVals, metadata(), complete, includeSorts, sortOrder);
}
@Override
public InternalAggregation buildEmptyAggregation() {
- return new InternalGeoLine(name, null, null, 0, metadata(), true, includeSorts);
+ return new InternalGeoLine(name, null, null, metadata(), true, includeSorts, sortOrder);
}
@Override
public void doClose() {
- Releasables.close(paths, idxs, sortValues);
+ Releasables.close(sort, extra);
}
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
index 4f0b897154b46..3ae6d0f4de826 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -5,16 +5,17 @@
*/
package org.elasticsearch.xpack.spatial.search.aggregations;
-import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.CardinalityUpperBound;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
import org.elasticsearch.search.aggregations.support.MultiValuesSource;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
+import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
import java.util.Map;
@@ -22,21 +23,23 @@
final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory {
private boolean includeSort;
+ private SortOrder sortOrder;
GeoLineAggregatorFactory(String name,
Map configs,
- DocValueFormat format, QueryShardContext queryShardContext, AggregatorFactory parent,
+ DocValueFormat format, AggregationContext aggregationContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
- Map metaData, boolean includeSort) throws IOException {
- super(name, configs, format, queryShardContext, parent, subFactoriesBuilder, metaData);
+ Map metaData, boolean includeSort, SortOrder sortOrder) throws IOException {
+ super(name, configs, format, aggregationContext, parent, subFactoriesBuilder, metaData);
this.includeSort = includeSort;
+ this.sortOrder = sortOrder;
}
@Override
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
Map metaData) throws IOException {
- return new GeoLineAggregator(name, null, searchContext, parent, metaData, includeSort);
+ return new GeoLineAggregator(name, null, searchContext, parent, metaData, includeSort, sortOrder);
}
@Override
@@ -48,11 +51,11 @@ protected Aggregator doCreateInternal(SearchContext searchContext,
Map metaData) throws IOException {
MultiValuesSource.AnyMultiValuesSource valuesSources =
new MultiValuesSource.AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
- return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort);
+ return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort, sortOrder);
}
@Override
public String getStatsSubtype() {
- return configs.get(GeoLineAggregationBuilder.GEO_POINT_FIELD.getPreferredName()).valueSourceType().typeName();
+ return configs.get(GeoLineAggregationBuilder.POINT_FIELD.getPreferredName()).valueSourceType().typeName();
}
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
new file mode 100644
index 0000000000000..425f70d6edb02
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
@@ -0,0 +1,161 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.apache.lucene.geo.GeoEncodingUtils;
+import org.apache.lucene.index.LeafReaderContext;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.lease.Releasable;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.DoubleArray;
+import org.elasticsearch.common.util.LongArray;
+import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.AggregationExecutionException;
+import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.search.sort.BucketedSort;
+import org.elasticsearch.search.sort.SortOrder;
+import org.elasticsearch.xpack.core.common.search.aggregations.MissingHelper;
+
+import java.io.IOException;
+
+import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
+
+public class GeoLineBucketedSort extends BucketedSort.ForDoubles {
+ private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+
+ public GeoLineBucketedSort(BigArrays bigArrays, SortOrder sortOrder, DocValueFormat format, int bucketSize,
+ MultiValuesSource.AnyMultiValuesSource valuesSources, GeoLineBucketedSort.Extra extra) {
+ super(bigArrays, sortOrder, format, bucketSize, extra);
+ this.valuesSources = valuesSources;
+ }
+
+ public double[] getSortValues(long bucket) {
+ int bucketSize = getBucketSize();
+ long rootIndex = bucket * bucketSize;
+ if (rootIndex >= values().size()) {
+ // We've never seen this bucket.
+ return new double[]{};
+ }
+ long start = inHeapMode(bucket) ? rootIndex : (rootIndex + getNextGatherOffset(rootIndex) + 1);
+ long end = rootIndex + bucketSize;
+ double[] result = new double[(int)(end - start)];
+ int i = 0;
+ for (long index = start; index < end; index++) {
+ double timestampValue = ((DoubleArray)values()).get(index);
+ result[i++] = timestampValue;
+ }
+ return result;
+ }
+
+ public long[] getPoints(long bucket) {
+ int bucketSize = getBucketSize();
+ long rootIndex = bucket * bucketSize;
+ if (rootIndex >= values().size()) {
+ // We've never seen this bucket.
+ return new long[]{};
+ }
+ long start = inHeapMode(bucket) ? rootIndex : (rootIndex + getNextGatherOffset(rootIndex) + 1);
+ long end = rootIndex + bucketSize;
+ long[] result = new long[(int)(end - start)];
+ int i = 0;
+ for (long index = start; index < end; index++) {
+ long geoPointValue = ((Extra) extra).values.get(index);
+ result[i++] = geoPointValue;
+ }
+ return result;
+ }
+
+ @Override
+ public BucketedSort.Leaf forLeaf(LeafReaderContext ctx) throws IOException {
+ return new BucketedSort.ForDoubles.Leaf(ctx) {
+ private final SortedNumericDoubleValues docSortValues = valuesSources.getNumericField(SORT_FIELD.getPreferredName(), ctx);
+ private double docValue;
+
+ @Override
+ protected boolean advanceExact(int doc) throws IOException {
+ if (docSortValues.advanceExact(doc)) {
+ if (docSortValues.docValueCount() > 1) {
+ throw new AggregationExecutionException("Encountered more than one sort value for a " +
+ "single document. Use a script to combine multiple sort-values-per-doc into a single value.");
+ }
+
+ // There should always be one weight if advanceExact lands us here, either
+ // a real weight or a `missing` weight
+ assert docSortValues.docValueCount() == 1;
+ docValue = docSortValues.nextValue();
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ protected double docValue() {
+ return docValue;
+ }
+ };
+ }
+
+ static class Extra implements BucketedSort.ExtraData, Releasable {
+
+ private final BigArrays bigArrays;
+ private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+ private LongArray values;
+ private final MissingHelper empty;
+
+ Extra(BigArrays bigArrays, MultiValuesSource.AnyMultiValuesSource valuesSources) {
+ this.bigArrays = bigArrays;
+ this.valuesSources = valuesSources;
+ this.values = bigArrays.newLongArray(1, false);
+ this.empty = new MissingHelper(bigArrays);
+ }
+
+ @Override
+ public void swap(long lhs, long rhs) {
+ long tmp = values.get(lhs);
+ values.set(lhs, values.get(rhs));
+ values.set(rhs, tmp);
+ empty.swap(lhs, rhs);
+ }
+
+ @Override
+ public Loader loader(LeafReaderContext ctx) throws IOException {
+ final MultiGeoPointValues docGeoPointValues = valuesSources
+ .getGeoPointField(GeoLineAggregationBuilder.POINT_FIELD.getPreferredName(), ctx);
+ return (index, doc) -> {
+ if (false == docGeoPointValues.advanceExact(doc)) {
+ empty.markMissing(index);
+ return;
+ }
+
+ if (docGeoPointValues.docValueCount() > 1) {
+ throw new AggregationExecutionException("Encountered more than one geo_point value for a " +
+ "single document. Use a script to combine multiple geo_point-values-per-doc into a single value.");
+ }
+
+ if (index > values.size()) {
+ values = bigArrays.grow(values, index + 1);
+ }
+
+ final GeoPoint point = docGeoPointValues.nextValue();
+ int encodedLat = GeoEncodingUtils.encodeLatitude(point.lat());
+ int encodedLon = GeoEncodingUtils.encodeLongitude(point.lon());
+ long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
+
+ values.set(index, lonLat);
+ empty.markNotMissing(index);
+ };
+ }
+
+ @Override
+ public void close() {
+ Releasables.close(values, empty);
+ }
+ }
+}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 255b915598e47..eec9ec88f9644 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -5,42 +5,41 @@
*/
package org.elasticsearch.xpack.spatial.search.aggregations;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
import org.apache.lucene.geo.GeoEncodingUtils;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.sort.SortOrder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
/**
* A single line string representing a sorted sequence of geo-points
*/
public class InternalGeoLine extends InternalAggregation {
- private static final Logger logger = LogManager.getLogger(InternalGeoLine.class);
private static final double SCALE = Math.pow(10, 6);
private long[] line;
private double[] sortVals;
- private int length;
private boolean complete;
private boolean includeSorts;
+ private SortOrder sortOrder;
- InternalGeoLine(String name, long[] line, double[] sortVals, int length, Map metadata, boolean complete,
- boolean includeSorts) {
+ InternalGeoLine(String name, long[] line, double[] sortVals, Map metadata, boolean complete,
+ boolean includeSorts, SortOrder sortOrder) {
super(name, metadata);
this.line = line;
this.sortVals = sortVals;
- this.length = length;
this.complete = complete;
this.includeSorts = includeSorts;
+ this.sortOrder = sortOrder;
}
/**
@@ -49,17 +48,19 @@ public class InternalGeoLine extends InternalAggregation {
public InternalGeoLine(StreamInput in) throws IOException {
super(in);
this.line = in.readLongArray();
- this.length = in.readVInt();
+ this.sortVals = in.readDoubleArray();
this.complete = in.readBoolean();
this.includeSorts = in.readBoolean();
+ this.sortOrder = SortOrder.readFromStream(in);
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeLongArray(line);
- out.writeVInt(length);
+ out.writeDoubleArray(sortVals);
out.writeBoolean(complete);
out.writeBoolean(includeSorts);
+ sortOrder.writeTo(out);
}
@Override
@@ -69,29 +70,34 @@ public InternalAggregation reduce(List aggregations, Reduce
boolean includeSorts = true;
for (InternalAggregation aggregation : aggregations) {
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
- mergedSize += geoLine.length;
+ mergedSize += geoLine.line.length;
complete &= geoLine.complete;
includeSorts &= geoLine.includeSorts;
}
- complete &= mergedSize <= 10000;
+ complete &= mergedSize <= GeoLineAggregator.MAX_PATH_SIZE;
long[] finalList = new long[mergedSize];
double[] finalSortVals = new double[mergedSize];
int idx = 0;
for (InternalAggregation aggregation : aggregations) {
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
- for (int i = 0; i < geoLine.length; i++) {
+ for (int i = 0; i < geoLine.line.length; i++) {
finalSortVals[idx] = geoLine.sortVals[i];
finalList[idx] = geoLine.line[i];
idx += 1;
}
}
- new PathArraySorter(finalList, finalSortVals, mergedSize).sort();
- long[] finalCappedList = Arrays.copyOf(finalList, Math.min(10000, mergedSize));
- double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(10000, mergedSize));
- return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, mergedSize, getMetadata(), complete, includeSorts);
+ new PathArraySorter(finalList, finalSortVals, sortOrder).sort();
+ long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
+ double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
+ return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder);
+ }
+
+ @Override
+ protected boolean mustReduceOnSingleInternalAgg() {
+ return false;
}
@Override
@@ -103,18 +109,30 @@ public long[] line() {
return line;
}
+ public double[] sortVals() {
+ return sortVals;
+ }
+
public int length() {
- return length;
+ return line.length;
}
public boolean isComplete() {
return complete;
}
+ public boolean includeSorts() {
+ return includeSorts;
+ }
+
+ public SortOrder sortOrder() {
+ return sortOrder;
+ }
+
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
final List coordinates = new ArrayList<>();
- for (int i = 0; i < length; i++) {
+ for (int i = 0; i < line.length; i++) {
int x = (int) (line[i] >> 32);
int y = (int) line[i];
coordinates.add(new double[] {
@@ -141,14 +159,8 @@ private double roundDegrees(double degree) {
return Math.round(degree * SCALE) / SCALE;
}
- @Override
- public String toString() {
- return Strings.toString(this);
- }
-
@Override
public Object getProperty(List path) {
- logger.error("what in the world");
if (path.isEmpty()) {
return this;
} else if (path.size() == 1 && "value".equals(path.get(0))) {
@@ -157,4 +169,30 @@ public Object getProperty(List path) {
throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path);
}
}
+
+ @Override
+ public String toString() {
+ return Strings.toString(this);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(super.hashCode(), Arrays.hashCode(line), Arrays.hashCode(sortVals), complete, includeSorts, sortOrder);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null || getClass() != obj.getClass()) return false;
+ if (super.equals(obj) == false) return false;
+
+ InternalGeoLine that = (InternalGeoLine) obj;
+ return super.equals(obj)
+ && Arrays.equals(line, that.line)
+ && Arrays.equals(sortVals, that.sortVals)
+ && Objects.equals(complete, that.complete)
+ && Objects.equals(includeSorts, that.includeSorts)
+ && Objects.equals(sortOrder, that.sortOrder);
+
+ }
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
index 813eb8d10f89f..8218d39c95e04 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
@@ -6,23 +6,25 @@
package org.elasticsearch.xpack.spatial.search.aggregations;
import org.apache.lucene.util.IntroSorter;
+import org.elasticsearch.search.sort.SortOrder;
final class PathArraySorter extends IntroSorter {
private final long[] points;
private final double[] sortValues;
private double sortValuePivot;
- private int length;
+ private final SortOrder sortOrder;
- PathArraySorter(long[] points, double[] sortValues, int length) {
+ PathArraySorter(long[] points, double[] sortValues, SortOrder sortOrder) {
+ assert points.length == sortValues.length;
this.points = points;
this.sortValues = sortValues;
this.sortValuePivot = 0;
- this.length = length;
+ this.sortOrder = sortOrder;
}
public void sort() {
- sort(0, length);
+ sort(0, points.length);
}
@Override
@@ -42,6 +44,9 @@ protected void setPivot(int i) {
@Override
protected int comparePivot(int j) {
- return Double.compare(sortValuePivot, sortValues[j]);
+ if (SortOrder.ASC.equals(sortOrder)) {
+ return Double.compare(sortValuePivot, sortValues[j]);
+ }
+ return Double.compare(sortValues[j], sortValuePivot);
}
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 02d6b8053c7a6..15a2992d466c5 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -30,6 +30,7 @@
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
+import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.xpack.spatial.SpatialPlugin;
import java.io.IOException;
@@ -49,38 +50,45 @@ protected List getSearchPlugins() {
return Collections.singletonList(new SpatialPlugin());
}
- public void testSomething() throws IOException {
+
+ public void testAggregator() throws IOException {
+ testAggregator(SortOrder.ASC);
+ testAggregator(SortOrder.DESC);
+ }
+
+ private void testAggregator(SortOrder sortOrder) throws IOException {
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder()
.setFieldName("value_field")
.build();
MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("sort_field").build();
GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
.value(valueConfig)
+ .sortOrder(sortOrder)
.sort(sortConfig);
TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name")
.field("group_id")
.subAggregation(lineAggregationBuilder);
- int numGroups = randomIntBetween(1, 3);
+ int numGroups = randomIntBetween(1, 4);
Map lines = new HashMap<>(numGroups);
Map indexedPoints = new HashMap<>(numGroups);
Map indexedSortValues = new HashMap<>(numGroups);
for (int groupOrd = 0; groupOrd < numGroups; groupOrd++) {
int numPoints = randomIntBetween(2, 20000);
- boolean complete = numPoints <= 10000;
- int arrayLength = randomIntBetween(numPoints, numPoints);
- long[] points = new long[arrayLength];
- double[] sortValues = new double[arrayLength];
+ boolean complete = numPoints < GeoLineAggregator.MAX_PATH_SIZE;
+ long[] points = new long[numPoints];
+ double[] sortValues = new double[numPoints];
for (int i = 0; i < numPoints; i++) {
Point point = GeometryTestUtils.randomPoint(false);
int encodedLat = GeoEncodingUtils.encodeLatitude(point.getLat());
int encodedLon = GeoEncodingUtils.encodeLongitude(point.getLon());
long lonLat = (((long) encodedLon) << 32) | encodedLat & 0xffffffffL;
points[i] = lonLat;
- sortValues[i] = i;
+ sortValues[i] = SortOrder.ASC.equals(sortOrder) ? i : numPoints - i;
}
+ int lineSize = Math.min(numPoints, GeoLineAggregator.MAX_PATH_SIZE);
lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
- Arrays.copyOf(points, arrayLength), Arrays.copyOf(sortValues, arrayLength), numPoints, null, complete, true));
+ Arrays.copyOf(points, lineSize), Arrays.copyOf(sortValues, lineSize), null, complete, true, sortOrder));
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints - 1);
@@ -114,10 +122,14 @@ public void testSomething() throws IOException {
}, terms -> {
for (Terms.Bucket bucket : terms.getBuckets()) {
InternalGeoLine expectedGeoLine = lines.get(bucket.getKeyAsString());
- assertThat(bucket.getDocCount(), equalTo((long) expectedGeoLine.length()));
InternalGeoLine geoLine = bucket.getAggregations().get("_name");
+ assertThat(geoLine.length(), equalTo(expectedGeoLine.length()));
assertThat(geoLine.isComplete(), equalTo(expectedGeoLine.isComplete()));
- //assertArrayEquals(expectedGeoLine.line(), geoLine.line());
+ for (int i = 0; i < geoLine.sortVals().length; i++) {
+ geoLine.sortVals()[i] = NumericUtils.sortableLongToDouble((long) geoLine.sortVals()[i]);
+ }
+ assertArrayEquals(expectedGeoLine.sortVals(), geoLine.sortVals(), 0.0001);
+ assertArrayEquals(expectedGeoLine.line(), geoLine.line());
}
});
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
index 107582be55da2..887f914be1200 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -5,15 +5,24 @@
*/
package org.elasticsearch.xpack.spatial.search.aggregations;
+import org.elasticsearch.common.ParseField;
+import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.plugins.SearchPlugin;
+import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.ParsedAggregation;
+import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.InternalAggregationTestCase;
import org.elasticsearch.xpack.spatial.SpatialPlugin;
import java.io.IOException;
-import java.util.Collections;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.stream.Stream;
+
+import static java.util.stream.Collectors.toList;
public class InternalGeoLineTests extends InternalAggregationTestCase {
@@ -24,25 +33,85 @@ protected SearchPlugin registerPlugin() {
@Override
protected InternalGeoLine createTestInstance(String name, Map metadata) {
- return null;
+ int length = randomIntBetween(2, 2 * GeoLineAggregator.MAX_PATH_SIZE);
+ long[] points = new long[length];
+ double[] sortVals = new double[length];
+ for (int i = 0; i < length; i++) {
+ points[i] = i;
+ sortVals[i] = i;
+ }
+ boolean complete = length <= GeoLineAggregator.MAX_PATH_SIZE;
+ return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), randomFrom(SortOrder.values()));
}
@Override
protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
- return null;
+ String name = instance.getName();
+ long[] line = Arrays.copyOf(instance.line(), instance.line().length);
+ double[] sortVals = Arrays.copyOf(instance.sortVals(), instance.sortVals().length);
+ Map metadata = instance.getMetadata();
+ boolean complete = instance.isComplete();
+ boolean includeSorts = instance.includeSorts();
+ SortOrder sortOrder = instance.sortOrder();
+ switch (randomIntBetween(0, 6)) {
+ case 0:
+ name += randomAlphaOfLength(5);
+ break;
+ case 1:
+ line[0] = line[0] + 1000000L;
+ break;
+ case 2:
+ sortVals[0] = sortVals[0] + 10000;
+ break;
+ case 3:
+ if (metadata == null) {
+ metadata = new HashMap<>(1);
+ } else {
+ metadata = new HashMap<>(instance.getMetadata());
+ }
+ metadata.put(randomAlphaOfLength(15), randomInt());
+ break;
+ case 4:
+ complete = !complete;
+ break;
+ case 5:
+ includeSorts = !includeSorts;
+ break;
+ case 6:
+ sortOrder = SortOrder.ASC.equals(sortOrder) ? SortOrder.DESC : SortOrder.ASC;
+ break;
+ default:
+ throw new AssertionError("Illegal randomisation branch");
+ }
+ return new InternalGeoLine(name, line, sortVals, metadata, complete, includeSorts, sortOrder);
}
@Override
protected List randomResultsToReduce(String name, int size) {
- return Collections.emptyList();
+ return Stream.generate(() -> createTestInstance(name, null)).limit(size).collect(toList());
}
@Override
protected void assertReduced(InternalGeoLine reduced, List inputs) {
+ // TODO(talevy)
+ // assert final line is sorted
}
@Override
protected void assertFromXContent(InternalGeoLine aggregation, ParsedAggregation parsedAggregation) throws IOException {
// There is no ParsedGeoLine yet so we cannot test it here
}
+
+ @Override
+ protected List getNamedXContents() {
+ List extendedNamedXContents = new ArrayList<>(super.getNamedXContents());
+ extendedNamedXContents.add(new NamedXContentRegistry.Entry(Aggregation.class,
+ new ParseField(GeoLineAggregationBuilder.NAME),
+ (p, c) -> {
+ assumeTrue("There is no ParsedGeoLine yet", false);
+ return null;
+ }
+ ));
+ return extendedNamedXContents;
+ }
}
From baaea0eb995e49d47cba9df60ee53347f710adca Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Thu, 29 Oct 2020 16:34:13 -0700
Subject: [PATCH 11/23] add size param
---
.../aggregations/GeoLineAggregationBuilder.java | 13 ++++++++++++-
.../search/aggregations/GeoLineAggregator.java | 5 ++++-
.../aggregations/GeoLineAggregatorFactory.java | 8 +++++---
.../search/aggregations/InternalGeoLine.java | 3 ++-
4 files changed, 23 insertions(+), 6 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index 5f71a3f1e61f8..c64b57e79a8d1 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -39,6 +39,7 @@ public class GeoLineAggregationBuilder
static final ParseField SORT_FIELD = new ParseField("sort");
static final ParseField ORDER_FIELD = new ParseField("sort_order");
static final ParseField INCLUDE_SORT_FIELD = new ParseField("include_sort");
+ static final ParseField SIZE_FIELD = new ParseField("size");
public static final String NAME = "geo_line";
@@ -50,10 +51,12 @@ public class GeoLineAggregationBuilder
MultiValuesSourceParseHelper.declareField(SORT_FIELD.getPreferredName(), PARSER, true, false, false);
PARSER.declareString((builder, order) -> builder.sortOrder(SortOrder.fromString(order)), ORDER_FIELD);
PARSER.declareBoolean(GeoLineAggregationBuilder::includeSort, INCLUDE_SORT_FIELD);
+ PARSER.declareInt(GeoLineAggregationBuilder::size, SIZE_FIELD);
}
private boolean includeSort;
private SortOrder sortOrder = SortOrder.ASC;
+ private int size = GeoLineAggregator.MAX_PATH_SIZE;
public static void registerUsage(ValuesSourceRegistry.Builder builder) {
builder.registerUsage(NAME, CoreValuesSourceType.GEOPOINT);
@@ -87,6 +90,14 @@ public GeoLineAggregationBuilder sortOrder(SortOrder sortOrder) {
return this;
}
+ public GeoLineAggregationBuilder size(int size) {
+ if (size > GeoLineAggregator.MAX_PATH_SIZE) {
+ throw new IllegalArgumentException("invalid [size] value [" + size + "] must be <= " + GeoLineAggregator.MAX_PATH_SIZE);
+ }
+ this.size = size;
+ return this;
+ }
+
@Override
protected AggregationBuilder shallowCopy(AggregatorFactories.Builder factoriesBuilder, Map metaData) {
return new GeoLineAggregationBuilder(this, factoriesBuilder, metaData);
@@ -116,7 +127,7 @@ protected MultiValuesSourceAggregatorFactory innerBuild(AggregationContext aggre
AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
return new GeoLineAggregatorFactory(name, configs, format, aggregationContext, parent, subFactoriesBuilder, metadata,
- includeSort, sortOrder);
+ includeSort, sortOrder, size);
}
public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index a409dbb84947d..737282da3a3c7 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -32,9 +32,11 @@ final class GeoLineAggregator extends MetricsAggregator {
private final GeoLineBucketedSort.Extra extra;
private final boolean includeSorts;
private final SortOrder sortOrder;
+ private final int size;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
- Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder) throws IOException {
+ Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder,
+ int size) throws IOException {
super(name, context, parent, metaData);
this.valuesSources = valuesSources;
if (valuesSources != null) {
@@ -46,6 +48,7 @@ final class GeoLineAggregator extends MetricsAggregator {
}
this.includeSorts = includeSorts;
this.sortOrder = sortOrder;
+ this.size = size;
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
index 3ae6d0f4de826..b77d0a531457a 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -24,22 +24,24 @@ final class GeoLineAggregatorFactory extends MultiValuesSourceAggregatorFactory
private boolean includeSort;
private SortOrder sortOrder;
+ private int size;
GeoLineAggregatorFactory(String name,
Map configs,
DocValueFormat format, AggregationContext aggregationContext, AggregatorFactory parent,
AggregatorFactories.Builder subFactoriesBuilder,
- Map metaData, boolean includeSort, SortOrder sortOrder) throws IOException {
+ Map metaData, boolean includeSort, SortOrder sortOrder, int size) throws IOException {
super(name, configs, format, aggregationContext, parent, subFactoriesBuilder, metaData);
this.includeSort = includeSort;
this.sortOrder = sortOrder;
+ this.size = size;
}
@Override
protected Aggregator createUnmapped(SearchContext searchContext,
Aggregator parent,
Map metaData) throws IOException {
- return new GeoLineAggregator(name, null, searchContext, parent, metaData, includeSort, sortOrder);
+ return new GeoLineAggregator(name, null, searchContext, parent, metaData, includeSort, sortOrder, size);
}
@Override
@@ -51,7 +53,7 @@ protected Aggregator doCreateInternal(SearchContext searchContext,
Map metaData) throws IOException {
MultiValuesSource.AnyMultiValuesSource valuesSources =
new MultiValuesSource.AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
- return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort, sortOrder);
+ return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort, sortOrder, size);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index eec9ec88f9644..53cf04c86196c 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -88,8 +88,9 @@ public InternalAggregation reduce(List aggregations, Reduce
idx += 1;
}
}
+ // the final reduce should always be in ascending order
- new PathArraySorter(finalList, finalSortVals, sortOrder).sort();
+ new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder);
From 708be6216bb10f2fa33defa2fc26a8d30be89c63 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Fri, 30 Oct 2020 12:38:49 -0700
Subject: [PATCH 12/23] fix up some tests
---
.../GeoLineAggregationBuilder.java | 11 +--
.../GeoLineAggregationBuilderTests.java | 76 +++++++++++++++++++
.../aggregations/GeoLineAggregatorTests.java | 2 +-
.../rest-api-spec/test/60_geo_line.yml | 51 +++++++++++++
4 files changed, 134 insertions(+), 6 deletions(-)
create mode 100644 x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
create mode 100644 x-pack/plugin/spatial/src/yamlRestTest/resources/rest-api-spec/test/60_geo_line.yml
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index c64b57e79a8d1..f2961f3f008f4 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -91,8 +91,9 @@ public GeoLineAggregationBuilder sortOrder(SortOrder sortOrder) {
}
public GeoLineAggregationBuilder size(int size) {
- if (size > GeoLineAggregator.MAX_PATH_SIZE) {
- throw new IllegalArgumentException("invalid [size] value [" + size + "] must be <= " + GeoLineAggregator.MAX_PATH_SIZE);
+ if (size <= 0 || size > GeoLineAggregator.MAX_PATH_SIZE) {
+ throw new IllegalArgumentException("invalid [size] value [" + size + "] must be a positive integer <= "
+ + GeoLineAggregator.MAX_PATH_SIZE);
}
this.size = size;
return this;
@@ -130,9 +131,9 @@ protected MultiValuesSourceAggregatorFactory innerBuild(AggregationContext aggre
includeSort, sortOrder, size);
}
- public GeoLineAggregationBuilder value(MultiValuesSourceFieldConfig valueConfig) {
- valueConfig = Objects.requireNonNull(valueConfig, "Configuration for field [" + POINT_FIELD + "] cannot be null");
- field(POINT_FIELD.getPreferredName(), valueConfig);
+ public GeoLineAggregationBuilder point(MultiValuesSourceFieldConfig pointConfig) {
+ pointConfig = Objects.requireNonNull(pointConfig, "Configuration for field [" + POINT_FIELD + "] cannot be null");
+ field(POINT_FIELD.getPreferredName(), pointConfig);
return this;
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
new file mode 100644
index 0000000000000..910cf320b2534
--- /dev/null
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.aggregations.support.MultiValuesSourceFieldConfig;
+import org.elasticsearch.search.sort.SortOrder;
+import org.elasticsearch.test.AbstractSerializingTestCase;
+
+import java.io.IOException;
+
+import static org.hamcrest.Matchers.equalTo;
+
+public class GeoLineAggregationBuilderTests extends AbstractSerializingTestCase {
+
+ @Override
+ protected GeoLineAggregationBuilder doParseInstance(XContentParser parser) throws IOException {
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.START_OBJECT));
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.FIELD_NAME));
+ String name = parser.currentName();
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.START_OBJECT));
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.FIELD_NAME));
+ assertThat(parser.currentName(), equalTo(GeoLineAggregationBuilder.NAME));
+ GeoLineAggregationBuilder parsed = GeoLineAggregationBuilder.PARSER.apply(parser, name);
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.END_OBJECT));
+ assertThat(parser.nextToken(), equalTo(XContentParser.Token.END_OBJECT));
+ return parsed;
+ }
+
+ @Override
+ protected Writeable.Reader instanceReader() {
+ return GeoLineAggregationBuilder::new;
+ }
+
+ @Override
+ protected GeoLineAggregationBuilder createTestInstance() {
+ MultiValuesSourceFieldConfig pointConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName(randomAlphaOfLength(5))
+ .build();
+ MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName(randomAlphaOfLength(6)).build();
+ GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
+ .point(pointConfig)
+ .sort(sortConfig);
+ if (randomBoolean()) {
+ SortOrder sortOrder = randomFrom(SortOrder.values());
+ lineAggregationBuilder.sortOrder(sortOrder);
+ }
+ if (randomBoolean()) {
+ lineAggregationBuilder.size(randomIntBetween(1, GeoLineAggregator.MAX_PATH_SIZE));
+ }
+ if (randomBoolean()) {
+ lineAggregationBuilder.includeSort(randomBoolean());
+ }
+ return lineAggregationBuilder;
+ }
+
+ public void testInvalidSize() {
+ MultiValuesSourceFieldConfig pointConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName(randomAlphaOfLength(5))
+ .build();
+ MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName(randomAlphaOfLength(6)).build();
+ GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
+ .point(pointConfig)
+ .sort(sortConfig);
+ expectThrows(IllegalArgumentException.class, () -> lineAggregationBuilder.size(0));
+ expectThrows(IllegalArgumentException.class,
+ () -> lineAggregationBuilder.size(GeoLineAggregator.MAX_PATH_SIZE + randomIntBetween(1, 10)));
+ }
+}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 15a2992d466c5..1d27e720f8913 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -62,7 +62,7 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
.build();
MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("sort_field").build();
GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
- .value(valueConfig)
+ .point(valueConfig)
.sortOrder(sortOrder)
.sort(sortConfig);
TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name")
diff --git a/x-pack/plugin/spatial/src/yamlRestTest/resources/rest-api-spec/test/60_geo_line.yml b/x-pack/plugin/spatial/src/yamlRestTest/resources/rest-api-spec/test/60_geo_line.yml
new file mode 100644
index 0000000000000..f156e4db9586f
--- /dev/null
+++ b/x-pack/plugin/spatial/src/yamlRestTest/resources/rest-api-spec/test/60_geo_line.yml
@@ -0,0 +1,51 @@
+---
+"Test geo_line aggregation on geo points":
+ - do:
+ indices.create:
+ index: races
+ body:
+ mappings:
+ properties:
+ race_id:
+ type: keyword
+ position:
+ type: geo_point
+
+ - do:
+ bulk:
+ refresh: true
+ body:
+ - index:
+ _index: races
+ _id: 1
+ - '{"position": "POINT(4.912350 52.374081)", "race_id": "Amsterdam", "timestamp": 4}'
+ - index:
+ _index: races
+ _id: 2
+ - '{"position": "POINT(4.901618 52.369219)", "race_id": "Amsterdam", "timestamp": 3}'
+ - index:
+ _index: races
+ _id: 3
+ - '{"position": "POINT(4.914722 52.371667)", "race_id": "Amsterdam", "timestamp": 10}'
+
+ - do:
+ search:
+ rest_total_hits_as_int: true
+ index: races
+ size: 0
+ body:
+ aggs:
+ trace:
+ geo_line:
+ point:
+ field: position
+ sort:
+ field: timestamp
+ - match: { hits.total: 3 }
+ - match: { aggregations.trace.type: "Feature" }
+ - match: { aggregations.trace.geometry.type: "LineString" }
+ - length: { aggregations.trace.geometry.coordinates: 3 }
+ - match: { aggregations.trace.geometry.coordinates.0: [4.901618, 52.369219] }
+ - match: { aggregations.trace.geometry.coordinates.1: [4.91235, 52.374081] }
+ - match: { aggregations.trace.geometry.coordinates.2: [4.914722, 52.371667] }
+ - is_true: aggregations.trace.properties.complete
From 6a3ebae32bfa2678742fbdc9677b0d1211099477 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Fri, 30 Oct 2020 13:23:21 -0700
Subject: [PATCH 13/23] fix final reduction
---
.../search/aggregations/InternalGeoLine.java | 7 ++++---
.../search/aggregations/GeoLineAggregatorTests.java | 13 +++++++++++--
2 files changed, 15 insertions(+), 5 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 53cf04c86196c..7cae523b6b86a 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -89,8 +89,9 @@ public InternalAggregation reduce(List aggregations, Reduce
}
}
// the final reduce should always be in ascending order
-
- new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
+ if (reduceContext.isFinalReduce()) {
+ new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
+ }
long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder);
@@ -98,7 +99,7 @@ public InternalAggregation reduce(List aggregations, Reduce
@Override
protected boolean mustReduceOnSingleInternalAgg() {
- return false;
+ return true;
}
@Override
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 1d27e720f8913..8a9fec0ce18f2 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -51,8 +51,11 @@ protected List getSearchPlugins() {
}
- public void testAggregator() throws IOException {
+ public void testAscending() throws IOException {
testAggregator(SortOrder.ASC);
+ }
+
+ public void testDescending() throws IOException {
testAggregator(SortOrder.DESC);
}
@@ -87,8 +90,14 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
sortValues[i] = SortOrder.ASC.equals(sortOrder) ? i : numPoints - i;
}
int lineSize = Math.min(numPoints, GeoLineAggregator.MAX_PATH_SIZE);
+
+ // re-sort line to be ascending
+ long[] linePoints = Arrays.copyOf(points, lineSize);
+ double[] lineSorts = Arrays.copyOf(sortValues, lineSize);
+ new PathArraySorter(linePoints, lineSorts, SortOrder.ASC).sort();
+
lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
- Arrays.copyOf(points, lineSize), Arrays.copyOf(sortValues, lineSize), null, complete, true, sortOrder));
+ linePoints, lineSorts, null, complete, true, sortOrder));
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints - 1);
From 517420458389d6698875fbcf2883df8f86a3d3ae Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 2 Nov 2020 15:53:11 -0800
Subject: [PATCH 14/23] add optional [size] param
---
.../aggregations/GeoLineAggregationBuilder.java | 7 ++++---
.../search/aggregations/GeoLineAggregator.java | 5 +----
.../search/aggregations/GeoLineBucketedSort.java | 2 ++
.../spatial/search/aggregations/InternalGeoLine.java | 6 +++---
.../aggregations/GeoLineAggregationBuilderTests.java | 4 ++--
.../search/aggregations/GeoLineAggregatorTests.java | 11 ++++++-----
.../search/aggregations/InternalGeoLineTests.java | 4 ++--
7 files changed, 20 insertions(+), 19 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index f2961f3f008f4..d25114d175448 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -56,7 +56,8 @@ public class GeoLineAggregationBuilder
private boolean includeSort;
private SortOrder sortOrder = SortOrder.ASC;
- private int size = GeoLineAggregator.MAX_PATH_SIZE;
+ private int size = MAX_PATH_SIZE;
+ static final int MAX_PATH_SIZE = 10000;
public static void registerUsage(ValuesSourceRegistry.Builder builder) {
builder.registerUsage(NAME, CoreValuesSourceType.GEOPOINT);
@@ -91,9 +92,9 @@ public GeoLineAggregationBuilder sortOrder(SortOrder sortOrder) {
}
public GeoLineAggregationBuilder size(int size) {
- if (size <= 0 || size > GeoLineAggregator.MAX_PATH_SIZE) {
+ if (size <= 0 || size > MAX_PATH_SIZE) {
throw new IllegalArgumentException("invalid [size] value [" + size + "] must be a positive integer <= "
- + GeoLineAggregator.MAX_PATH_SIZE);
+ + MAX_PATH_SIZE);
}
this.size = size;
return this;
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 737282da3a3c7..cfec31207754d 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -26,13 +26,11 @@
final class GeoLineAggregator extends MetricsAggregator {
/** Multiple ValuesSource with field names */
private final MultiValuesSource.AnyMultiValuesSource valuesSources;
- static final int MAX_PATH_SIZE = 10000;
private final GeoLineBucketedSort sort;
private final GeoLineBucketedSort.Extra extra;
private final boolean includeSorts;
private final SortOrder sortOrder;
- private final int size;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder,
@@ -41,14 +39,13 @@ final class GeoLineAggregator extends MetricsAggregator {
this.valuesSources = valuesSources;
if (valuesSources != null) {
this.extra = new GeoLineBucketedSort.Extra(context.bigArrays(), valuesSources);
- this.sort = new GeoLineBucketedSort(context.bigArrays(), sortOrder, null, MAX_PATH_SIZE, valuesSources, extra);
+ this.sort = new GeoLineBucketedSort(context.bigArrays(), sortOrder, null, size, valuesSources, extra);
} else {
this.extra = null;
this.sort = null;
}
this.includeSorts = includeSorts;
this.sortOrder = sortOrder;
- this.size = size;
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
index 425f70d6edb02..f301ae00510ff 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
@@ -91,6 +91,8 @@ protected boolean advanceExact(int doc) throws IOException {
assert docSortValues.docValueCount() == 1;
docValue = docSortValues.nextValue();
return true;
+ } else {
+ docValue = Long.MIN_VALUE;
}
return false;
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 7cae523b6b86a..10ba864b833c0 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -75,7 +75,7 @@ public InternalAggregation reduce(List aggregations, Reduce
includeSorts &= geoLine.includeSorts;
}
- complete &= mergedSize <= GeoLineAggregator.MAX_PATH_SIZE;
+ complete &= mergedSize <= GeoLineAggregationBuilder.MAX_PATH_SIZE;
long[] finalList = new long[mergedSize];
double[] finalSortVals = new double[mergedSize];
@@ -92,8 +92,8 @@ public InternalAggregation reduce(List aggregations, Reduce
if (reduceContext.isFinalReduce()) {
new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
}
- long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
- double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregator.MAX_PATH_SIZE, mergedSize));
+ long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregationBuilder.MAX_PATH_SIZE, mergedSize));
+ double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregationBuilder.MAX_PATH_SIZE, mergedSize));
return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder);
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
index 910cf320b2534..25aba6129501a 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilderTests.java
@@ -52,7 +52,7 @@ protected GeoLineAggregationBuilder createTestInstance() {
lineAggregationBuilder.sortOrder(sortOrder);
}
if (randomBoolean()) {
- lineAggregationBuilder.size(randomIntBetween(1, GeoLineAggregator.MAX_PATH_SIZE));
+ lineAggregationBuilder.size(randomIntBetween(1, GeoLineAggregationBuilder.MAX_PATH_SIZE));
}
if (randomBoolean()) {
lineAggregationBuilder.includeSort(randomBoolean());
@@ -71,6 +71,6 @@ public void testInvalidSize() {
.sort(sortConfig);
expectThrows(IllegalArgumentException.class, () -> lineAggregationBuilder.size(0));
expectThrows(IllegalArgumentException.class,
- () -> lineAggregationBuilder.size(GeoLineAggregator.MAX_PATH_SIZE + randomIntBetween(1, 10)));
+ () -> lineAggregationBuilder.size(GeoLineAggregationBuilder.MAX_PATH_SIZE + randomIntBetween(1, 10)));
}
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 8a9fec0ce18f2..f8cbeeb7c6567 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -60,6 +60,7 @@ public void testDescending() throws IOException {
}
private void testAggregator(SortOrder sortOrder) throws IOException {
+ int size = randomIntBetween(1, GeoLineAggregationBuilder.MAX_PATH_SIZE);
MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder()
.setFieldName("value_field")
.build();
@@ -67,7 +68,8 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
.point(valueConfig)
.sortOrder(sortOrder)
- .sort(sortConfig);
+ .sort(sortConfig)
+ .size(size);
TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name")
.field("group_id")
.subAggregation(lineAggregationBuilder);
@@ -77,8 +79,8 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
Map indexedPoints = new HashMap<>(numGroups);
Map indexedSortValues = new HashMap<>(numGroups);
for (int groupOrd = 0; groupOrd < numGroups; groupOrd++) {
- int numPoints = randomIntBetween(2, 20000);
- boolean complete = numPoints < GeoLineAggregator.MAX_PATH_SIZE;
+ int numPoints = randomIntBetween(2, 2 * size);
+ boolean complete = numPoints <= size;
long[] points = new long[numPoints];
double[] sortValues = new double[numPoints];
for (int i = 0; i < numPoints; i++) {
@@ -89,8 +91,7 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
points[i] = lonLat;
sortValues[i] = SortOrder.ASC.equals(sortOrder) ? i : numPoints - i;
}
- int lineSize = Math.min(numPoints, GeoLineAggregator.MAX_PATH_SIZE);
-
+ int lineSize = Math.min(numPoints, size);
// re-sort line to be ascending
long[] linePoints = Arrays.copyOf(points, lineSize);
double[] lineSorts = Arrays.copyOf(sortValues, lineSize);
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
index 887f914be1200..e43ed4714e08e 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -33,14 +33,14 @@ protected SearchPlugin registerPlugin() {
@Override
protected InternalGeoLine createTestInstance(String name, Map metadata) {
- int length = randomIntBetween(2, 2 * GeoLineAggregator.MAX_PATH_SIZE);
+ int length = randomIntBetween(2, 2 * GeoLineAggregationBuilder.MAX_PATH_SIZE);
long[] points = new long[length];
double[] sortVals = new double[length];
for (int i = 0; i < length; i++) {
points[i] = i;
sortVals[i] = i;
}
- boolean complete = length <= GeoLineAggregator.MAX_PATH_SIZE;
+ boolean complete = length <= GeoLineAggregationBuilder.MAX_PATH_SIZE;
return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), randomFrom(SortOrder.values()));
}
From a5c32cef36909ebe433c2cf53781b1b8ef929a3a Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Tue, 3 Nov 2020 16:10:24 -0800
Subject: [PATCH 15/23] fix tests
---
.../GeoLineAggregationBuilder.java | 2 ++
.../aggregations/GeoLineAggregator.java | 6 +++--
.../search/aggregations/InternalGeoLine.java | 23 ++++++++++++------
.../aggregations/GeoLineAggregatorTests.java | 4 ++--
.../aggregations/InternalGeoLineTests.java | 24 +++++++++++++------
5 files changed, 41 insertions(+), 18 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
index d25114d175448..30e87ea5f6b05 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregationBuilder.java
@@ -79,6 +79,7 @@ public GeoLineAggregationBuilder(StreamInput in) throws IOException {
super(in);
sortOrder = SortOrder.readFromStream(in);
includeSort = in.readBoolean();
+ size = in.readVInt();
}
public GeoLineAggregationBuilder includeSort(boolean includeSort) {
@@ -114,6 +115,7 @@ public BucketCardinality bucketCardinality() {
protected void innerWriteTo(StreamOutput out) throws IOException {
sortOrder.writeTo(out);
out.writeBoolean(includeSort);
+ out.writeVInt(size);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index cfec31207754d..c61c386967347 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -31,6 +31,7 @@ final class GeoLineAggregator extends MetricsAggregator {
private final GeoLineBucketedSort.Extra extra;
private final boolean includeSorts;
private final SortOrder sortOrder;
+ private final int size;
GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder,
@@ -46,6 +47,7 @@ final class GeoLineAggregator extends MetricsAggregator {
}
this.includeSorts = includeSorts;
this.sortOrder = sortOrder;
+ this.size = size;
}
@Override
@@ -81,12 +83,12 @@ public InternalAggregation buildAggregation(long bucket) {
double[] sortVals = sort.getSortValues(bucket);
long[] bucketLine = sort.getPoints(bucket);
new PathArraySorter(bucketLine, sortVals, sortOrder).sort();
- return new InternalGeoLine(name, bucketLine, sortVals, metadata(), complete, includeSorts, sortOrder);
+ return new InternalGeoLine(name, bucketLine, sortVals, metadata(), complete, includeSorts, sortOrder, size);
}
@Override
public InternalAggregation buildEmptyAggregation() {
- return new InternalGeoLine(name, null, null, metadata(), true, includeSorts, sortOrder);
+ return new InternalGeoLine(name, null, null, metadata(), true, includeSorts, sortOrder, size);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 10ba864b833c0..b2166fb03f0aa 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -31,15 +31,17 @@ public class InternalGeoLine extends InternalAggregation {
private boolean complete;
private boolean includeSorts;
private SortOrder sortOrder;
+ private int size;
InternalGeoLine(String name, long[] line, double[] sortVals, Map metadata, boolean complete,
- boolean includeSorts, SortOrder sortOrder) {
+ boolean includeSorts, SortOrder sortOrder, int size) {
super(name, metadata);
this.line = line;
this.sortVals = sortVals;
this.complete = complete;
this.includeSorts = includeSorts;
this.sortOrder = sortOrder;
+ this.size = size;
}
/**
@@ -52,6 +54,7 @@ public InternalGeoLine(StreamInput in) throws IOException {
this.complete = in.readBoolean();
this.includeSorts = in.readBoolean();
this.sortOrder = SortOrder.readFromStream(in);
+ this.size = in.readVInt();
}
@Override
@@ -61,6 +64,7 @@ protected void doWriteTo(StreamOutput out) throws IOException {
out.writeBoolean(complete);
out.writeBoolean(includeSorts);
sortOrder.writeTo(out);
+ out.writeVInt(size);
}
@Override
@@ -75,7 +79,7 @@ public InternalAggregation reduce(List aggregations, Reduce
includeSorts &= geoLine.includeSorts;
}
- complete &= mergedSize <= GeoLineAggregationBuilder.MAX_PATH_SIZE;
+ complete &= mergedSize <= size;
long[] finalList = new long[mergedSize];
double[] finalSortVals = new double[mergedSize];
@@ -92,9 +96,9 @@ public InternalAggregation reduce(List aggregations, Reduce
if (reduceContext.isFinalReduce()) {
new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
}
- long[] finalCappedList = Arrays.copyOf(finalList, Math.min(GeoLineAggregationBuilder.MAX_PATH_SIZE, mergedSize));
- double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(GeoLineAggregationBuilder.MAX_PATH_SIZE, mergedSize));
- return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder);
+ long[] finalCappedList = Arrays.copyOf(finalList, Math.min(size, mergedSize));
+ double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(size, mergedSize));
+ return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder, size);
}
@Override
@@ -131,6 +135,10 @@ public SortOrder sortOrder() {
return sortOrder;
}
+ public int size() {
+ return size;
+ }
+
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
final List coordinates = new ArrayList<>();
@@ -179,7 +187,7 @@ public String toString() {
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), Arrays.hashCode(line), Arrays.hashCode(sortVals), complete, includeSorts, sortOrder);
+ return Objects.hash(super.hashCode(), Arrays.hashCode(line), Arrays.hashCode(sortVals), complete, includeSorts, sortOrder, size);
}
@Override
@@ -194,7 +202,8 @@ public boolean equals(Object obj) {
&& Arrays.equals(sortVals, that.sortVals)
&& Objects.equals(complete, that.complete)
&& Objects.equals(includeSorts, that.includeSorts)
- && Objects.equals(sortOrder, that.sortOrder);
+ && Objects.equals(sortOrder, that.sortOrder)
+ && Objects.equals(size, that.size);
}
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index f8cbeeb7c6567..460595cd807ad 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -74,7 +74,7 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
.field("group_id")
.subAggregation(lineAggregationBuilder);
- int numGroups = randomIntBetween(1, 4);
+ int numGroups = randomIntBetween(1, 2);
Map lines = new HashMap<>(numGroups);
Map indexedPoints = new HashMap<>(numGroups);
Map indexedSortValues = new HashMap<>(numGroups);
@@ -98,7 +98,7 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
new PathArraySorter(linePoints, lineSorts, SortOrder.ASC).sort();
lines.put(String.valueOf(groupOrd), new InternalGeoLine("_name",
- linePoints, lineSorts, null, complete, true, sortOrder));
+ linePoints, lineSorts, null, complete, true, sortOrder, size));
for (int i = 0; i < randomIntBetween(1, numPoints); i++) {
int idx1 = randomIntBetween(0, numPoints - 1);
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
index e43ed4714e08e..6b2bcf7536125 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -23,6 +23,7 @@
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
+import static org.hamcrest.Matchers.equalTo;
public class InternalGeoLineTests extends InternalAggregationTestCase {
@@ -33,15 +34,16 @@ protected SearchPlugin registerPlugin() {
@Override
protected InternalGeoLine createTestInstance(String name, Map metadata) {
- int length = randomIntBetween(2, 2 * GeoLineAggregationBuilder.MAX_PATH_SIZE);
+ int length = randomIntBetween(2, GeoLineAggregationBuilder.MAX_PATH_SIZE);
long[] points = new long[length];
double[] sortVals = new double[length];
for (int i = 0; i < length; i++) {
points[i] = i;
sortVals[i] = i;
}
- boolean complete = length <= GeoLineAggregationBuilder.MAX_PATH_SIZE;
- return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), randomFrom(SortOrder.values()));
+ int size = randomIntBetween(length, GeoLineAggregationBuilder.MAX_PATH_SIZE);
+ boolean complete = length <= size;
+ return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), randomFrom(SortOrder.values()), size);
}
@Override
@@ -53,7 +55,8 @@ protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
boolean complete = instance.isComplete();
boolean includeSorts = instance.includeSorts();
SortOrder sortOrder = instance.sortOrder();
- switch (randomIntBetween(0, 6)) {
+ int size = instance.size();
+ switch (randomIntBetween(0, 7)) {
case 0:
name += randomAlphaOfLength(5);
break;
@@ -80,10 +83,13 @@ protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
case 6:
sortOrder = SortOrder.ASC.equals(sortOrder) ? SortOrder.DESC : SortOrder.ASC;
break;
+ case 7:
+ size = size + 1;
+ break;
default:
throw new AssertionError("Illegal randomisation branch");
}
- return new InternalGeoLine(name, line, sortVals, metadata, complete, includeSorts, sortOrder);
+ return new InternalGeoLine(name, line, sortVals, metadata, complete, includeSorts, sortOrder, size);
}
@Override
@@ -93,8 +99,12 @@ protected List randomResultsToReduce(String name, int size) {
@Override
protected void assertReduced(InternalGeoLine reduced, List inputs) {
- // TODO(talevy)
- // assert final line is sorted
+ int reducedLength = 0;
+ for (InternalGeoLine subLine : inputs) {
+ reducedLength += subLine.length();
+ }
+ int expectedReducedLength = Math.min(reducedLength, reduced.size());
+ assertThat(reduced.length(), equalTo(expectedReducedLength));
}
@Override
From c256eb99dd8e1344fd6b8b1adb8434474c337604 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 9 Nov 2020 19:19:23 -0800
Subject: [PATCH 16/23] move AnyMultiValueSource to GeoLineMultiValueSource
---
.../support/MultiValuesSource.java | 42 -------------
.../aggregations/GeoLineAggregator.java | 6 +-
.../GeoLineAggregatorFactory.java | 6 +-
.../aggregations/GeoLineBucketedSort.java | 10 ++--
.../support/GeoLineMultiValuesSource.java | 60 +++++++++++++++++++
5 files changed, 71 insertions(+), 53 deletions(-)
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoLineMultiValuesSource.java
diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
index 8a92f5bf54672..3d1d444c17ca1 100644
--- a/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
+++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/MultiValuesSource.java
@@ -19,9 +19,7 @@
package org.elasticsearch.search.aggregations.support;
import org.apache.lucene.index.LeafReaderContext;
-import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
-import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import java.io.IOException;
@@ -57,46 +55,6 @@ public SortedNumericDoubleValues getField(String fieldName, LeafReaderContext ct
}
}
- public static class AnyMultiValuesSource extends MultiValuesSource {
- public AnyMultiValuesSource(Map valuesSourceConfigs, QueryShardContext context) {
- values = new HashMap<>(valuesSourceConfigs.size());
- for (Map.Entry entry : valuesSourceConfigs.entrySet()) {
- final ValuesSource valuesSource = entry.getValue().getValuesSource();
- if (valuesSource instanceof ValuesSource.Numeric == false
- && valuesSource instanceof ValuesSource.GeoPoint == false) {
- throw new AggregationExecutionException("ValuesSource type " + valuesSource.toString() +
- "is not supported for multi-valued aggregation");
- }
- values.put(entry.getKey(), valuesSource);
- }
- }
-
- private ValuesSource getField(String fieldName) {
- ValuesSource valuesSource = values.get(fieldName);
- if (valuesSource == null) {
- throw new IllegalArgumentException("Could not find field name [" + fieldName + "] in multiValuesSource");
- }
- return valuesSource;
- }
-
- public SortedNumericDoubleValues getNumericField(String fieldName, LeafReaderContext ctx) throws IOException {
- ValuesSource valuesSource = getField(fieldName);
- if (valuesSource instanceof ValuesSource.Numeric) {
- return ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
- }
- throw new IllegalArgumentException("field [" + fieldName + "] is not a numeric type");
- }
-
- public MultiGeoPointValues getGeoPointField(String fieldName, LeafReaderContext ctx) {
- ValuesSource valuesSource = getField(fieldName);
- if (valuesSource instanceof ValuesSource.GeoPoint) {
- return ((ValuesSource.GeoPoint) valuesSource).geoPointValues(ctx);
- }
- throw new IllegalArgumentException("field [" + fieldName + "] is not a geo_point type");
- }
-
- }
-
public boolean needsScores() {
return values.values().stream().anyMatch(ValuesSource::needsScores);
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index c61c386967347..7ab9f6267c025 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -12,7 +12,7 @@
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
import org.elasticsearch.search.aggregations.metrics.MetricsAggregator;
-import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoLineMultiValuesSource;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.sort.BucketedSort;
import org.elasticsearch.search.sort.SortOrder;
@@ -25,7 +25,7 @@
**/
final class GeoLineAggregator extends MetricsAggregator {
/** Multiple ValuesSource with field names */
- private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+ private final GeoLineMultiValuesSource valuesSources;
private final GeoLineBucketedSort sort;
private final GeoLineBucketedSort.Extra extra;
@@ -33,7 +33,7 @@ final class GeoLineAggregator extends MetricsAggregator {
private final SortOrder sortOrder;
private final int size;
- GeoLineAggregator(String name, MultiValuesSource.AnyMultiValuesSource valuesSources, SearchContext context,
+ GeoLineAggregator(String name, GeoLineMultiValuesSource valuesSources, SearchContext context,
Aggregator parent, Map metaData, boolean includeSorts, SortOrder sortOrder,
int size) throws IOException {
super(name, context, parent, metaData);
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
index b77d0a531457a..a2778a97f9a8c 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorFactory.java
@@ -11,7 +11,7 @@
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.CardinalityUpperBound;
import org.elasticsearch.search.aggregations.support.AggregationContext;
-import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoLineMultiValuesSource;
import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
@@ -51,8 +51,8 @@ protected Aggregator doCreateInternal(SearchContext searchContext,
Aggregator parent,
CardinalityUpperBound cardinality,
Map metaData) throws IOException {
- MultiValuesSource.AnyMultiValuesSource valuesSources =
- new MultiValuesSource.AnyMultiValuesSource(configs, searchContext.getQueryShardContext());
+ GeoLineMultiValuesSource valuesSources =
+ new GeoLineMultiValuesSource(configs, searchContext.getQueryShardContext());
return new GeoLineAggregator(name, valuesSources, searchContext, parent, metaData, includeSort, sortOrder, size);
}
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
index f301ae00510ff..84b49bde1d102 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
@@ -18,7 +18,7 @@
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
-import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.xpack.spatial.search.aggregations.support.GeoLineMultiValuesSource;
import org.elasticsearch.search.sort.BucketedSort;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.xpack.core.common.search.aggregations.MissingHelper;
@@ -28,10 +28,10 @@
import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
public class GeoLineBucketedSort extends BucketedSort.ForDoubles {
- private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+ private final GeoLineMultiValuesSource valuesSources;
public GeoLineBucketedSort(BigArrays bigArrays, SortOrder sortOrder, DocValueFormat format, int bucketSize,
- MultiValuesSource.AnyMultiValuesSource valuesSources, GeoLineBucketedSort.Extra extra) {
+ GeoLineMultiValuesSource valuesSources, GeoLineBucketedSort.Extra extra) {
super(bigArrays, sortOrder, format, bucketSize, extra);
this.valuesSources = valuesSources;
}
@@ -107,11 +107,11 @@ protected double docValue() {
static class Extra implements BucketedSort.ExtraData, Releasable {
private final BigArrays bigArrays;
- private final MultiValuesSource.AnyMultiValuesSource valuesSources;
+ private final GeoLineMultiValuesSource valuesSources;
private LongArray values;
private final MissingHelper empty;
- Extra(BigArrays bigArrays, MultiValuesSource.AnyMultiValuesSource valuesSources) {
+ Extra(BigArrays bigArrays, GeoLineMultiValuesSource valuesSources) {
this.bigArrays = bigArrays;
this.valuesSources = valuesSources;
this.values = bigArrays.newLongArray(1, false);
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoLineMultiValuesSource.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoLineMultiValuesSource.java
new file mode 100644
index 0000000000000..6371d6f95f87e
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/support/GeoLineMultiValuesSource.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+
+package org.elasticsearch.xpack.spatial.search.aggregations.support;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
+import org.elasticsearch.index.query.QueryShardContext;
+import org.elasticsearch.search.aggregations.AggregationExecutionException;
+import org.elasticsearch.search.aggregations.support.MultiValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class GeoLineMultiValuesSource extends MultiValuesSource {
+ public GeoLineMultiValuesSource(Map valuesSourceConfigs, QueryShardContext context) {
+ values = new HashMap<>(valuesSourceConfigs.size());
+ for (Map.Entry entry : valuesSourceConfigs.entrySet()) {
+ final ValuesSource valuesSource = entry.getValue().getValuesSource();
+ if (valuesSource instanceof ValuesSource.Numeric == false
+ && valuesSource instanceof ValuesSource.GeoPoint == false) {
+ throw new AggregationExecutionException("ValuesSource type " + valuesSource.toString() +
+ "is not supported for multi-valued aggregation");
+ }
+ values.put(entry.getKey(), valuesSource);
+ }
+ }
+
+ private ValuesSource getField(String fieldName) {
+ ValuesSource valuesSource = values.get(fieldName);
+ if (valuesSource == null) {
+ throw new IllegalArgumentException("Could not find field name [" + fieldName + "] in multiValuesSource");
+ }
+ return valuesSource;
+ }
+
+ public SortedNumericDoubleValues getNumericField(String fieldName, LeafReaderContext ctx) throws IOException {
+ ValuesSource valuesSource = getField(fieldName);
+ if (valuesSource instanceof ValuesSource.Numeric) {
+ return ((ValuesSource.Numeric) valuesSource).doubleValues(ctx);
+ }
+ throw new IllegalArgumentException("field [" + fieldName + "] is not a numeric type");
+ }
+
+ public MultiGeoPointValues getGeoPointField(String fieldName, LeafReaderContext ctx) {
+ ValuesSource valuesSource = getField(fieldName);
+ if (valuesSource instanceof ValuesSource.GeoPoint) {
+ return ((ValuesSource.GeoPoint) valuesSource).geoPointValues(ctx);
+ }
+ throw new IllegalArgumentException("field [" + fieldName + "] is not a geo_point type");
+ }
+
+}
From 8fa5b91fada6c440cd7dfa18bcc30e20fc23b30c Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 16 Nov 2020 14:58:55 -0800
Subject: [PATCH 17/23] use priority queue when reducing (wip - broken)
---
.../aggregations/GeoLineAggregator.java | 1 +
.../aggregations/GeoLineBucketedSort.java | 12 ++
.../search/aggregations/InternalGeoLine.java | 26 +--
.../search/aggregations/MergedGeoLines.java | 169 ++++++++++++++++++
.../aggregations/GeoLineAggregatorTests.java | 2 +-
.../aggregations/InternalGeoLineTests.java | 71 ++++++--
.../aggregations/MergedGeoLinesTests.java | 49 +++++
7 files changed, 299 insertions(+), 31 deletions(-)
create mode 100644 x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
create mode 100644 x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
index 7ab9f6267c025..3417883f48465 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregator.java
@@ -80,6 +80,7 @@ public InternalAggregation buildAggregation(long bucket) {
return buildEmptyAggregation();
}
boolean complete = sort.inHeapMode(bucket) == false;
+ addRequestCircuitBreakerBytes((Double.SIZE + Long.SIZE) * sort.sizeOf(bucket));
double[] sortVals = sort.getSortValues(bucket);
long[] bucketLine = sort.getPoints(bucket);
new PathArraySorter(bucketLine, sortVals, sortOrder).sort();
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
index 84b49bde1d102..63f93deb24f86 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
@@ -36,6 +36,18 @@ public GeoLineBucketedSort(BigArrays bigArrays, SortOrder sortOrder, DocValueFor
this.valuesSources = valuesSources;
}
+ public long sizeOf(long bucket) {
+ int bucketSize = getBucketSize();
+ long rootIndex = bucket * bucketSize;
+ if (rootIndex >= values().size()) {
+ // We've never seen this bucket.
+ return 0;
+ }
+ long start = inHeapMode(bucket) ? rootIndex : (rootIndex + getNextGatherOffset(rootIndex) + 1);
+ long end = rootIndex + bucketSize;
+ return end - start;
+ }
+
public double[] getSortValues(long bucket) {
int bucketSize = getBucketSize();
long rootIndex = bucket * bucketSize;
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index b2166fb03f0aa..3e2080e2b588a 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -72,33 +72,25 @@ public InternalAggregation reduce(List aggregations, Reduce
int mergedSize = 0;
boolean complete = true;
boolean includeSorts = true;
+ List internalGeoLines = new ArrayList<>(aggregations.size());
for (InternalAggregation aggregation : aggregations) {
InternalGeoLine geoLine = (InternalGeoLine) aggregation;
+ internalGeoLines.add(geoLine);
mergedSize += geoLine.line.length;
complete &= geoLine.complete;
includeSorts &= geoLine.includeSorts;
}
-
complete &= mergedSize <= size;
+ int finalSize = Math.min(mergedSize, size);
- long[] finalList = new long[mergedSize];
- double[] finalSortVals = new double[mergedSize];
- int idx = 0;
- for (InternalAggregation aggregation : aggregations) {
- InternalGeoLine geoLine = (InternalGeoLine) aggregation;
- for (int i = 0; i < geoLine.line.length; i++) {
- finalSortVals[idx] = geoLine.sortVals[i];
- finalList[idx] = geoLine.line[i];
- idx += 1;
- }
- }
+ MergedGeoLines mergedGeoLines = new MergedGeoLines(internalGeoLines, finalSize, sortOrder);
+ mergedGeoLines.merge();
// the final reduce should always be in ascending order
- if (reduceContext.isFinalReduce()) {
- new PathArraySorter(finalList, finalSortVals, SortOrder.ASC).sort();
+ if (reduceContext.isFinalReduce() && SortOrder.DESC.equals(sortOrder)) {
+ new PathArraySorter(mergedGeoLines.getFinalPoints(), mergedGeoLines.getFinalSortValues(), SortOrder.ASC).sort();
}
- long[] finalCappedList = Arrays.copyOf(finalList, Math.min(size, mergedSize));
- double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(size, mergedSize));
- return new InternalGeoLine(name, finalCappedList, finalCappedSortVals, getMetadata(), complete, includeSorts, sortOrder, size);
+ return new InternalGeoLine(name, mergedGeoLines.getFinalPoints(), mergedGeoLines.getFinalSortValues(), getMetadata(), complete,
+ includeSorts, sortOrder, size);
}
@Override
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
new file mode 100644
index 0000000000000..2e04d25345eba
--- /dev/null
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
@@ -0,0 +1,169 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.search.sort.SortOrder;
+
+import java.util.List;
+
+/**
+ * Class to merge an arbitrary list of {@link InternalGeoLine} lines into a new line
+ * with the appropriate max length. The final point and sort values can be found in
+ * finalPoints and finalSortValues after merge is called.
+ */
+final class MergedGeoLines {
+
+ private final List geoLines;
+ private final int capacity;
+ private final SortOrder sortOrder;
+ private final int[] lineIndices; // index of which geoLine item represents
+ private final int[] idxsWithinLine; // index within the geoLine for the item
+ private int size;
+ private final long[] finalPoints;
+ private final double[] finalSortValues;
+
+ MergedGeoLines(List geoLines, int finalLength, SortOrder sortOrder) {
+ this.geoLines = geoLines;
+ this.capacity = geoLines.size();
+ this.sortOrder = sortOrder;
+ this.lineIndices = new int[capacity];
+ this.idxsWithinLine = new int[capacity];
+ this.size = 0;
+ this.finalPoints = new long[finalLength];
+ this.finalSortValues = new double[finalLength];
+ }
+
+ public long[] getFinalPoints() {
+ return finalPoints;
+ }
+
+ public double[] getFinalSortValues() {
+ return finalSortValues;
+ }
+
+ public void merge() {
+ // 1. add first element of each sub line to heap
+ for (int i = 0; i < geoLines.size(); i++) {
+ add(i, 0);
+ }
+
+ // 2. take lowest/greatest value from heap and re-insert the next value from the same sub-line that specific value was chosen from.
+
+ int i = 0;
+ while (i < finalPoints.length && size > 0) {
+ // take top from heap and place in finalLists
+ int lineIdx = lineIndices[0];
+ int idxInLine = idxsWithinLine[0];
+ finalPoints[i] = getTopPoint();
+ finalSortValues[i] = getTopSortValue();
+ removeTop();
+ InternalGeoLine lineChosen = geoLines.get(lineIdx);
+ if (idxInLine + 1 < lineChosen.line().length) {
+ add(lineIdx, idxInLine + 1);
+ }
+ i++;
+ }
+ }
+
+ private long getTopPoint() {
+ InternalGeoLine line = geoLines.get(lineIndices[0]);
+ return line.line()[idxsWithinLine[0]];
+ }
+
+ private double getTopSortValue() {
+ InternalGeoLine line = geoLines.get(lineIndices[0]);
+ return line.sortVals()[idxsWithinLine[0]];
+ }
+
+ private void removeTop() {
+ if (size == 0) {
+ throw new IllegalStateException();
+ }
+ lineIndices[0] = lineIndices[size - 1];
+ idxsWithinLine[0] = idxsWithinLine[size - 1];
+ size--;
+ heapifyDown();
+ }
+
+ private void add(int lineIndex, int idxWithinLine) {
+ if (size >= capacity) {
+ throw new IllegalStateException();
+ }
+ lineIndices[size] = lineIndex;
+ idxsWithinLine[size] = idxWithinLine;
+ size++;
+ heapifyUp();
+ }
+
+ private boolean correctOrdering(int i, int j) {
+ InternalGeoLine lineI = geoLines.get(lineIndices[i]);
+ InternalGeoLine lineJ = geoLines.get(lineIndices[j]);
+ double valI = lineI.sortVals()[idxsWithinLine[i]];
+ double valJ = lineJ.sortVals()[idxsWithinLine[j]];
+ if (SortOrder.ASC.equals(sortOrder)) {
+ return valI > valJ;
+ }
+ return valI < valJ;
+ }
+
+ private int getParentIndex(int i) {
+ return (i - 1) / 2;
+ }
+
+ private int getLeftChildIndex(int i) {
+ return 2 * i + 1;
+ }
+
+ private int getRightChildIndex(int i) {
+ return 2 * i + 2;
+ }
+
+ private boolean hasParent(int i) {
+ return i > 0;
+ }
+
+ private boolean hasLeftChild(int i) {
+ return getLeftChildIndex(i) < size;
+ }
+
+ private boolean hasRightChild(int i) {
+ return getRightChildIndex(i) < size;
+ }
+
+ private void heapifyUp() {
+ int i = size - 1;
+ while (hasParent(i) && correctOrdering(getParentIndex(i), i)) {
+ int parentIndex = getParentIndex(i);
+ swap(parentIndex, i);
+ i = parentIndex;
+ }
+ }
+
+ private void heapifyDown() {
+ int i = 0;
+ while (hasLeftChild(i)) {
+ int childIndex = getLeftChildIndex(i);
+ if (hasRightChild(i) && correctOrdering(getRightChildIndex(i), childIndex) == false) {
+ childIndex = getRightChildIndex(i);
+ }
+ if (correctOrdering(childIndex, i)) {
+ break;
+ } else {
+ swap(childIndex, i);
+ i = childIndex;
+ }
+ }
+ }
+
+ private void swap(int i, int j) {
+ int tmpLineIndex = lineIndices[i];
+ int tmpIdxWithinLine = idxsWithinLine[i];
+ lineIndices[i] = lineIndices[j];
+ idxsWithinLine[i] = idxsWithinLine[j];
+ lineIndices[j] = tmpLineIndex;
+ idxsWithinLine[j] = tmpIdxWithinLine;
+ }
+}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index 460595cd807ad..b279c3e5dd09b 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -138,7 +138,7 @@ private void testAggregator(SortOrder sortOrder) throws IOException {
for (int i = 0; i < geoLine.sortVals().length; i++) {
geoLine.sortVals()[i] = NumericUtils.sortableLongToDouble((long) geoLine.sortVals()[i]);
}
- assertArrayEquals(expectedGeoLine.sortVals(), geoLine.sortVals(), 0.0001);
+ assertArrayEquals(expectedGeoLine.sortVals(), geoLine.sortVals(), 0d);
assertArrayEquals(expectedGeoLine.line(), geoLine.line());
}
});
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
index 6b2bcf7536125..f542c212e4ac1 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -20,9 +20,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.stream.Stream;
-import static java.util.stream.Collectors.toList;
import static org.hamcrest.Matchers.equalTo;
public class InternalGeoLineTests extends InternalAggregationTestCase {
@@ -32,18 +30,32 @@ protected SearchPlugin registerPlugin() {
return new SpatialPlugin();
}
- @Override
- protected InternalGeoLine createTestInstance(String name, Map metadata) {
- int length = randomIntBetween(2, GeoLineAggregationBuilder.MAX_PATH_SIZE);
+ static InternalGeoLine randomInstance(String name, Map metadata, int size, double magicDecimal) {
+ int length = randomIntBetween(2, size);
+ SortOrder sortOrder = randomFrom(SortOrder.values());
long[] points = new long[length];
double[] sortVals = new double[length];
for (int i = 0; i < length; i++) {
- points[i] = i;
- sortVals[i] = i;
+ points[i] = randomNonNegativeLong();
+ sortVals[i] = randomIntBetween(1, 100) + magicDecimal;
+ }
+ Arrays.sort(sortVals);
+ if (SortOrder.DESC.equals(sortOrder)) {
+ // reverse the list
+ for (int i = 0, j = sortVals.length - 1; i < j; i++, j--) {
+ double tmp = sortVals[i];
+ sortVals[i] = sortVals[j];
+ sortVals[j] = tmp;
+ }
}
- int size = randomIntBetween(length, GeoLineAggregationBuilder.MAX_PATH_SIZE);
boolean complete = length <= size;
- return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), randomFrom(SortOrder.values()), size);
+ return new InternalGeoLine(name, points, sortVals, metadata, complete, randomBoolean(), sortOrder, size);
+ }
+
+ @Override
+ protected InternalGeoLine createTestInstance(String name, Map metadata) {
+ int size = randomIntBetween(10, GeoLineAggregationBuilder.MAX_PATH_SIZE);
+ return randomInstance(name, metadata, size, randomDoubleBetween(0, 1, false));
}
@Override
@@ -94,17 +106,50 @@ protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
@Override
protected List randomResultsToReduce(String name, int size) {
- return Stream.generate(() -> createTestInstance(name, null)).limit(size).collect(toList());
+ int maxLineLength = randomIntBetween(10, GeoLineAggregationBuilder.MAX_PATH_SIZE);
+ List instances = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ // use the magicDecimal to have absolute ordering between heap-sort and testing array sorting
+ instances.add(randomInstance(name, null, maxLineLength, ((double) i) / size));
+ }
+ return instances;
}
@Override
protected void assertReduced(InternalGeoLine reduced, List inputs) {
- int reducedLength = 0;
+ int mergedLength = 0;
for (InternalGeoLine subLine : inputs) {
- reducedLength += subLine.length();
+ mergedLength += subLine.length();
}
- int expectedReducedLength = Math.min(reducedLength, reduced.size());
+ boolean complete = mergedLength <= reduced.size();
+ int expectedReducedLength = Math.min(mergedLength, reduced.size());
assertThat(reduced.length(), equalTo(expectedReducedLength));
+ assertThat(complete, equalTo(reduced.isComplete()));
+
+ // check arrays
+ long[] finalList = new long[mergedLength];
+ double[] finalSortVals = new double[mergedLength];
+ int idx = 0;
+ for (InternalGeoLine geoLine : inputs) {
+ for (int i = 0; i < geoLine.line().length; i++) {
+ finalSortVals[idx] = geoLine.sortVals()[i];
+ finalList[idx] = geoLine.line()[i];
+ idx += 1;
+ }
+ }
+
+ new PathArraySorter(finalList, finalSortVals, reduced.sortOrder()).sort();
+
+ // cap to max length
+ long[] finalCappedPoints = Arrays.copyOf(finalList, Math.min(reduced.size(), mergedLength));
+ double[] finalCappedSortVals = Arrays.copyOf(finalSortVals, Math.min(reduced.size(), mergedLength));
+
+ if (SortOrder.DESC.equals(reduced.sortOrder())) {
+ new PathArraySorter(finalCappedPoints, finalCappedSortVals, SortOrder.ASC).sort();
+ }
+
+ assertArrayEquals(finalCappedSortVals, reduced.sortVals(), 0d);
+ assertArrayEquals(finalCappedPoints, reduced.line());
}
@Override
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
new file mode 100644
index 0000000000000..e99655bbe9a6b
--- /dev/null
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License;
+ * you may not use this file except in compliance with the Elastic License.
+ */
+package org.elasticsearch.xpack.spatial.search.aggregations;
+
+import org.elasticsearch.search.sort.SortOrder;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class MergedGeoLinesTests extends ESTestCase {
+
+ public InternalGeoLine randomLine(SortOrder sortOrder, int maxLength) {
+ String name = randomAlphaOfLength(5);
+ int length = randomBoolean() ? maxLength : randomIntBetween(1, maxLength);
+ boolean complete = length < maxLength;
+ long[] points = new long[length];
+ double[] sortValues = new double[length];
+ int randomIncrement = randomBoolean() ? randomIntBetween(1, 5) : 0;
+ for (int i = 0; i < length; i++) {
+ points[i] = randomIntBetween(1, 100);
+ sortValues[i] = (i + 1) * 2 + randomIncrement;
+ }
+ return new InternalGeoLine(name, points, sortValues, Collections.emptyMap(), complete, randomBoolean(), sortOrder, maxLength);
+ }
+
+ public void testSimpleMerge() {
+ int numLines = 100;
+ int maxLength = 500;
+ int finalLength = 0;
+ SortOrder sortOrder = SortOrder.ASC;
+ List geoLines = new ArrayList<>();
+ for (int i = 0; i < numLines; i++) {
+ geoLines.add(randomLine(sortOrder, maxLength));
+ finalLength += geoLines.get(i).length();
+ }
+ finalLength = Math.min(maxLength, finalLength);
+ MergedGeoLines mergedGeoLines = new MergedGeoLines(geoLines, finalLength, sortOrder);
+ mergedGeoLines.merge();
+ double[] sortedValues = Arrays.copyOf(mergedGeoLines.getFinalSortValues(), mergedGeoLines.getFinalSortValues().length);
+ Arrays.sort(sortedValues);
+ assertArrayEquals(sortedValues, mergedGeoLines.getFinalSortValues(), 0d);
+ }
+}
From 37bf8f6786d2a60313504156144dbdd0ac72b48d Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Tue, 17 Nov 2020 08:19:55 -0800
Subject: [PATCH 18/23] fix internalgeolinetests
---
.../aggregations/InternalGeoLineTests.java | 10 +++++-----
.../aggregations/MergedGeoLinesTests.java | 17 ++++++++++-------
2 files changed, 15 insertions(+), 12 deletions(-)
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
index f542c212e4ac1..230b8a92c76d0 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLineTests.java
@@ -30,14 +30,13 @@ protected SearchPlugin registerPlugin() {
return new SpatialPlugin();
}
- static InternalGeoLine randomInstance(String name, Map metadata, int size, double magicDecimal) {
+ static InternalGeoLine randomInstance(String name, Map metadata, int size, SortOrder sortOrder, double magicDecimal) {
int length = randomIntBetween(2, size);
- SortOrder sortOrder = randomFrom(SortOrder.values());
long[] points = new long[length];
double[] sortVals = new double[length];
for (int i = 0; i < length; i++) {
points[i] = randomNonNegativeLong();
- sortVals[i] = randomIntBetween(1, 100) + magicDecimal;
+ sortVals[i] = i + magicDecimal;
}
Arrays.sort(sortVals);
if (SortOrder.DESC.equals(sortOrder)) {
@@ -55,7 +54,7 @@ static InternalGeoLine randomInstance(String name, Map metadata,
@Override
protected InternalGeoLine createTestInstance(String name, Map metadata) {
int size = randomIntBetween(10, GeoLineAggregationBuilder.MAX_PATH_SIZE);
- return randomInstance(name, metadata, size, randomDoubleBetween(0, 1, false));
+ return randomInstance(name, metadata, size, randomFrom(SortOrder.values()), randomDoubleBetween(0, 1, false));
}
@Override
@@ -106,11 +105,12 @@ protected InternalGeoLine mutateInstance(InternalGeoLine instance) {
@Override
protected List randomResultsToReduce(String name, int size) {
+ SortOrder sortOrder = randomFrom(SortOrder.values());
int maxLineLength = randomIntBetween(10, GeoLineAggregationBuilder.MAX_PATH_SIZE);
List instances = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
// use the magicDecimal to have absolute ordering between heap-sort and testing array sorting
- instances.add(randomInstance(name, null, maxLineLength, ((double) i) / size));
+ instances.add(randomInstance(name, null, maxLineLength, sortOrder, ((double) i) / size));
}
return instances;
}
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
index e99655bbe9a6b..9fae79fe60b41 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLinesTests.java
@@ -15,35 +15,38 @@
public class MergedGeoLinesTests extends ESTestCase {
- public InternalGeoLine randomLine(SortOrder sortOrder, int maxLength) {
+ public InternalGeoLine randomLine(SortOrder sortOrder, int maxLength, double magicDecimal) {
String name = randomAlphaOfLength(5);
int length = randomBoolean() ? maxLength : randomIntBetween(1, maxLength);
boolean complete = length < maxLength;
long[] points = new long[length];
double[] sortValues = new double[length];
- int randomIncrement = randomBoolean() ? randomIntBetween(1, 5) : 0;
for (int i = 0; i < length; i++) {
points[i] = randomIntBetween(1, 100);
- sortValues[i] = (i + 1) * 2 + randomIncrement;
+ sortValues[i] = i + magicDecimal;
}
return new InternalGeoLine(name, points, sortValues, Collections.emptyMap(), complete, randomBoolean(), sortOrder, maxLength);
}
public void testSimpleMerge() {
- int numLines = 100;
- int maxLength = 500;
+ int numLines = 10;
+ int maxLength = 100;
int finalLength = 0;
SortOrder sortOrder = SortOrder.ASC;
List geoLines = new ArrayList<>();
for (int i = 0; i < numLines; i++) {
- geoLines.add(randomLine(sortOrder, maxLength));
+ geoLines.add(randomLine(sortOrder, maxLength, ((double) i) / numLines));
finalLength += geoLines.get(i).length();
}
finalLength = Math.min(maxLength, finalLength);
MergedGeoLines mergedGeoLines = new MergedGeoLines(geoLines, finalLength, sortOrder);
mergedGeoLines.merge();
+
+ // assert that the mergedGeoLines are sorted (does not necessarily validate correctness, but it is a good heuristic)
+ long[] sortedPoints = Arrays.copyOf(mergedGeoLines.getFinalPoints(), mergedGeoLines.getFinalPoints().length);
double[] sortedValues = Arrays.copyOf(mergedGeoLines.getFinalSortValues(), mergedGeoLines.getFinalSortValues().length);
- Arrays.sort(sortedValues);
+ new PathArraySorter(sortedPoints, sortedValues, sortOrder).sort();
assertArrayEquals(sortedValues, mergedGeoLines.getFinalSortValues(), 0d);
+ assertArrayEquals(sortedPoints, mergedGeoLines.getFinalPoints());
}
}
From be487a05d721d9e14da3eb94bc16608dd4b7505a Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Tue, 17 Nov 2020 16:15:23 -0800
Subject: [PATCH 19/23] cleanup and add docs
---
.../search/sort/BucketedSortTestCase.java | 2 +-
.../license/XPackLicenseState.java | 2 ++
.../aggregations/GeoLineBucketedSort.java | 20 +++++++++++++++++++
.../search/aggregations/InternalGeoLine.java | 13 ++++++++++++
.../search/aggregations/MergedGeoLines.java | 7 +++++--
.../search/aggregations/PathArraySorter.java | 3 +++
6 files changed, 44 insertions(+), 3 deletions(-)
diff --git a/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java b/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
index 9a2071e5ad01a..80d711cde0f3a 100644
--- a/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
+++ b/server/src/test/java/org/elasticsearch/search/sort/BucketedSortTestCase.java
@@ -212,7 +212,7 @@ public void testTwoHitsDesc() throws IOException {
assertThat(sort.getValues(0, extra.valueBuilder()), contains(extraValue(3000, 3), extraValue(200, 2)));
}
}
-
+
public void testTwoHitsAsc() throws IOException {
try (T sort = build(SortOrder.ASC, 2, new double[] {1, 2, 3})) {
BucketedSort.Leaf leaf = sort.forLeaf(null);
diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java
index 753ae5de3b44c..d2c7754b59a67 100644
--- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java
+++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/XPackLicenseState.java
@@ -96,6 +96,8 @@ public enum Feature {
SPATIAL_GEO_GRID(OperationMode.GOLD, true),
+ SPATIAL_GEO_LINE(OperationMode.GOLD, true),
+
ANALYTICS(OperationMode.MISSING, true),
SEARCHABLE_SNAPSHOTS(OperationMode.ENTERPRISE, true);
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
index 63f93deb24f86..b804b43a86ac2 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineBucketedSort.java
@@ -27,6 +27,13 @@
import static org.elasticsearch.xpack.spatial.search.aggregations.GeoLineAggregationBuilder.SORT_FIELD;
+/**
+ * A bigArrays sorter of both a geo_line's sort-values and points.
+ *
+ * This class accumulates geo_points within buckets and heapifies the
+ * bucket based on whether there are too many items in the bucket that
+ * need to be dropped based on their sort value.
+ */
public class GeoLineBucketedSort extends BucketedSort.ForDoubles {
private final GeoLineMultiValuesSource valuesSources;
@@ -48,6 +55,11 @@ public long sizeOf(long bucket) {
return end - start;
}
+ /**
+ * @param bucket the bucket ordinal
+ * @return the array of sort-values for the specific bucket. This array may not necessarily be heapified already, so no ordering is
+ * guaranteed.
+ */
public double[] getSortValues(long bucket) {
int bucketSize = getBucketSize();
long rootIndex = bucket * bucketSize;
@@ -66,6 +78,10 @@ public double[] getSortValues(long bucket) {
return result;
}
+ /**
+ * @param bucket the bucket ordinal
+ * @return the array of points, ordered by the their respective sort-value for the specific bucket.
+ */
public long[] getPoints(long bucket) {
int bucketSize = getBucketSize();
long rootIndex = bucket * bucketSize;
@@ -116,6 +132,10 @@ protected double docValue() {
};
}
+ /**
+ * An {@link BucketedSort.ExtraData} representing the geo-point for a document
+ * within a bucket.
+ */
static class Extra implements BucketedSort.ExtraData, Releasable {
private final BigArrays bigArrays;
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
index 3e2080e2b588a..2c150bfd54b6c 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/InternalGeoLine.java
@@ -33,6 +33,19 @@ public class InternalGeoLine extends InternalAggregation {
private SortOrder sortOrder;
private int size;
+ /**
+ * A geo_line representing the bucket for a {@link GeoLineAggregationBuilder}. The values of
line and sortVals
+ * are expected to be sorted using sortOrder.
+ *
+ * @param name the name of the aggregation
+ * @param line the ordered geo-points representing the line
+ * @param sortVals the ordered sort-values associated with the points in the line (e.g. timestamp)
+ * @param metadata the aggregation's metadata
+ * @param complete true iff the line is representative of all the points that fall within the bucket. False otherwise.
+ * @param includeSorts true iff the sort-values should be rendered in xContent as properties of the line-string. False otherwise.
+ * @param sortOrder the {@link SortOrder} for the line. Whether the points are to be plotted in asc or desc order
+ * @param size the max length of the line-string.
+ */
InternalGeoLine(String name, long[] line, double[] sortVals, Map metadata, boolean complete,
boolean includeSorts, SortOrder sortOrder, int size) {
super(name, metadata);
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
index 2e04d25345eba..1f99d69d9c81d 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
@@ -22,8 +22,8 @@ final class MergedGeoLines {
private final int[] lineIndices; // index of which geoLine item represents
private final int[] idxsWithinLine; // index within the geoLine for the item
private int size;
- private final long[] finalPoints;
- private final double[] finalSortValues;
+ private final long[] finalPoints; // the final sorted list of points, sorted by their respective sort-values. valid after merge
+ private final double[] finalSortValues; // the final sorted list of sort-values. valid after merge.
MergedGeoLines(List geoLines, int finalLength, SortOrder sortOrder) {
this.geoLines = geoLines;
@@ -44,6 +44,9 @@ public double[] getFinalSortValues() {
return finalSortValues;
}
+ /**
+ * merges geoLines into one sorted list of values representing the combined line.
+ */
public void merge() {
// 1. add first element of each sub line to heap
for (int i = 0; i < geoLines.size(); i++) {
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
index 8218d39c95e04..8d67757b132f9 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/PathArraySorter.java
@@ -8,6 +8,9 @@
import org.apache.lucene.util.IntroSorter;
import org.elasticsearch.search.sort.SortOrder;
+/**
+ * An {@link IntroSorter} that sorts points and sortValues using the
+ */
final class PathArraySorter extends IntroSorter {
private final long[] points;
From d4304538810514fa73cd40e8a7b2b63bcc06345f Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Tue, 17 Nov 2020 16:21:01 -0800
Subject: [PATCH 20/23] update geo_line license to Gold
---
.../org/elasticsearch/xpack/spatial/SpatialPlugin.java | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
index 6ba2fc8e5584a..51cbe276a0291 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/SpatialPlugin.java
@@ -26,7 +26,6 @@
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.ValueCountAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry;
-import org.elasticsearch.xpack.core.XPackField;
import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction;
import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
@@ -110,7 +109,8 @@ public List getAggregations() {
new AggregationSpec(
GeoLineAggregationBuilder.NAME,
GeoLineAggregationBuilder::new,
- usage.track(SpatialStatsAction.Item.GEOLINE, checkLicense(GeoLineAggregationBuilder.PARSER)))
+ usage.track(SpatialStatsAction.Item.GEOLINE,
+ checkLicense(GeoLineAggregationBuilder.PARSER, XPackLicenseState.Feature.SPATIAL_GEO_LINE)))
.addResultReader(InternalGeoLine::new)
.setAggregatorRegistrar(GeoLineAggregationBuilder::registerUsage));
}
@@ -196,10 +196,10 @@ private static void registerCardinalityAggregator(ValuesSourceRegistry.Builder b
builder.register(CardinalityAggregationBuilder.REGISTRY_KEY, GeoShapeValuesSourceType.instance(), CardinalityAggregator::new, true);
}
- private ContextParser checkLicense(ContextParser realParser) {
+ private ContextParser checkLicense(ContextParser realParser, XPackLicenseState.Feature feature) {
return (parser, name) -> {
- if (getLicenseState().checkFeature(XPackLicenseState.Feature.SPATIAL) == false) {
- throw LicenseUtils.newComplianceException(XPackField.SPATIAL);
+ if (getLicenseState().checkFeature(feature) == false) {
+ throw LicenseUtils.newComplianceException(feature.name());
}
return realParser.parse(parser, name);
};
From f64ce61a451a1146491fffa1dbc199506c0b6f38 Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Wed, 18 Nov 2020 15:26:51 -0800
Subject: [PATCH 21/23] add missing test and add docs
---
.../metrics/geoline-aggregation.asciidoc | 141 ++++++++++++++++++
.../aggregations/GeoLineAggregatorTests.java | 51 +++++++
2 files changed, 192 insertions(+)
create mode 100644 docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
diff --git a/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc b/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
new file mode 100644
index 0000000000000..9a5065c6cec71
--- /dev/null
+++ b/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
@@ -0,0 +1,141 @@
+[role="xpack"]
+[testenv="gold"]
+[[search-aggregations-metrics-geo-line]]
+=== Geo-Line Aggregation
+++++
+Geo-Line
+++++
+
+The `geo_line` aggregation aggregates all `geo_point` values within a bucket into a LineString ordered
+by the chosen `sort` field. This `sort` can be a date field, for example. The bucket returned is a valid
+https://tools.ietf.org/html/rfc7946#section-3.2[GeoJSON Feature] representing the line geometry.
+
+[source,console,id=search-aggregations-metrics-geo-line-simple]
+----
+PUT test
+{
+ "mappings": {
+ "dynamic": "strict",
+ "_source": {
+ "enabled": false
+ },
+ "properties": {
+ "my_location": {
+ "type": "geo_point"
+ },
+ "group": {
+ "type": "keyword"
+ },
+ "@timestamp": {
+ "type": "date"
+ }
+ }
+ }
+}
+
+POST /test/_bulk?refresh
+{"index": {}}
+{"my_location": {"lat":37.3450570, "lon": -122.0499820}, "@timestamp": "2013-09-06T16:00:36"}
+{"index": {}}
+{"my_location": {"lat": 37.3451320, "lon": -122.0499820}, "@timestamp": "2013-09-06T16:00:37Z"}
+{"index": {}}
+{"my_location": {"lat": 37.349283, "lon": -122.0505010}, "@timestamp": "2013-09-06T16:00:37Z"}
+
+POST /test/_search?filter_path=aggregations
+{
+ "aggs": {
+ "line": {
+ "geo_line": {
+ "point": {"field": "my_location"},
+ "sort": {"field": "@timestamp"}
+ }
+ }
+ }
+}
+----
+
+Which returns:
+
+[source,js]
+----
+{
+ "aggregations": {
+ "line": {
+ "type" : "Feature",
+ "geometry" : {
+ "type" : "LineString",
+ "coordinates" : [
+ [
+ -122.049982,
+ 37.345057
+ ],
+ [
+ -122.050501,
+ 37.349283
+ ],
+ [
+ -122.049982,
+ 37.345132
+ ]
+ ]
+ },
+ "properties" : {
+ "complete" : true
+ }
+ }
+ }
+}
+----
+// TESTRESPONSE
+
+[[search-aggregations-metrics-geo-line-options]]
+==== Options
+
+`point`::
+(Required)
+
+This option specifies the name of the `geo_point` field
+
+Example usage configuring `my_location` as the point field:
+
+[source,js]
+----
+"point": {
+ "field": "my_location"
+}
+----
+
+`sort`::
+(Required)
+
+This option specifies the name of the numeric field to use as the sort key
+for ordering the points
+
+Example usage configuring `@timestamp` as the sort key:
+
+[source,js]
+----
+"point": {
+ "field": "@timestamp"
+}
+----
+
+`include_sort`::
+(Optional, boolean, default: `false`)
+
+This option includes, when true, an additional array of the sort values in the
+feature properties.
+
+`sort_order`::
+(Optional, string, default: `"ASC"`)
+
+This option accepts one of two values: "ASC", "DESC".
+
+The line is sorted in ascending order by the sort key when set to "ASC", and in descending
+with "DESC".
+
+`size`::
+(Optional, integer, default: `10000`)
+
+The maximum length of the line represented in the aggregation. Valid sizes are
+between one and 10000.
diff --git a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
index b279c3e5dd09b..b3c43259558dd 100644
--- a/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
+++ b/x-pack/plugin/spatial/src/test/java/org/elasticsearch/xpack/spatial/search/aggregations/GeoLineAggregatorTests.java
@@ -50,6 +50,57 @@ protected List getSearchPlugins() {
return Collections.singletonList(new SpatialPlugin());
}
+ // test that missing values are ignored
+ public void testMissingValues() throws IOException {
+ MultiValuesSourceFieldConfig valueConfig = new MultiValuesSourceFieldConfig.Builder()
+ .setFieldName("value_field")
+ .build();
+ MultiValuesSourceFieldConfig sortConfig = new MultiValuesSourceFieldConfig.Builder().setFieldName("sort_field").build();
+ GeoLineAggregationBuilder lineAggregationBuilder = new GeoLineAggregationBuilder("_name")
+ .point(valueConfig)
+ .sortOrder(SortOrder.ASC)
+ .sort(sortConfig)
+ .size(10);
+
+ TermsAggregationBuilder aggregationBuilder = new TermsAggregationBuilder("_name")
+ .field("group_id")
+ .subAggregation(lineAggregationBuilder);
+
+ long lonLat = (((long) GeoEncodingUtils.encodeLongitude(90.0)) << 32) | GeoEncodingUtils.encodeLatitude(45.0) & 0xffffffffL;
+ //input
+ long[] points = new long[] {lonLat, 0, lonLat, 0,lonLat, lonLat, lonLat};
+ double[] sortValues = new double[]{1, 0, 2, 0, 3, 4, 5};
+ //expected
+ long[] expectedAggPoints = new long[] {lonLat, lonLat, lonLat, lonLat, lonLat};
+ double[] expectedAggSortValues = new double[]{
+ NumericUtils.doubleToSortableLong(1),
+ NumericUtils.doubleToSortableLong(2),
+ NumericUtils.doubleToSortableLong(3),
+ NumericUtils.doubleToSortableLong(4),
+ NumericUtils.doubleToSortableLong(5)
+ };
+
+ testCase(new MatchAllDocsQuery(), aggregationBuilder, iw -> {
+
+ for (int i = 0; i < points.length; i++) {
+ if (points[i] == 0) {
+ // do not index value
+ iw.addDocument(Collections.singletonList(new SortedDocValuesField("group_id", new BytesRef("group"))));
+ } else {
+ iw.addDocument(Arrays.asList(new LatLonDocValuesField("value_field", 45.0, 90.0),
+ new SortedNumericDocValuesField("sort_field", NumericUtils.doubleToSortableLong(sortValues[i])),
+ new SortedDocValuesField("group_id", new BytesRef("group"))));
+ }
+ }
+ }, terms -> {
+ assertThat(terms.getBuckets().size(), equalTo(1));
+ InternalGeoLine geoLine = terms.getBuckets().get(0).getAggregations().get("_name");
+ assertThat(geoLine.length(), equalTo(5));
+ assertTrue(geoLine.isComplete());
+ assertArrayEquals(expectedAggPoints, geoLine.line());
+ assertArrayEquals(expectedAggSortValues, geoLine.sortVals(), 0d);
+ });
+ }
public void testAscending() throws IOException {
testAggregator(SortOrder.ASC);
From ebff212049475ff2d8498cc069849be08987dbad Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Wed, 18 Nov 2020 15:37:07 -0800
Subject: [PATCH 22/23] resolve some docs issues
---
docs/reference/aggregations/metrics.asciidoc | 2 ++
.../reference/aggregations/metrics/geoline-aggregation.asciidoc | 2 ++
2 files changed, 4 insertions(+)
diff --git a/docs/reference/aggregations/metrics.asciidoc b/docs/reference/aggregations/metrics.asciidoc
index ea3195567ce67..cb961d735123b 100644
--- a/docs/reference/aggregations/metrics.asciidoc
+++ b/docs/reference/aggregations/metrics.asciidoc
@@ -23,6 +23,8 @@ include::metrics/geobounds-aggregation.asciidoc[]
include::metrics/geocentroid-aggregation.asciidoc[]
+include::metrics/geoline-aggregation.asciidoc[]
+
include::metrics/matrix-stats-aggregation.asciidoc[]
include::metrics/max-aggregation.asciidoc[]
diff --git a/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc b/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
index 9a5065c6cec71..4d02ba6346cd1 100644
--- a/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
+++ b/docs/reference/aggregations/metrics/geoline-aggregation.asciidoc
@@ -104,6 +104,7 @@ Example usage configuring `my_location` as the point field:
"field": "my_location"
}
----
+// NOTCONSOLE
`sort`::
(Required)
@@ -119,6 +120,7 @@ Example usage configuring `@timestamp` as the sort key:
"field": "@timestamp"
}
----
+// NOTCONSOLE
`include_sort`::
(Optional, boolean, default: `false`)
From 1971ce5b53cbcedd0edcf8fe8234388b3f4d491c Mon Sep 17 00:00:00 2001
From: Tal Levy
Date: Mon, 23 Nov 2020 09:34:32 -0800
Subject: [PATCH 23/23] guard for empty internalgeolines
---
.../xpack/spatial/search/aggregations/MergedGeoLines.java | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
diff --git a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
index 1f99d69d9c81d..8b8c4aeee1c02 100644
--- a/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
+++ b/x-pack/plugin/spatial/src/main/java/org/elasticsearch/xpack/spatial/search/aggregations/MergedGeoLines.java
@@ -50,7 +50,9 @@ public double[] getFinalSortValues() {
public void merge() {
// 1. add first element of each sub line to heap
for (int i = 0; i < geoLines.size(); i++) {
- add(i, 0);
+ if (geoLines.size() > 0) {
+ add(i, 0);
+ }
}
// 2. take lowest/greatest value from heap and re-insert the next value from the same sub-line that specific value was chosen from.