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.