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.
This commit is contained in:
parent
5d001d1578
commit
b31d3ddd3e
|
@ -285,38 +285,46 @@ public class GeoUtils {
|
||||||
* @param normLon Whether to normalize longitude.
|
* @param normLon Whether to normalize longitude.
|
||||||
*/
|
*/
|
||||||
public static void normalizePoint(GeoPoint point, boolean normLat, boolean normLon) {
|
public static void normalizePoint(GeoPoint point, boolean normLat, boolean normLon) {
|
||||||
double lat = point.lat();
|
double[] pt = {point.lon(), point.lat()};
|
||||||
double lon = point.lon();
|
normalizePoint(pt, normLon, normLat);
|
||||||
|
point.reset(pt[1], pt[0]);
|
||||||
normLat = normLat && (lat>90 || lat <= -90);
|
}
|
||||||
normLon = normLon && (lon>180 || lon <= -180);
|
|
||||||
|
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) {
|
if (normLat) {
|
||||||
lat = centeredModulus(lat, 360);
|
lonLat[1] = centeredModulus(lonLat[1], 360);
|
||||||
boolean shift = true;
|
boolean shift = true;
|
||||||
if (lat < -90) {
|
if (lonLat[1] < -90) {
|
||||||
lat = -180 - lat;
|
lonLat[1] = -180 - lonLat[1];
|
||||||
} else if (lat > 90) {
|
} else if (lonLat[1] > 90) {
|
||||||
lat = 180 - lat;
|
lonLat[1] = 180 - lonLat[1];
|
||||||
} else {
|
} else {
|
||||||
// No need to shift the longitude, and the latitude is normalized
|
// No need to shift the longitude, and the latitude is normalized
|
||||||
shift = false;
|
shift = false;
|
||||||
}
|
}
|
||||||
if (shift) {
|
if (shift) {
|
||||||
if (normLon) {
|
if (normLon) {
|
||||||
lon += 180;
|
lonLat[0] += 180;
|
||||||
} else {
|
} else {
|
||||||
// Longitude won't be normalized,
|
// Longitude won't be normalized,
|
||||||
// keep it in the form x+k*360 (with x in ]-180;180])
|
// 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.
|
// 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) {
|
if (normLon) {
|
||||||
lon = centeredModulus(lon, 360);
|
lonLat[0] = centeredModulus(lonLat[0], 360);
|
||||||
}
|
}
|
||||||
point.reset(lat, lon);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static double centeredModulus(double dividend, double divisor) {
|
private static double centeredModulus(double dividend, double divisor) {
|
||||||
|
|
|
@ -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.cardinality.InternalCardinality;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsParser;
|
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsParser;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.InternalGeoBounds;
|
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.InternalMax;
|
||||||
import org.elasticsearch.search.aggregations.metrics.max.MaxParser;
|
import org.elasticsearch.search.aggregations.metrics.max.MaxParser;
|
||||||
import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
|
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(ReverseNestedParser.class);
|
||||||
multibinderAggParser.addBinding().to(TopHitsParser.class);
|
multibinderAggParser.addBinding().to(TopHitsParser.class);
|
||||||
multibinderAggParser.addBinding().to(GeoBoundsParser.class);
|
multibinderAggParser.addBinding().to(GeoBoundsParser.class);
|
||||||
|
multibinderAggParser.addBinding().to(GeoCentroidParser.class);
|
||||||
multibinderAggParser.addBinding().to(ScriptedMetricParser.class);
|
multibinderAggParser.addBinding().to(ScriptedMetricParser.class);
|
||||||
multibinderAggParser.addBinding().to(ChildrenParser.class);
|
multibinderAggParser.addBinding().to(ChildrenParser.class);
|
||||||
for (Class<? extends Aggregator.Parser> parser : aggParsers) {
|
for (Class<? extends Aggregator.Parser> parser : aggParsers) {
|
||||||
|
@ -357,6 +360,7 @@ public class SearchModule extends AbstractModule {
|
||||||
InternalHDRPercentileRanks.registerStreams();
|
InternalHDRPercentileRanks.registerStreams();
|
||||||
InternalCardinality.registerStreams();
|
InternalCardinality.registerStreams();
|
||||||
InternalScriptedMetric.registerStreams();
|
InternalScriptedMetric.registerStreams();
|
||||||
|
InternalGeoCentroid.registerStreams();
|
||||||
|
|
||||||
// buckets
|
// buckets
|
||||||
InternalGlobal.registerStreams();
|
InternalGlobal.registerStreams();
|
||||||
|
|
|
@ -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.cardinality.CardinalityBuilder;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsBuilder;
|
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.Max;
|
||||||
import org.elasticsearch.search.aggregations.metrics.max.MaxBuilder;
|
import org.elasticsearch.search.aggregations.metrics.max.MaxBuilder;
|
||||||
import org.elasticsearch.search.aggregations.metrics.min.Min;
|
import org.elasticsearch.search.aggregations.metrics.min.Min;
|
||||||
|
@ -287,6 +289,13 @@ public class AggregationBuilders {
|
||||||
return new GeoBoundsBuilder(name);
|
return new GeoBoundsBuilder(name);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
* Create a new {@link ScriptedMetric} aggregation with the given name.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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.bucket.terms.Terms;
|
||||||
import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
|
import org.elasticsearch.search.aggregations.metrics.cardinality.Cardinality;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
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.percentiles.Percentiles;
|
||||||
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
import org.elasticsearch.search.aggregations.metrics.stats.Stats;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
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.cardinality;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds;
|
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.histogram;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.percentiles;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.stats;
|
||||||
|
@ -192,4 +194,11 @@ public class MissingValueIT extends ESIntegTestCase {
|
||||||
assertEquals(new GeoPoint(2,1), bounds.topLeft());
|
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());
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -19,35 +19,23 @@
|
||||||
|
|
||||||
package org.elasticsearch.search.aggregations.metrics;
|
package org.elasticsearch.search.aggregations.metrics;
|
||||||
|
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
|
||||||
import org.elasticsearch.action.search.SearchResponse;
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
import org.elasticsearch.common.geo.GeoPoint;
|
import org.elasticsearch.common.geo.GeoPoint;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.common.util.BigArray;
|
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.global.Global;
|
||||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Bucket;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBounds;
|
||||||
import org.elasticsearch.search.aggregations.metrics.geobounds.GeoBoundsAggregator;
|
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.elasticsearch.test.ESIntegTestCase;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.geoBounds;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
|
import static org.elasticsearch.search.aggregations.AggregationBuilders.global;
|
||||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
|
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.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
|
||||||
import static org.hamcrest.Matchers.allOf;
|
import static org.hamcrest.Matchers.allOf;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
@ -60,167 +48,22 @@ import static org.hamcrest.Matchers.sameInstance;
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@ESIntegTestCase.SuiteScopeTestCase
|
@ESIntegTestCase.SuiteScopeTestCase
|
||||||
public class GeoBoundsIT extends ESIntegTestCase {
|
public class GeoBoundsIT extends AbstractGeoTestCase {
|
||||||
|
|
||||||
private static final String SINGLE_VALUED_FIELD_NAME = "geo_value";
|
private static final String aggName = "geoBounds";
|
||||||
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()));
|
|
||||||
}
|
|
||||||
|
|
||||||
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);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void singleValuedField() throws Exception {
|
public void singleValuedField() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx")
|
SearchResponse response = client().prepareSearch(IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
||||||
|
@ -232,10 +75,10 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
@Test
|
@Test
|
||||||
public void testSingleValuedField_getProperty() throws Exception {
|
public void testSingleValuedField_getProperty() throws Exception {
|
||||||
SearchResponse searchResponse = client()
|
SearchResponse searchResponse = client()
|
||||||
.prepareSearch("idx")
|
.prepareSearch(IDX_NAME)
|
||||||
.setQuery(matchAllQuery())
|
.setQuery(matchAllQuery())
|
||||||
.addAggregation(
|
.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();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(searchResponse);
|
assertSearchResponse(searchResponse);
|
||||||
|
@ -247,35 +90,35 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
assertThat(global.getAggregations(), notNullValue());
|
assertThat(global.getAggregations(), notNullValue());
|
||||||
assertThat(global.getAggregations().asMap().size(), equalTo(1));
|
assertThat(global.getAggregations().asMap().size(), equalTo(1));
|
||||||
|
|
||||||
GeoBounds geobounds = global.getAggregations().get("geoBounds");
|
GeoBounds geobounds = global.getAggregations().get(aggName);
|
||||||
assertThat(geobounds, notNullValue());
|
assertThat(geobounds, notNullValue());
|
||||||
assertThat(geobounds.getName(), equalTo("geoBounds"));
|
assertThat(geobounds.getName(), equalTo(aggName));
|
||||||
assertThat((GeoBounds) global.getProperty("geoBounds"), sameInstance(geobounds));
|
assertThat((GeoBounds) global.getProperty(aggName), sameInstance(geobounds));
|
||||||
GeoPoint topLeft = geobounds.topLeft();
|
GeoPoint topLeft = geobounds.topLeft();
|
||||||
GeoPoint bottomRight = geobounds.bottomRight();
|
GeoPoint bottomRight = geobounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
||||||
assertThat(topLeft.lon(), equalTo(singleTopLeft.lon()));
|
assertThat(topLeft.lon(), equalTo(singleTopLeft.lon()));
|
||||||
assertThat(bottomRight.lat(), equalTo(singleBottomRight.lat()));
|
assertThat(bottomRight.lat(), equalTo(singleBottomRight.lat()));
|
||||||
assertThat(bottomRight.lon(), equalTo(singleBottomRight.lon()));
|
assertThat(bottomRight.lon(), equalTo(singleBottomRight.lon()));
|
||||||
assertThat((double) global.getProperty("geoBounds.top"), equalTo(singleTopLeft.lat()));
|
assertThat((double) global.getProperty(aggName + ".top"), equalTo(singleTopLeft.lat()));
|
||||||
assertThat((double) global.getProperty("geoBounds.left"), equalTo(singleTopLeft.lon()));
|
assertThat((double) global.getProperty(aggName + ".left"), equalTo(singleTopLeft.lon()));
|
||||||
assertThat((double) global.getProperty("geoBounds.bottom"), equalTo(singleBottomRight.lat()));
|
assertThat((double) global.getProperty(aggName + ".bottom"), equalTo(singleBottomRight.lat()));
|
||||||
assertThat((double) global.getProperty("geoBounds.right"), equalTo(singleBottomRight.lon()));
|
assertThat((double) global.getProperty(aggName + ".right"), equalTo(singleBottomRight.lon()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void multiValuedField() throws Exception {
|
public void multiValuedField() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx")
|
SearchResponse response = client().prepareSearch(IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(MULTI_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(MULTI_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
|
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(multiTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(multiTopLeft.lat()));
|
||||||
|
@ -286,17 +129,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void unmapped() throws Exception {
|
public void unmapped() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx_unmapped")
|
SearchResponse response = client().prepareSearch(UNMAPPED_IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft, equalTo(null));
|
assertThat(topLeft, equalTo(null));
|
||||||
|
@ -305,17 +147,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void partiallyUnmapped() throws Exception {
|
public void partiallyUnmapped() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx", "idx_unmapped")
|
SearchResponse response = client().prepareSearch(IDX_NAME, UNMAPPED_IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(singleTopLeft.lat()));
|
||||||
|
@ -326,16 +167,16 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void emptyAggregation() throws Exception {
|
public void emptyAggregation() throws Exception {
|
||||||
SearchResponse searchResponse = client().prepareSearch("empty_idx")
|
SearchResponse searchResponse = client().prepareSearch(EMPTY_IDX_NAME)
|
||||||
.setQuery(matchAllQuery())
|
.setQuery(matchAllQuery())
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertThat(searchResponse.getHits().getTotalHits(), equalTo(0l));
|
assertThat(searchResponse.getHits().getTotalHits(), equalTo(0l));
|
||||||
GeoBounds geoBounds = searchResponse.getAggregations().get("geoBounds");
|
GeoBounds geoBounds = searchResponse.getAggregations().get(aggName);
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft, equalTo(null));
|
assertThat(topLeft, equalTo(null));
|
||||||
|
@ -344,8 +185,8 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void singleValuedFieldNearDateLine() throws Exception {
|
public void singleValuedFieldNearDateLine() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx_dateline")
|
SearchResponse response = client().prepareSearch(DATELINE_IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false))
|
.wrapLongitude(false))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
|
@ -354,9 +195,9 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
GeoPoint geoValuesTopLeft = new GeoPoint(38, -179);
|
GeoPoint geoValuesTopLeft = new GeoPoint(38, -179);
|
||||||
GeoPoint geoValuesBottomRight = new GeoPoint(-24, 178);
|
GeoPoint geoValuesBottomRight = new GeoPoint(-24, 178);
|
||||||
|
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
|
||||||
|
@ -371,15 +212,15 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
GeoPoint geoValuesTopLeft = new GeoPoint(38, 170);
|
GeoPoint geoValuesTopLeft = new GeoPoint(38, 170);
|
||||||
GeoPoint geoValuesBottomRight = new GeoPoint(-24, -175);
|
GeoPoint geoValuesBottomRight = new GeoPoint(-24, -175);
|
||||||
|
|
||||||
SearchResponse response = client().prepareSearch("idx_dateline")
|
SearchResponse response = client().prepareSearch(DATELINE_IDX_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(true))
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(true))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
|
assertThat(topLeft.lat(), equalTo(geoValuesTopLeft.lat()));
|
||||||
|
@ -393,8 +234,8 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void singleValuedFieldAsSubAggToHighCardTermsAgg() {
|
public void singleValuedFieldAsSubAggToHighCardTermsAgg() {
|
||||||
SearchResponse response = client().prepareSearch("high_card_idx")
|
SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME)
|
||||||
.addAggregation(terms("terms").field(NUMBER_FIELD_NAME).subAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME)
|
.addAggregation(terms("terms").field(NUMBER_FIELD_NAME).subAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME)
|
||||||
.wrapLongitude(false)))
|
.wrapLongitude(false)))
|
||||||
.execute().actionGet();
|
.execute().actionGet();
|
||||||
|
|
||||||
|
@ -409,9 +250,9 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
Bucket bucket = buckets.get(i);
|
Bucket bucket = buckets.get(i);
|
||||||
assertThat(bucket, notNullValue());
|
assertThat(bucket, notNullValue());
|
||||||
assertThat("Bucket " + bucket.getKey() + " has wrong number of documents", bucket.getDocCount(), equalTo(1l));
|
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, 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().getLat(), allOf(greaterThanOrEqualTo(-90.0), lessThanOrEqualTo(90.0)));
|
||||||
assertThat(geoBounds.topLeft().getLon(), allOf(greaterThanOrEqualTo(-180.0), lessThanOrEqualTo(180.0)));
|
assertThat(geoBounds.topLeft().getLon(), allOf(greaterThanOrEqualTo(-180.0), lessThanOrEqualTo(180.0)));
|
||||||
assertThat(geoBounds.bottomRight().getLat(), allOf(greaterThanOrEqualTo(-90.0), lessThanOrEqualTo(90.0)));
|
assertThat(geoBounds.bottomRight().getLat(), allOf(greaterThanOrEqualTo(-90.0), lessThanOrEqualTo(90.0)));
|
||||||
|
@ -421,14 +262,14 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void singleValuedFieldWithZeroLon() throws Exception {
|
public void singleValuedFieldWithZeroLon() throws Exception {
|
||||||
SearchResponse response = client().prepareSearch("idx_zero")
|
SearchResponse response = client().prepareSearch(IDX_ZERO_NAME)
|
||||||
.addAggregation(geoBounds("geoBounds").field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)).execute().actionGet();
|
.addAggregation(geoBounds(aggName).field(SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)).execute().actionGet();
|
||||||
|
|
||||||
assertSearchResponse(response);
|
assertSearchResponse(response);
|
||||||
|
|
||||||
GeoBounds geoBounds = response.getAggregations().get("geoBounds");
|
GeoBounds geoBounds = response.getAggregations().get(aggName);
|
||||||
assertThat(geoBounds, notNullValue());
|
assertThat(geoBounds, notNullValue());
|
||||||
assertThat(geoBounds.getName(), equalTo("geoBounds"));
|
assertThat(geoBounds.getName(), equalTo(aggName));
|
||||||
GeoPoint topLeft = geoBounds.topLeft();
|
GeoPoint topLeft = geoBounds.topLeft();
|
||||||
GeoPoint bottomRight = geoBounds.bottomRight();
|
GeoPoint bottomRight = geoBounds.bottomRight();
|
||||||
assertThat(topLeft.lat(), equalTo(1.0));
|
assertThat(topLeft.lat(), equalTo(1.0));
|
||||||
|
@ -436,5 +277,4 @@ public class GeoBoundsIT extends ESIntegTestCase {
|
||||||
assertThat(bottomRight.lat(), equalTo(1.0));
|
assertThat(bottomRight.lat(), equalTo(1.0));
|
||||||
assertThat(bottomRight.lon(), equalTo(0.0));
|
assertThat(bottomRight.lon(), equalTo(0.0));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -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]);
|
||||||
|
}
|
||||||
|
}
|
|
@ -46,9 +46,10 @@ import java.util.Random;
|
||||||
import static com.spatial4j.core.shape.SpatialRelation.CONTAINS;
|
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 JtsSpatialContext ctx = ShapeBuilder.SPATIAL_CONTEXT;
|
||||||
protected static final double xDIVISIBLE = 2;
|
protected static final double xDIVISIBLE = 2;
|
||||||
|
@ -247,11 +248,9 @@ public class RandomShapeGenerator {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected static Point xRandomPointIn(Random rand, Rectangle r) {
|
protected static Point xRandomPointIn(Random rand, Rectangle r) {
|
||||||
double x = r.getMinX() + rand.nextDouble()*r.getWidth();
|
double[] pt = new double[2];
|
||||||
double y = r.getMinY() + rand.nextDouble()*r.getHeight();
|
randomPointIn(rand, r.getMinX(), r.getMinY(), r.getMaxX(), r.getMaxY(), pt);
|
||||||
x = xNormX(x);
|
Point p = ctx.makePoint(pt[0], pt[1]);
|
||||||
y = xNormY(y);
|
|
||||||
Point p = ctx.makePoint(x,y);
|
|
||||||
RandomizedTest.assertEquals(CONTAINS, r.relate(p));
|
RandomizedTest.assertEquals(CONTAINS, r.relate(p));
|
||||||
return p;
|
return p;
|
||||||
}
|
}
|
||||||
|
@ -314,12 +313,4 @@ public class RandomShapeGenerator {
|
||||||
}
|
}
|
||||||
return ctx.makeRectangle(minX, maxX, minY, maxY);
|
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;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,8 @@ include::metrics/extendedstats-aggregation.asciidoc[]
|
||||||
|
|
||||||
include::metrics/geobounds-aggregation.asciidoc[]
|
include::metrics/geobounds-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/geocentroid-aggregation.asciidoc[]
|
||||||
|
|
||||||
include::metrics/max-aggregation.asciidoc[]
|
include::metrics/max-aggregation.asciidoc[]
|
||||||
|
|
||||||
include::metrics/min-aggregation.asciidoc[]
|
include::metrics/min-aggregation.asciidoc[]
|
||||||
|
|
|
@ -0,0 +1,104 @@
|
||||||
|
[[search-aggregations-metrics-geocentroid-aggregation]]
|
||||||
|
=== Geo Centroid Aggregation
|
||||||
|
|
||||||
|
A metric aggregation that computes the weighted centroid from all coordinate values for a <<geo-point>> field.
|
||||||
|
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"query" : {
|
||||||
|
"match" : { "crime" : "burglary" }
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"centroid" : {
|
||||||
|
"geo_centroid" : {
|
||||||
|
"field" : "location" <1>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The `geo_centroid` aggregation specifies the field to use for computing the centroid. (NOTE: field must be a <<geo-point>> type)
|
||||||
|
|
||||||
|
The above aggregation demonstrates how one would compute the centroid of the location field for all documents with a crime type of burglary
|
||||||
|
|
||||||
|
The response for the above aggregation:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"centroid": {
|
||||||
|
"location": {
|
||||||
|
"lat": 80.45,
|
||||||
|
"lon": -160.22
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
The `geo_centroid` aggregation is more interesting when combined as a sub-aggregation to other bucket aggregations.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"query" : {
|
||||||
|
"match" : { "crime" : "burglary" }
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"towns" : {
|
||||||
|
"terms" : { "field" : "town" },
|
||||||
|
"aggs" : {
|
||||||
|
"centroid" : {
|
||||||
|
"geo_centroid" : { "field" : "location" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The above example uses `geo_centroid` as a sub-aggregation to a <<search-aggregations-bucket-terms-aggregation, terms>> bucket aggregation
|
||||||
|
for finding the central location for all crimes of type burglary in each town.
|
||||||
|
|
||||||
|
The response for the above aggregation:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"buckets": [
|
||||||
|
{
|
||||||
|
"key": "Los Altos",
|
||||||
|
"doc_count": 113,
|
||||||
|
"centroid": {
|
||||||
|
"location": {
|
||||||
|
"lat": 37.3924582824111,
|
||||||
|
"lon": -122.12104808539152
|
||||||
|
}
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": "Mountain View",
|
||||||
|
"doc_count": 92,
|
||||||
|
"centroid": {
|
||||||
|
"location": {
|
||||||
|
"lat": 37.382152481004596,
|
||||||
|
"lon": -122.08116559311748
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
Loading…
Reference in New Issue