summaryrefslogtreecommitdiff
path: root/core/src
diff options
context:
space:
mode:
authorNicholas Knize <nknize@gmail.com>2015-09-16 16:54:54 -0500
committerNicholas Knize <nknize@gmail.com>2015-10-14 16:19:09 -0500
commitb31d3ddd3e5b13a73d56fb6ba9e08feebad793e4 (patch)
tree2bbda1bbf259058a8478c3ca473383df3a8fdcf6 /core/src
parent5d001d15781fc56b3bf6e7e54f2d0d736cd602e4 (diff)
Adds geo_centroid metric aggregator
This commit adds a new metric aggregator for computing the geo_centroid over a set of geo_point fields. This can be combined with other aggregators (e.g., geohash_grid, significant_terms) for computing the geospatial centroid based on the document sets from other aggregation results.
Diffstat (limited to 'core/src')
-rw-r--r--core/src/main/java/org/elasticsearch/common/geo/GeoUtils.java38
-rw-r--r--core/src/main/java/org/elasticsearch/search/SearchModule.java4
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java9
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroid.java31
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregator.java144
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidBuilder.java33
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidParser.java63
-rw-r--r--core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java154
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java9
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java241
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java254
-rw-r--r--core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java166
-rw-r--r--core/src/test/java/org/elasticsearch/test/geo/RandomGeoGenerator.java68
-rw-r--r--core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java21
14 files changed, 998 insertions, 237 deletions
diff --git a/core/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/core/src/main/java/org/elasticsearch/common/geo/GeoUtils.java
index 62fe81a5f1..f29cdd6092 100644
--- a/core/src/main/java/org/elasticsearch/common/geo/GeoUtils.java
+++ b/core/src/main/java/org/elasticsearch/common/geo/GeoUtils.java
@@ -285,38 +285,46 @@ public class GeoUtils {
* @param normLon Whether to normalize longitude.
*/
public static void normalizePoint(GeoPoint point, boolean normLat, boolean normLon) {
- double lat = point.lat();
- double lon = point.lon();
-
- normLat = normLat && (lat>90 || lat <= -90);
- normLon = normLon && (lon>180 || lon <= -180);
-
+ double[] pt = {point.lon(), point.lat()};
+ normalizePoint(pt, normLon, normLat);
+ point.reset(pt[1], pt[0]);
+ }
+
+ public static void normalizePoint(double[] lonLat) {
+ normalizePoint(lonLat, true, true);
+ }
+
+ public static void normalizePoint(double[] lonLat, boolean normLon, boolean normLat) {
+ assert lonLat != null && lonLat.length == 2;
+
+ normLat = normLat && (lonLat[1] > 90 || lonLat[1] <= -90);
+ normLon = normLon && (lonLat[0] > 180 || lonLat[0] <= -180);
+
if (normLat) {
- lat = centeredModulus(lat, 360);
+ lonLat[1] = centeredModulus(lonLat[1], 360);
boolean shift = true;
- if (lat < -90) {
- lat = -180 - lat;
- } else if (lat > 90) {
- lat = 180 - lat;
+ if (lonLat[1] < -90) {
+ lonLat[1] = -180 - lonLat[1];
+ } else if (lonLat[1] > 90) {
+ lonLat[1] = 180 - lonLat[1];
} else {
// No need to shift the longitude, and the latitude is normalized
shift = false;
}
if (shift) {
if (normLon) {
- lon += 180;
+ lonLat[0] += 180;
} else {
// Longitude won't be normalized,
// keep it in the form x+k*360 (with x in ]-180;180])
// by only changing x, assuming k is meaningful for the user application.
- lon += normalizeLon(lon) > 0 ? -180 : 180;
+ lonLat[0] += normalizeLon(lonLat[0]) > 0 ? -180 : 180;
}
}
}
if (normLon) {
- lon = centeredModulus(lon, 360);
+ lonLat[0] = centeredModulus(lonLat[0], 360);
}
- point.reset(lat, lon);
}
private static double centeredModulus(double dividend, double divisor) {
diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java
index 7959eaa0a9..054cc5b950 100644
--- a/core/src/main/java/org/elasticsearch/search/SearchModule.java
+++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java
@@ -73,6 +73,8 @@ import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityPars
import org.elasticsearch.search.aggregations.metrics.cardinality.InternalCardinality;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsParser;
import org.elasticsearch.search.aggregations.metrics.geobounds.InternalGeoBounds;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidParser;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.InternalGeoCentroid;
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
import org.elasticsearch.search.aggregations.metrics.max.MaxParser;
import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
@@ -288,6 +290,7 @@ public class SearchModule extends AbstractModule {
multibinderAggParser.addBinding().to(ReverseNestedParser.class);
multibinderAggParser.addBinding().to(TopHitsParser.class);
multibinderAggParser.addBinding().to(GeoBoundsParser.class);
+ multibinderAggParser.addBinding().to(GeoCentroidParser.class);
multibinderAggParser.addBinding().to(ScriptedMetricParser.class);
multibinderAggParser.addBinding().to(ChildrenParser.class);
for (Class<? extends Aggregator.Parser> parser : aggParsers) {
@@ -357,6 +360,7 @@ public class SearchModule extends AbstractModule {
InternalHDRPercentileRanks.registerStreams();
InternalCardinality.registerStreams();
InternalScriptedMetric.registerStreams();
+ InternalGeoCentroid.registerStreams();
// buckets
InternalGlobal.registerStreams();
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
index 13a162df7f..dd9c5a30a9 100644
--- a/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java
@@ -55,6 +55,8 @@ import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
import org.elasticsearch.search.aggregations.metrics.cardinality.CardinalityBuilder;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsBuilder;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroid;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidBuilder;
import org.elasticsearch.search.aggregations.metrics.max.Max;
import org.elasticsearch.search.aggregations.metrics.max.MaxBuilder;
import org.elasticsearch.search.aggregations.metrics.min.Min;
@@ -288,6 +290,13 @@ public class AggregationBuilders {
}
/**
+ * Create a new {@link GeoCentroid} aggregation with the given name.
+ */
+ public static GeoCentroidBuilder geoCentroid(String name) {
+ return new GeoCentroidBuilder(name);
+ }
+
+ /**
* Create a new {@link ScriptedMetric} aggregation with the given name.
*/
public static ScriptedMetricBuilder scriptedMetric(String name) {
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroid.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroid.java
new file mode 100644
index 0000000000..2cdf462f04
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroid.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.geocentroid;
+
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.search.aggregations.Aggregation;
+
+/**
+ * Interface for {@link org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator}
+ */
+public interface GeoCentroid extends Aggregation {
+ GeoPoint centroid();
+ long count();
+}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregator.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregator.java
new file mode 100644
index 0000000000..e0a307e8c0
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregator.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.geocentroid;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.XGeoHashUtils;
+import org.apache.lucene.util.XGeoUtils;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.lease.Releasables;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.common.util.LongArray;
+import org.elasticsearch.index.fielddata.MultiGeoPointValues;
+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.metrics.geobounds.InternalGeoBounds;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+import org.elasticsearch.search.aggregations.support.AggregationContext;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A geo metric aggregator that computes a geo-centroid from a {@code geo_point} type field
+ */
+public final class GeoCentroidAggregator extends MetricsAggregator {
+ private final ValuesSource.GeoPoint valuesSource;
+ LongArray centroids;
+ LongArray counts;
+
+ protected GeoCentroidAggregator(String name, AggregationContext aggregationContext, Aggregator parent,
+ ValuesSource.GeoPoint valuesSource, List<PipelineAggregator> pipelineAggregators,
+ Map<String, Object> metaData) throws IOException {
+ super(name, aggregationContext, parent, pipelineAggregators, metaData);
+ this.valuesSource = valuesSource;
+ if (valuesSource != null) {
+ final BigArrays bigArrays = context.bigArrays();
+ centroids = bigArrays.newLongArray(1, true);
+ counts = bigArrays.newLongArray(1, true);
+ }
+ }
+
+ @Override
+ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException {
+ if (valuesSource == null) {
+ return LeafBucketCollector.NO_OP_COLLECTOR;
+ }
+ final BigArrays bigArrays = context.bigArrays();
+ final MultiGeoPointValues values = valuesSource.geoPointValues(ctx);
+ return new LeafBucketCollectorBase(sub, values) {
+ @Override
+ public void collect(int doc, long bucket) throws IOException {
+ centroids = bigArrays.grow(centroids, bucket + 1);
+ counts = bigArrays.grow(counts, bucket + 1);
+
+ values.setDocument(doc);
+ final int valueCount = values.count();
+ if (valueCount > 0) {
+ double[] pt = new double[2];
+ // get the previously accumulated number of counts
+ long prevCounts = counts.get(bucket);
+ // increment by the number of points for this document
+ counts.increment(bucket, valueCount);
+ // get the previous GeoPoint if a moving avg was computed
+ if (prevCounts > 0) {
+ final GeoPoint centroid = GeoPoint.fromIndexLong(centroids.get(bucket));
+ pt[0] = centroid.lon();
+ pt[1] = centroid.lat();
+ }
+ // update the moving average
+ for (int i = 0; i < valueCount; ++i) {
+ GeoPoint value = values.valueAt(i);
+ pt[0] = pt[0] + (value.getLon() - pt[0]) / ++prevCounts;
+ pt[1] = pt[1] + (value.getLat() - pt[1]) / prevCounts;
+ }
+ centroids.set(bucket, XGeoUtils.mortonHash(pt[0], pt[1]));
+ }
+ }
+ };
+ }
+
+ @Override
+ public InternalAggregation buildAggregation(long bucket) {
+ if (valuesSource == null || bucket >= centroids.size()) {
+ return buildEmptyAggregation();
+ }
+ final long bucketCount = counts.get(bucket);
+ final GeoPoint bucketCentroid = (bucketCount > 0) ? GeoPoint.fromIndexLong(centroids.get(bucket)) :
+ new GeoPoint(Double.NaN, Double.NaN);
+ return new InternalGeoCentroid(name, bucketCentroid , bucketCount, pipelineAggregators(), metaData());
+ }
+
+ @Override
+ public InternalAggregation buildEmptyAggregation() {
+ return new InternalGeoCentroid(name, null, 0l, pipelineAggregators(), metaData());
+ }
+
+ @Override
+ public void doClose() {
+ Releasables.close(centroids, counts);
+ }
+
+ public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.GeoPoint> {
+ protected Factory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config) {
+ super(name, InternalGeoBounds.TYPE.name(), config);
+ }
+
+ @Override
+ protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent,
+ List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) throws IOException {
+ return new GeoCentroidAggregator(name, aggregationContext, parent, null, pipelineAggregators, metaData);
+ }
+
+ @Override
+ protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent,
+ boolean collectsFromSingleBucket, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData)
+ throws IOException {
+ return new GeoCentroidAggregator(name, aggregationContext, parent, valuesSource, pipelineAggregators, metaData);
+ }
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidBuilder.java
new file mode 100644
index 0000000000..9d6823c675
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidBuilder.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.geocentroid;
+
+import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregationBuilder;
+
+/**
+ * Builder class for {@link org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator}
+ */
+public class GeoCentroidBuilder extends ValuesSourceMetricsAggregationBuilder<GeoCentroidBuilder> {
+
+ public GeoCentroidBuilder(String name) {
+ super(name, InternalGeoCentroid.TYPE.name());
+ }
+
+}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidParser.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidParser.java
new file mode 100644
index 0000000000..49a7bc8e96
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidParser.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.geocentroid;
+
+import org.elasticsearch.common.xcontent.XContentParser;
+import org.elasticsearch.search.SearchParseException;
+import org.elasticsearch.search.aggregations.Aggregator;
+import org.elasticsearch.search.aggregations.AggregatorFactory;
+import org.elasticsearch.search.aggregations.support.ValueType;
+import org.elasticsearch.search.aggregations.support.ValuesSource;
+import org.elasticsearch.search.aggregations.support.ValuesSourceParser;
+import org.elasticsearch.search.internal.SearchContext;
+
+import java.io.IOException;
+
+/**
+ * Parser class for {@link org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator}
+ */
+public class GeoCentroidParser implements Aggregator.Parser {
+
+ @Override
+ public String type() {
+ return InternalGeoCentroid.TYPE.name();
+ }
+
+ @Override
+ public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
+ ValuesSourceParser<ValuesSource.GeoPoint> vsParser = ValuesSourceParser.geoPoint(aggregationName, InternalGeoCentroid.TYPE, context)
+ .targetValueType(ValueType.GEOPOINT)
+ .formattable(true)
+ .build();
+ XContentParser.Token token;
+ String currentFieldName = null;
+ while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
+ if (token == XContentParser.Token.FIELD_NAME) {
+ currentFieldName = parser.currentName();
+ } else if (vsParser.token(currentFieldName, token, parser)) {
+ continue;
+ } else {
+ throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: ["
+ + currentFieldName + "].", parser.getTokenLocation());
+ }
+ }
+ return new GeoCentroidAggregator.Factory(aggregationName, vsParser.config());
+ }
+}
diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java
new file mode 100644
index 0000000000..192e6438ba
--- /dev/null
+++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/InternalGeoCentroid.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.geocentroid;
+
+import org.apache.lucene.util.XGeoUtils;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentBuilderString;
+import org.elasticsearch.search.aggregations.AggregationStreams;
+import org.elasticsearch.search.aggregations.InternalAggregation;
+import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Serialization and merge logic for {@link org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroidAggregator}
+ */
+public class InternalGeoCentroid extends InternalMetricsAggregation implements GeoCentroid {
+
+ public final static Type TYPE = new Type("geo_centroid");
+ public final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
+ @Override
+ public InternalGeoCentroid readResult(StreamInput in) throws IOException {
+ InternalGeoCentroid result = new InternalGeoCentroid();
+ result.readFrom(in);
+ return result;
+ }
+ };
+
+ public static void registerStreams() {
+ AggregationStreams.registerStream(STREAM, TYPE.stream());
+ }
+
+ protected GeoPoint centroid;
+ protected long count;
+
+ protected InternalGeoCentroid() {
+ }
+
+ public InternalGeoCentroid(String name, GeoPoint centroid, long count, List<PipelineAggregator>
+ pipelineAggregators, Map<String, Object> metaData) {
+ super(name, pipelineAggregators, metaData);
+ this.centroid = centroid;
+ this.count = count;
+ }
+
+ @Override
+ public GeoPoint centroid() {
+ return (centroid == null || Double.isNaN(centroid.lon()) ? null : centroid);
+ }
+
+ @Override
+ public long count() {
+ return count;
+ }
+
+ @Override
+ public Type type() {
+ return TYPE;
+ }
+
+ @Override
+ public InternalGeoCentroid doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
+ double lonSum = Double.NaN;
+ double latSum = Double.NaN;
+ int totalCount = 0;
+ for (InternalAggregation aggregation : aggregations) {
+ InternalGeoCentroid centroidAgg = (InternalGeoCentroid) aggregation;
+ if (centroidAgg.count > 0) {
+ totalCount += centroidAgg.count;
+ if (Double.isNaN(lonSum)) {
+ lonSum = centroidAgg.count * centroidAgg.centroid.getLon();
+ latSum = centroidAgg.count * centroidAgg.centroid.getLat();
+ } else {
+ lonSum += (centroidAgg.count * centroidAgg.centroid.getLon());
+ latSum += (centroidAgg.count * centroidAgg.centroid.getLat());
+ }
+ }
+ }
+ final GeoPoint result = (Double.isNaN(lonSum)) ? null : new GeoPoint(latSum/totalCount, lonSum/totalCount);
+ return new InternalGeoCentroid(name, result, totalCount, pipelineAggregators(), getMetaData());
+ }
+
+ @Override
+ public Object getProperty(List<String> path) {
+ if (path.isEmpty()) {
+ return this;
+ } else if (path.size() == 1) {
+ String coordinate = path.get(0);
+ switch (coordinate) {
+ case "value":
+ return centroid;
+ case "lat":
+ return centroid.lat();
+ case "lon":
+ return centroid.lon();
+ default:
+ throw new IllegalArgumentException("Found unknown path element [" + coordinate + "] in [" + getName() + "]");
+ }
+ } else {
+ throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path);
+ }
+ }
+
+ @Override
+ protected void doReadFrom(StreamInput in) throws IOException {
+ count = in.readVLong();
+ if (count > 0) {
+ centroid = GeoPoint.fromIndexLong(in.readLong());
+ }
+ }
+
+ @Override
+ protected void doWriteTo(StreamOutput out) throws IOException {
+ out.writeVLong(count);
+ if (centroid != null) {
+ out.writeLong(XGeoUtils.mortonHash(centroid.lon(), centroid.lat()));
+ }
+ }
+
+ static class Fields {
+ public static final XContentBuilderString CENTROID = new XContentBuilderString("location");
+ }
+
+ @Override
+ public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
+ if (centroid != null) {
+ builder.startObject(Fields.CENTROID).field("lat", centroid.lat()).field("lon", centroid.lon()).endObject();
+ }
+ return builder;
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java
index f76e1ad9e5..fbde232dbd 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/MissingValueIT.java
@@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroid;
import org.elasticsearch.search.aggregations.metrics.percentiles.Percentiles;
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
import org.elasticsearch.test.ESIntegTestCase;
@@ -33,6 +34,7 @@ import org.elasticsearch.test.ESIntegTestCase;
import static org.elasticsearch.search.aggregations.AggregationBuilders.cardinality;
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.geoCentroid;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
@@ -192,4 +194,11 @@ public class MissingValueIT extends ESIntegTestCase {
assertEquals(new GeoPoint(2,1), bounds.topLeft());
}
+ public void testGeoCentroid() {
+ SearchResponse response = client().prepareSearch("idx").addAggregation(geoCentroid("centroid").field("location").missing("2,1")).get();
+ assertSearchResponse(response);
+ GeoCentroid centroid = response.getAggregations().get("centroid");
+ assertEquals(new GeoPoint(1.5, 1.5), centroid.centroid());
+ }
+
}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java
new file mode 100644
index 0000000000..ebf29b8874
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/AbstractGeoTestCase.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics;
+
+import com.carrotsearch.hppc.ObjectIntHashMap;
+import com.carrotsearch.hppc.ObjectIntMap;
+import com.carrotsearch.hppc.ObjectObjectHashMap;
+import com.carrotsearch.hppc.ObjectObjectMap;
+import org.apache.lucene.util.XGeoHashUtils;
+import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.SearchHitField;
+import org.elasticsearch.search.sort.SortBuilders;
+import org.elasticsearch.search.sort.SortOrder;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.elasticsearch.test.geo.RandomGeoGenerator;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
+import static org.hamcrest.Matchers.equalTo;
+
+/**
+ *
+ */
+@ESIntegTestCase.SuiteScopeTestCase
+public abstract class AbstractGeoTestCase extends ESIntegTestCase {
+
+ protected static final String SINGLE_VALUED_FIELD_NAME = "geo_value";
+ protected static final String MULTI_VALUED_FIELD_NAME = "geo_values";
+ protected static final String NUMBER_FIELD_NAME = "l_values";
+ protected static final String UNMAPPED_IDX_NAME = "idx_unmapped";
+ protected static final String IDX_NAME = "idx";
+ protected static final String EMPTY_IDX_NAME = "empty_idx";
+ protected static final String DATELINE_IDX_NAME = "dateline_idx";
+ protected static final String HIGH_CARD_IDX_NAME = "high_card_idx";
+ protected static final String IDX_ZERO_NAME = "idx_zero";
+
+ protected static int numDocs;
+ protected static int numUniqueGeoPoints;
+ protected static GeoPoint[] singleValues, multiValues;
+ protected static GeoPoint singleTopLeft, singleBottomRight, multiTopLeft, multiBottomRight, singleCentroid, multiCentroid, unmappedCentroid;
+ protected static ObjectIntMap<String> expectedDocCountsForGeoHash = null;
+ protected static ObjectObjectMap<String, GeoPoint> expectedCentroidsForGeoHash = null;
+
+ @Override
+ public void setupSuiteScopeCluster() throws Exception {
+ createIndex(UNMAPPED_IDX_NAME);
+ assertAcked(prepareCreate(IDX_NAME)
+ .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point,geohash_prefix=true,geohash_precision=12",
+ MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
+
+ singleTopLeft = new GeoPoint(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
+ singleBottomRight = new GeoPoint(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
+ multiTopLeft = new GeoPoint(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
+ multiBottomRight = new GeoPoint(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
+ singleCentroid = new GeoPoint(0, 0);
+ multiCentroid = new GeoPoint(0, 0);
+ unmappedCentroid = new GeoPoint(0, 0);
+
+ numDocs = randomIntBetween(6, 20);
+ numUniqueGeoPoints = randomIntBetween(1, numDocs);
+ expectedDocCountsForGeoHash = new ObjectIntHashMap<>(numDocs * 2);
+ expectedCentroidsForGeoHash = new ObjectObjectHashMap<>(numDocs * 2);
+
+ singleValues = new GeoPoint[numUniqueGeoPoints];
+ for (int i = 0 ; i < singleValues.length; i++)
+ {
+ singleValues[i] = RandomGeoGenerator.randomPoint(random());
+ updateBoundsTopLeft(singleValues[i], singleTopLeft);
+ updateBoundsBottomRight(singleValues[i], singleBottomRight);
+ }
+
+ multiValues = new GeoPoint[numUniqueGeoPoints];
+ for (int i = 0 ; i < multiValues.length; i++)
+ {
+ multiValues[i] = RandomGeoGenerator.randomPoint(random());
+ updateBoundsTopLeft(multiValues[i], multiTopLeft);
+ updateBoundsBottomRight(multiValues[i], multiBottomRight);
+ }
+
+ List<IndexRequestBuilder> builders = new ArrayList<>();
+
+ GeoPoint singleVal;
+ final GeoPoint[] multiVal = new GeoPoint[2];
+ double newMVLat, newMVLon;
+ for (int i = 0; i < numDocs; i++) {
+ singleVal = singleValues[i % numUniqueGeoPoints];
+ multiVal[0] = multiValues[i % numUniqueGeoPoints];
+ multiVal[1] = multiValues[(i+1) % numUniqueGeoPoints];
+ builders.add(client().prepareIndex(IDX_NAME, "type").setSource(jsonBuilder()
+ .startObject()
+ .array(SINGLE_VALUED_FIELD_NAME, singleVal.lon(), singleVal.lat())
+ .startArray(MULTI_VALUED_FIELD_NAME)
+ .startArray().value(multiVal[0].lon()).value(multiVal[0].lat()).endArray()
+ .startArray().value(multiVal[1].lon()).value(multiVal[1].lat()).endArray()
+ .endArray()
+ .field(NUMBER_FIELD_NAME, i)
+ .field("tag", "tag" + i)
+ .endObject()));
+ singleCentroid = singleCentroid.reset(singleCentroid.lat() + (singleVal.lat() - singleCentroid.lat()) / (i+1),
+ singleCentroid.lon() + (singleVal.lon() - singleCentroid.lon()) / (i+1));
+ newMVLat = (multiVal[0].lat() + multiVal[1].lat())/2d;
+ newMVLon = (multiVal[0].lon() + multiVal[1].lon())/2d;
+ multiCentroid = multiCentroid.reset(multiCentroid.lat() + (newMVLat - multiCentroid.lat()) / (i+1),
+ multiCentroid.lon() + (newMVLon - multiCentroid.lon()) / (i+1));
+ }
+
+ assertAcked(prepareCreate(EMPTY_IDX_NAME).addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point"));
+
+ assertAcked(prepareCreate(DATELINE_IDX_NAME)
+ .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point", MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
+
+ GeoPoint[] geoValues = new GeoPoint[5];
+ geoValues[0] = new GeoPoint(38, 178);
+ geoValues[1] = new GeoPoint(12, -179);
+ geoValues[2] = new GeoPoint(-24, 170);
+ geoValues[3] = new GeoPoint(32, -175);
+ geoValues[4] = new GeoPoint(-11, 178);
+
+ for (int i = 0; i < 5; i++) {
+ builders.add(client().prepareIndex(DATELINE_IDX_NAME, "type").setSource(jsonBuilder()
+ .startObject()
+ .array(SINGLE_VALUED_FIELD_NAME, geoValues[i].lon(), geoValues[i].lat())
+ .field(NUMBER_FIELD_NAME, i)
+ .field("tag", "tag" + i)
+ .endObject()));
+ }
+ assertAcked(prepareCreate(HIGH_CARD_IDX_NAME).setSettings(Settings.builder().put("number_of_shards", 2))
+ .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point", MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
+
+ for (int i = 0; i < 2000; i++) {
+ singleVal = singleValues[i % numUniqueGeoPoints];
+ builders.add(client().prepareIndex(HIGH_CARD_IDX_NAME, "type").setSource(jsonBuilder()
+ .startObject()
+ .array(SINGLE_VALUED_FIELD_NAME, singleVal.lon(), singleVal.lat())
+ .startArray(MULTI_VALUED_FIELD_NAME)
+ .startArray().value(multiValues[i % numUniqueGeoPoints].lon()).value(multiValues[i % numUniqueGeoPoints].lat()).endArray()
+ .startArray().value(multiValues[(i + 1) % numUniqueGeoPoints].lon()).value(multiValues[(i + 1) % numUniqueGeoPoints].lat()).endArray()
+ .endArray()
+ .field(NUMBER_FIELD_NAME, i)
+ .field("tag", "tag" + i)
+ .endObject()));
+ updateGeohashBucketsCentroid(singleVal);
+ }
+
+ builders.add(client().prepareIndex(IDX_ZERO_NAME, "type").setSource(
+ jsonBuilder().startObject().array(SINGLE_VALUED_FIELD_NAME, 0.0, 1.0).endObject()));
+ assertAcked(prepareCreate(IDX_ZERO_NAME).addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point"));
+
+ indexRandom(true, builders);
+ ensureSearchable();
+
+ // Added to debug a test failure where the terms aggregation seems to be reporting two documents with the same value for NUMBER_FIELD_NAME. This will check that after
+ // random indexing each document only has 1 value for NUMBER_FIELD_NAME and it is the correct value. Following this initial change its seems that this call was getting
+ // more that 2000 hits (actual value was 2059) so now it will also check to ensure all hits have the correct index and type
+ SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME).addField(NUMBER_FIELD_NAME).addSort(SortBuilders.fieldSort(NUMBER_FIELD_NAME)
+ .order(SortOrder.ASC)).setSize(5000).get();
+ assertSearchResponse(response);
+ long totalHits = response.getHits().totalHits();
+ XContentBuilder builder = XContentFactory.jsonBuilder().startObject();
+ response.toXContent(builder, ToXContent.EMPTY_PARAMS);
+ builder.endObject();
+ logger.info("Full high_card_idx Response Content:\n{ {} }", builder.string());
+ for (int i = 0; i < totalHits; i++) {
+ SearchHit searchHit = response.getHits().getAt(i);
+ assertThat("Hit " + i + " with id: " + searchHit.getId(), searchHit.getIndex(), equalTo("high_card_idx"));
+ assertThat("Hit " + i + " with id: " + searchHit.getId(), searchHit.getType(), equalTo("type"));
+ SearchHitField hitField = searchHit.field(NUMBER_FIELD_NAME);
+
+ assertThat("Hit " + i + " has wrong number of values", hitField.getValues().size(), equalTo(1));
+ Integer value = hitField.getValue();
+ assertThat("Hit " + i + " has wrong value", value, equalTo(i));
+ }
+ assertThat(totalHits, equalTo(2000l));
+ }
+
+ private void updateGeohashBucketsCentroid(final GeoPoint location) {
+ String hash = XGeoHashUtils.stringEncode(location.lon(), location.lat(), XGeoHashUtils.PRECISION);
+ for (int precision = XGeoHashUtils.PRECISION; precision > 0; --precision) {
+ final String h = hash.substring(0, precision);
+ expectedDocCountsForGeoHash.put(h, expectedDocCountsForGeoHash.getOrDefault(h, 0) + 1);
+ expectedCentroidsForGeoHash.put(h, updateHashCentroid(h, location));
+ }
+ }
+
+ private GeoPoint updateHashCentroid(String hash, final GeoPoint location) {
+ GeoPoint centroid = expectedCentroidsForGeoHash.getOrDefault(hash, null);
+ if (centroid == null) {
+ return new GeoPoint(location.lat(), location.lon());
+ }
+ final int docCount = expectedDocCountsForGeoHash.get(hash);
+ final double newLon = centroid.lon() + (location.lon() - centroid.lon()) / docCount;
+ final double newLat = centroid.lat() + (location.lat() - centroid.lat()) / docCount;
+ return centroid.reset(newLat, newLon);
+ }
+
+ private void updateBoundsBottomRight(GeoPoint geoPoint, GeoPoint currentBound) {
+ if (geoPoint.lat() < currentBound.lat()) {
+ currentBound.resetLat(geoPoint.lat());
+ }
+ if (geoPoint.lon() > currentBound.lon()) {
+ currentBound.resetLon(geoPoint.lon());
+ }
+ }
+
+ private void updateBoundsTopLeft(GeoPoint geoPoint, GeoPoint currentBound) {
+ if (geoPoint.lat() > currentBound.lat()) {
+ currentBound.resetLat(geoPoint.lat());
+ }
+ if (geoPoint.lon() < currentBound.lon()) {
+ currentBound.resetLon(geoPoint.lon());
+ }
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java
index 6d2d61e707..056fc59219 100644
--- a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java
@@ -19,35 +19,23 @@
package org.elasticsearch.search.aggregations.metrics;
-import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.geo.GeoPoint;
-import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArray;
-import org.elasticsearch.common.xcontent.ToXContent;
-import org.elasticsearch.common.xcontent.XContentBuilder;
-import org.elasticsearch.common.xcontent.XContentFactory;
-import org.elasticsearch.search.SearchHit;
-import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.aggregations.bucket.global.Global;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregator;
-import org.elasticsearch.search.sort.SortBuilders;
-import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test;
-import java.util.ArrayList;
import java.util.List;
-import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds;
import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
-import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.equalTo;
@@ -60,167 +48,22 @@ import static org.hamcrest.Matchers.sameInstance;
*
*/
@ESIntegTestCase.SuiteScopeTestCase
-public class GeoBoundsIT extends ESIntegTestCase {
-
- private static final String SINGLE_VALUED_FIELD_NAME = "geo_value";
- private static final String MULTI_VALUED_FIELD_NAME = "geo_values";
- private static final String NUMBER_FIELD_NAME = "l_values";
-
- static int numDocs;
- static int numUniqueGeoPoints;
- static GeoPoint[] singleValues, multiValues;
- static GeoPoint singleTopLeft, singleBottomRight, multiTopLeft, multiBottomRight, unmappedTopLeft, unmappedBottomRight;
-
- @Override
- public void setupSuiteScopeCluster() throws Exception {
- assertAcked(prepareCreate("idx")
- .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point", MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
- createIndex("idx_unmapped");
-
- unmappedTopLeft = new GeoPoint(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
- unmappedBottomRight = new GeoPoint(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
- singleTopLeft = new GeoPoint(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
- singleBottomRight = new GeoPoint(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
- multiTopLeft = new GeoPoint(Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY);
- multiBottomRight = new GeoPoint(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY);
-
- numDocs = randomIntBetween(6, 20);
- numUniqueGeoPoints = randomIntBetween(1, numDocs);
-
- singleValues = new GeoPoint[numUniqueGeoPoints];
- for (int i = 0 ; i < singleValues.length; i++)
- {
- singleValues[i] = randomGeoPoint();
- updateBoundsTopLeft(singleValues[i], singleTopLeft);
- updateBoundsBottomRight(singleValues[i], singleBottomRight);
- }
-
- multiValues = new GeoPoint[numUniqueGeoPoints];
- for (int i = 0 ; i < multiValues.length; i++)
- {
- multiValues[i] = randomGeoPoint();
- updateBoundsTopLeft(multiValues[i], multiTopLeft);
- updateBoundsBottomRight(multiValues[i], multiBottomRight);
- }
-
- List<IndexRequestBuilder> builders = new ArrayList<>();
-
-
- for (int i = 0; i < numDocs; i++) {
- builders.add(client().prepareIndex("idx", "type").setSource(jsonBuilder()
- .startObject()
- .array(SINGLE_VALUED_FIELD_NAME, singleValues[i % numUniqueGeoPoints].lon(), singleValues[i % numUniqueGeoPoints].lat())
- .startArray(MULTI_VALUED_FIELD_NAME)
- .startArray().value(multiValues[i % numUniqueGeoPoints].lon()).value(multiValues[i % numUniqueGeoPoints].lat()).endArray()
- .startArray().value(multiValues[(i+1) % numUniqueGeoPoints].lon()).value(multiValues[(i+1) % numUniqueGeoPoints].lat()).endArray()
- .endArray()
- .field(NUMBER_FIELD_NAME, i)
- .field("tag", "tag" + i)
- .endObject()));
- }
-
- assertAcked(prepareCreate("empty_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point"));
-
- assertAcked(prepareCreate("idx_dateline")
- .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point", MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
-
- GeoPoint[] geoValues = new GeoPoint[5];
- geoValues[0] = new GeoPoint(38, 178);
- geoValues[1] = new GeoPoint(12, -179);
- geoValues[2] = new GeoPoint(-24, 170);
- geoValues[3] = new GeoPoint(32, -175);
- geoValues[4] = new GeoPoint(-11, 178);
-
- for (int i = 0; i < 5; i++) {
- builders.add(client().prepareIndex("idx_dateline", "type").setSource(jsonBuilder()
- .startObject()
- .array(SINGLE_VALUED_FIELD_NAME, geoValues[i].lon(), geoValues[i].lat())
- .field(NUMBER_FIELD_NAME, i)
- .field("tag", "tag" + i)
- .endObject()));
- }
- assertAcked(prepareCreate("high_card_idx").setSettings(Settings.builder().put("number_of_shards", 2))
- .addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point", MULTI_VALUED_FIELD_NAME, "type=geo_point", NUMBER_FIELD_NAME, "type=long", "tag", "type=string,index=not_analyzed"));
-
-
- for (int i = 0; i < 2000; i++) {
- builders.add(client().prepareIndex("high_card_idx", "type").setSource(jsonBuilder()
- .startObject()
- .array(SINGLE_VALUED_FIELD_NAME, singleValues[i % numUniqueGeoPoints].lon(), singleValues[i % numUniqueGeoPoints].lat())
- .startArray(MULTI_VALUED_FIELD_NAME)
- .startArray().value(multiValues[i % numUniqueGeoPoints].lon()).value(multiValues[i % numUniqueGeoPoints].lat()).endArray()
- .startArray().value(multiValues[(i+1) % numUniqueGeoPoints].lon()).value(multiValues[(i+1) % numUniqueGeoPoints].lat()).endArray()
- .endArray()
- .field(NUMBER_FIELD_NAME, i)
- .field("tag", "tag" + i)
- .endObject()));
- }
+public class GeoBoundsIT extends AbstractGeoTestCase {
- builders.add(client().prepareIndex("idx_zero", "type").setSource(
- jsonBuilder().startObject().array(SINGLE_VALUED_FIELD_NAME, 0.0, 1.0).endObject()));
- assertAcked(prepareCreate("idx_zero").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=geo_point"));
-
- indexRandom(true, builders);
- ensureSearchable();
-
- // Added to debug a test failure where the terms aggregation seems to be reporting two documents with the same value for NUMBER_FIELD_NAME. This will check that after
- // random indexing each document only has 1 value for NUMBER_FIELD_NAME and it is the correct value. Following this initial change its seems that this call was getting
- // more that 2000 hits (actual value was 2059) so now it will also check to ensure all hits have the correct index and type
- SearchResponse response = client().prepareSearch("high_card_idx").addField(NUMBER_FIELD_NAME).addSort(SortBuilders.fieldSort(NUMBER_FIELD_NAME).order(SortOrder.ASC)).setSize(5000).get();
- assertSearchResponse(response);
- long totalHits = response.getHits().totalHits();
- XContentBuilder builder = XContentFactory.jsonBuilder().startObject();
- response.toXContent(builder, ToXContent.EMPTY_PARAMS);
- builder.endObject();
- logger.info("Full high_card_idx Response Content:\n{ {} }", builder.string());
- for (int i = 0; i < totalHits; i++) {
- SearchHit searchHit = response.getHits().getAt(i);
- assertThat("Hit " + i + " with id: " + searchHit.getId(), searchHit.getIndex(), equalTo("high_card_idx"));
- assertThat("Hit " + i + " with id: " + searchHit.getId(), searchHit.getType(), equalTo("type"));
- SearchHitField hitField = searchHit.field(NUMBER_FIELD_NAME);
-
- assertThat("Hit " + i + " has wrong number of values", hitField.getValues().size(), equalTo(1));
- Integer value = hitField.getValue();
- assertThat("Hit " + i + " has wrong value", value, equalTo(i));
- }
- assertThat(totalHits, equalTo(2000l));
- }
-
- private void updateBoundsBottomRight(GeoPoint geoPoint, GeoPoint currentBound) {
- if (geoPoint.lat() < currentBound.lat()) {
- currentBound.resetLat(geoPoint.lat());
- }
- if (geoPoint.lon() > currentBound.lon()) {
- currentBound.resetLon(geoPoint.lon());
- }
- }
-
- private void updateBoundsTopLeft(GeoPoint geoPoint, GeoPoint currentBound) {
- if (geoPoint.lat() > currentBound.lat()) {
- currentBound.resetLat(geoPoint.lat());
- }
- if (geoPoint.lon() < currentBound.lon()) {
- currentBound.resetLon(geoPoint.lon());
- }
- }
-
- private GeoPoint randomGeoPoint() {
- return new GeoPoint((randomDouble() * 180) - 90, (randomDouble() * 360) - 180);
- }
+ private static final String aggName = "geoBounds";
@Test
public void singleValuedField() throws Exception {
- SearchResponse response = client().prepareSearch("idx")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
assertSearchResponse(response);
-
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
@@ -232,10 +75,10 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void testSingleValuedField_getProperty() throws Exception {
SearchResponse searchResponse = client()
- .prepareSearch("idx")
+ .prepareSearch(IDX_NAME)
.setQuery(matchAllQuery())
.addAggregation(
- global("global").subAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)))
+ global("global").subAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)))
.execute().actionGet();
assertSearchResponse(searchResponse);
@@ -247,35 +90,35 @@ public class GeoBoundsIT extends ESIntegTestCase {
assertThat(global.getAggregations(), notNullValue());
assertThat(global.getAggregations().asMap().size(), equalTo(1));
- GeoBounds geobounds = global.getAggregations().get("geoBounds");
+ GeoBounds geobounds = global.getAggregations().get(aggName);
assertThat(geobounds, notNullValue());
- assertThat(geobounds.getName(), equalTo("geoBounds"));
- assertThat((GeoBounds) global.getProperty("geoBounds"), sameInstance(geobounds));
+ assertThat(geobounds.getName(), equalTo(aggName));
+ assertThat((GeoBounds) global.getProperty(aggName), sameInstance(geobounds));
GeoPoint topLeft = geobounds.topLeft();
GeoPoint bottomRight = geobounds.bottomRight();
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
assertThat(topLeft.lon(), equalTo(singleTopLeft.lon()));
assertThat(bottomRight.lat(), equalTo(singleBottomRight.lat()));
assertThat(bottomRight.lon(), equalTo(singleBottomRight.lon()));
- assertThat((double) global.getProperty("geoBounds.top"), equalTo(singleTopLeft.lat()));
- assertThat((double) global.getProperty("geoBounds.left"), equalTo(singleTopLeft.lon()));
- assertThat((double) global.getProperty("geoBounds.bottom"), equalTo(singleBottomRight.lat()));
- assertThat((double) global.getProperty("geoBounds.right"), equalTo(singleBottomRight.lon()));
+ assertThat((double) global.getProperty(aggName + ".top"), equalTo(singleTopLeft.lat()));
+ assertThat((double) global.getProperty(aggName + ".left"), equalTo(singleTopLeft.lon()));
+ assertThat((double) global.getProperty(aggName + ".bottom"), equalTo(singleBottomRight.lat()));
+ assertThat((double) global.getProperty(aggName + ".right"), equalTo(singleBottomRight.lon()));
}
@Test
public void multiValuedField() throws Exception {
- SearchResponse response = client().prepareSearch("idx")
- .addAggregation(geoBounds("geoBounds").field(MULTI_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(MULTI_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
assertSearchResponse(response);
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(multiTopLeft.lat()));
@@ -286,17 +129,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void unmapped() throws Exception {
- SearchResponse response = client().prepareSearch("idx_unmapped")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(UNMAPPED_IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
assertSearchResponse(response);
-
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft, equalTo(null));
@@ -305,17 +147,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void partiallyUnmapped() throws Exception {
- SearchResponse response = client().prepareSearch("idx", "idx_unmapped")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(IDX_NAME, UNMAPPED_IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
assertSearchResponse(response);
-
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
@@ -326,16 +167,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void emptyAggregation() throws Exception {
- SearchResponse searchResponse = client().prepareSearch("empty_idx")
+ SearchResponse searchResponse = client().prepareSearch(EMPTY_IDX_NAME)
.setQuery(matchAllQuery())
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
assertThat(searchResponse.getHits().getTotalHits(), equalTo(0l));
- GeoBounds geoBounds = searchResponse.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = searchResponse.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft, equalTo(null));
@@ -344,8 +185,8 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void singleValuedFieldNearDateLine() throws Exception {
- SearchResponse response = client().prepareSearch("idx_dateline")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(DATELINE_IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false))
.execute().actionGet();
@@ -354,9 +195,9 @@ public class GeoBoundsIT extends ESIntegTestCase {
GeoPoint geoValuesTopLeft = new GeoPoint(38, -179);
GeoPoint geoValuesBottomRight = new GeoPoint(-24, 178);
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
@@ -371,15 +212,15 @@ public class GeoBoundsIT extends ESIntegTestCase {
GeoPoint geoValuesTopLeft = new GeoPoint(38, 170);
GeoPoint geoValuesBottomRight = new GeoPoint(-24, -175);
- SearchResponse response = client().prepareSearch("idx_dateline")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(true))
+ SearchResponse response = client().prepareSearch(DATELINE_IDX_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(true))
.execute().actionGet();
assertSearchResponse(response);
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
@@ -393,8 +234,8 @@ public class GeoBoundsIT extends ESIntegTestCase {
*/
@Test
public void singleValuedFieldAsSubAggToHighCardTermsAgg() {
- SearchResponse response = client().prepareSearch("high_card_idx")
- .addAggregation(terms("terms").field(NUMBER_FIELD_NAME).subAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
+ SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME)
+ .addAggregation(terms("terms").field(NUMBER_FIELD_NAME).subAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
.wrapLongitude(false)))
.execute().actionGet();
@@ -409,9 +250,9 @@ public class GeoBoundsIT extends ESIntegTestCase {
Bucket bucket = buckets.get(i);
assertThat(bucket, notNullValue());
assertThat("Bucket " + bucket.getKey() + " has wrong number of documents", bucket.getDocCount(), equalTo(1l));
- GeoBounds geoBounds = bucket.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = bucket.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
assertThat(geoBounds.topLeft().getLat(), allOf(greaterThanOrEqualTo(-90.0), lessThanOrEqualTo(90.0)));
assertThat(geoBounds.topLeft().getLon(), allOf(greaterThanOrEqualTo(-180.0), lessThanOrEqualTo(180.0)));
assertThat(geoBounds.bottomRight().getLat(), allOf(greaterThanOrEqualTo(-90.0), lessThanOrEqualTo(90.0)));
@@ -421,14 +262,14 @@ public class GeoBoundsIT extends ESIntegTestCase {
@Test
public void singleValuedFieldWithZeroLon() throws Exception {
- SearchResponse response = client().prepareSearch("idx_zero")
- .addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)).execute().actionGet();
+ SearchResponse response = client().prepareSearch(IDX_ZERO_NAME)
+ .addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)).execute().actionGet();
assertSearchResponse(response);
- GeoBounds geoBounds = response.getAggregations().get("geoBounds");
+ GeoBounds geoBounds = response.getAggregations().get(aggName);
assertThat(geoBounds, notNullValue());
- assertThat(geoBounds.getName(), equalTo("geoBounds"));
+ assertThat(geoBounds.getName(), equalTo(aggName));
GeoPoint topLeft = geoBounds.topLeft();
GeoPoint bottomRight = geoBounds.bottomRight();
assertThat(topLeft.lat(), equalTo(1.0));
@@ -436,5 +277,4 @@ public class GeoBoundsIT extends ESIntegTestCase {
assertThat(bottomRight.lat(), equalTo(1.0));
assertThat(bottomRight.lon(), equalTo(0.0));
}
-
}
diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java
new file mode 100644
index 0000000000..adb49d4d17
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics;
+
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.common.geo.GeoPoint;
+import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid;
+import org.elasticsearch.search.aggregations.bucket.global.Global;
+import org.elasticsearch.search.aggregations.metrics.geocentroid.GeoCentroid;
+import org.elasticsearch.test.ESIntegTestCase;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.geoCentroid;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.geohashGrid;
+import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
+import static org.hamcrest.Matchers.*;
+
+/**
+ * Integration Test for GeoCentroid metric aggregator
+ */
+@ESIntegTestCase.SuiteScopeTestCase
+public class GeoCentroidIT extends AbstractGeoTestCase {
+ private static final String aggName = "geoCentroid";
+
+ @Test
+ public void emptyAggregation() throws Exception {
+ SearchResponse response = client().prepareSearch(EMPTY_IDX_NAME)
+ .setQuery(matchAllQuery())
+ .addAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ GeoCentroid geoCentroid = response.getAggregations().get(aggName);
+ assertThat(response.getHits().getTotalHits(), equalTo(0l));
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(null));
+ }
+
+ @Test
+ public void unmapped() throws Exception {
+ SearchResponse response = client().prepareSearch(UNMAPPED_IDX_NAME)
+ .addAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ GeoCentroid geoCentroid = response.getAggregations().get(aggName);
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(null));
+ }
+
+ @Test
+ public void partiallyUnmapped() throws Exception {
+ SearchResponse response = client().prepareSearch(IDX_NAME, UNMAPPED_IDX_NAME)
+ .addAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ GeoCentroid geoCentroid = response.getAggregations().get(aggName);
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(singleCentroid));
+ }
+
+ @Test
+ public void singleValuedField() throws Exception {
+ SearchResponse response = client().prepareSearch(IDX_NAME)
+ .setQuery(matchAllQuery())
+ .addAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ GeoCentroid geoCentroid = response.getAggregations().get(aggName);
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(singleCentroid));
+ }
+
+ @Test
+ public void singleValueField_getProperty() throws Exception {
+ SearchResponse response = client().prepareSearch(IDX_NAME)
+ .setQuery(matchAllQuery())
+ .addAggregation(global("global").subAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME)))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ Global global = response.getAggregations().get("global");
+ assertThat(global, notNullValue());
+ assertThat(global.getName(), equalTo("global"));
+ assertThat(global.getDocCount(), equalTo((long) numDocs));
+ assertThat(global.getAggregations(), notNullValue());
+ assertThat(global.getAggregations().asMap().size(), equalTo(1));
+
+ GeoCentroid geoCentroid = global.getAggregations().get(aggName);
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ assertThat((GeoCentroid) global.getProperty(aggName), sameInstance(geoCentroid));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(singleCentroid));
+ assertThat((GeoPoint) global.getProperty(aggName + ".value"), equalTo(singleCentroid));
+ assertThat((double) global.getProperty(aggName + ".lat"), closeTo(singleCentroid.lat(), 1e-5));
+ assertThat((double) global.getProperty(aggName + ".lon"), closeTo(singleCentroid.lon(), 1e-5));
+ }
+
+ @Test
+ public void multiValuedField() throws Exception {
+ SearchResponse searchResponse = client().prepareSearch(IDX_NAME)
+ .setQuery(matchAllQuery())
+ .addAggregation(geoCentroid(aggName).field(MULTI_VALUED_FIELD_NAME))
+ .execute().actionGet();
+ assertSearchResponse(searchResponse);
+
+ GeoCentroid geoCentroid = searchResponse.getAggregations().get(aggName);
+ assertThat(geoCentroid, notNullValue());
+ assertThat(geoCentroid.getName(), equalTo(aggName));
+ GeoPoint centroid = geoCentroid.centroid();
+ assertThat(centroid, equalTo(multiCentroid));
+ }
+
+ @Test
+ public void singleValueFieldAsSubAggToGeohashGrid() throws Exception {
+ SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME)
+ .addAggregation(geohashGrid("geoGrid").field(SINGLE_VALUED_FIELD_NAME)
+ .subAggregation(geoCentroid(aggName)))
+ .execute().actionGet();
+ assertSearchResponse(response);
+
+ GeoHashGrid grid = response.getAggregations().get("geoGrid");
+ assertThat(grid, notNullValue());
+ assertThat(grid.getName(), equalTo("geoGrid"));
+ List<GeoHashGrid.Bucket> buckets = grid.getBuckets();
+ for (int i=0; i < buckets.size(); ++i) {
+ GeoHashGrid.Bucket cell = buckets.get(i);
+ String geohash = cell.getKeyAsString();
+ GeoPoint expectedCentroid = expectedCentroidsForGeoHash.get(geohash);
+ GeoCentroid centroidAgg = cell.getAggregations().get(aggName);
+ assertEquals("Geohash " + geohash + " has wrong centroid ", expectedCentroid, centroidAgg.centroid());
+ }
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/test/geo/RandomGeoGenerator.java b/core/src/test/java/org/elasticsearch/test/geo/RandomGeoGenerator.java
new file mode 100644
index 0000000000..23c7e5c64c
--- /dev/null
+++ b/core/src/test/java/org/elasticsearch/test/geo/RandomGeoGenerator.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.test.geo;
+
+import org.apache.lucene.util.XGeoUtils;
+import org.elasticsearch.common.geo.GeoPoint;
+
+import java.util.Random;
+
+/**
+ * Random geo generation utilities for randomized {@code geo_point} type testing
+ * does not depend on jts or spatial4j. Use {@link org.elasticsearch.test.geo.RandomShapeGenerator}
+ * to create random OGC compliant shapes.
+ */
+public class RandomGeoGenerator {
+
+ public static void randomPoint(Random r, double[] pt) {
+ final double[] min = {-180, -90};
+ final double[] max = {180, 90};
+ randomPointIn(r, min[0], min[1], max[0], max[1], pt);
+ }
+
+ public static void randomPointIn(Random r, final double minLon, final double minLat,
+ final double maxLon, final double maxLat, double[] pt) {
+ assert pt != null && pt.length == 2;
+
+ // normalize min and max
+ double[] min = {XGeoUtils.normalizeLon(minLon), XGeoUtils.normalizeLat(minLat)};
+ double[] max = {XGeoUtils.normalizeLon(maxLon), XGeoUtils.normalizeLat(maxLat)};
+ final double[] tMin = new double[2];
+ final double[] tMax = new double[2];
+ tMin[0] = Math.min(min[0], max[0]);
+ tMax[0] = Math.max(min[0], max[0]);
+ tMin[1] = Math.min(min[1], max[1]);
+ tMax[1] = Math.max(min[1], max[1]);
+
+ pt[0] = tMin[0] + r.nextDouble() * (tMax[0] - tMin[0]);
+ pt[1] = tMin[1] + r.nextDouble() * (tMax[1] - tMin[1]);
+ }
+
+ public static GeoPoint randomPoint(Random r) {
+ return randomPointIn(r, -180, -90, 180, 90);
+ }
+
+ public static GeoPoint randomPointIn(Random r, final double minLon, final double minLat,
+ final double maxLon, final double maxLat) {
+ double[] pt = new double[2];
+ randomPointIn(r, minLon, minLat, maxLon, maxLat, pt);
+ return new GeoPoint(pt[1], pt[0]);
+ }
+}
diff --git a/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java b/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java
index 2bf231e203..e8dda96255 100644
--- a/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java
+++ b/core/src/test/java/org/elasticsearch/test/geo/RandomShapeGenerator.java
@@ -46,9 +46,10 @@ import java.util.Random;
import static com.spatial4j.core.shape.SpatialRelation.CONTAINS;
/**
- * Random geoshape generation utilities for randomized Geospatial testing
+ * Random geoshape generation utilities for randomized {@code geo_shape} type testing
+ * depends on jts and spatial4j
*/
-public class RandomShapeGenerator {
+public class RandomShapeGenerator extends RandomGeoGenerator {
protected static JtsSpatialContext ctx = ShapeBuilder.SPATIAL_CONTEXT;
protected static final double xDIVISIBLE = 2;
@@ -247,11 +248,9 @@ public class RandomShapeGenerator {
}
protected static Point xRandomPointIn(Random rand, Rectangle r) {
- double x = r.getMinX() + rand.nextDouble()*r.getWidth();
- double y = r.getMinY() + rand.nextDouble()*r.getHeight();
- x = xNormX(x);
- y = xNormY(y);
- Point p = ctx.makePoint(x,y);
+ double[] pt = new double[2];
+ randomPointIn(rand, r.getMinX(), r.getMinY(), r.getMaxX(), r.getMaxY(), pt);
+ Point p = ctx.makePoint(pt[0], pt[1]);
RandomizedTest.assertEquals(CONTAINS, r.relate(p));
return p;
}
@@ -314,12 +313,4 @@ public class RandomShapeGenerator {
}
return ctx.makeRectangle(minX, maxX, minY, maxY);
}
-
- protected static double xNormX(double x) {
- return ctx.isGeo() ? DistanceUtils.normLonDEG(x) : x;
- }
-
- protected static double xNormY(double y) {
- return ctx.isGeo() ? DistanceUtils.normLatDEG(y) : y;
- }
}