parent
34d567a133
commit
c7f6c5266d
|
@ -83,6 +83,8 @@ include::search/request-body.asciidoc[]
|
||||||
|
|
||||||
include::search/facets.asciidoc[]
|
include::search/facets.asciidoc[]
|
||||||
|
|
||||||
|
include::search/aggregations.asciidoc[]
|
||||||
|
|
||||||
include::search/suggesters.asciidoc[]
|
include::search/suggesters.asciidoc[]
|
||||||
|
|
||||||
include::search/multi-search.asciidoc[]
|
include::search/multi-search.asciidoc[]
|
||||||
|
|
|
@ -0,0 +1,70 @@
|
||||||
|
[[search-aggregations]]
|
||||||
|
== Aggregations
|
||||||
|
|
||||||
|
Aggregations grew out of the <<search-facets, facets>> module and the long expirience of how users use it (and would like to use it) for real-time data analytics purposes. As such, it serves as the next generation replacement for the functionality we currently refer to as "faceting".
|
||||||
|
|
||||||
|
<<search-facets, Facets>> provide a great way to aggregate data within a document set context. This context is defined by the executed query in combination with the different levels of filters that can be defined (filtered queries, top level filters, and facet level filters). While powerful, their implementation is not designed from ground up to support complex aggregations and thus limited.
|
||||||
|
|
||||||
|
.Are facets deprecated?
|
||||||
|
**********************************
|
||||||
|
As the functionality facets offer is a subset of the the one offered by aggregations, over time, we would like to see users move to aggregations for all realtime data analytics. That said, we are well aware that such transitions/migrations take time, and for this reason we are keeping the facets around for the time being. Nonetheless, facets are and should be considered deprecated and will likely be removed in one of the future major releases.
|
||||||
|
**********************************
|
||||||
|
|
||||||
|
The aggregations module breaks the barriers the current facet implementation put in place. The new name ("Aggregations") also indicate the intention here - a generic yet extremely powerful framework for building aggregations - any types of aggregations.
|
||||||
|
|
||||||
|
An aggregation can be seen as a _unit-of-work_ that builds analytic information over a set of documents. The context of the execution defines what this document set is (e.g. a top level aggregation executes within the context of the executed query/filters of the search request).
|
||||||
|
|
||||||
|
There are many different types of aggregations, each with its own purpose and output. To better understand these types, it is often easier to break them into two main families:
|
||||||
|
|
||||||
|
_Bucketing_::
|
||||||
|
A family of aggregations that build buckets, where each bucket is associated with a _key_ and a document criteria. When the aggregations is executed, the buckets criterias are evaluated on every document in the context and when matches, the document is considered to "fall in" the relevant bucket. By the end of the aggreagation process, we'll end up with a list of buckets - each one with a set of documents that "belong" to it.
|
||||||
|
|
||||||
|
_Metric_::
|
||||||
|
Aggregations that keep track and compute metrics over a set of documents
|
||||||
|
|
||||||
|
The interesting part comes next, since each bucket effectively defines a document set (all documents belonging to the bucket), one can potentially associated aggregations on the bucket level, and those will execute within the context of that bucket. This is where the real power of aggregations kicks in: *aggregations can be nested!*
|
||||||
|
|
||||||
|
NOTE: Bucketing aggregations can have sub-aggregations (bucketing or metric). The sub aggregations will be computed for
|
||||||
|
each of the buckets their parent aggregation generates. There is not hard limit on the level/depth of nested aggregations (one can nest an aggregation under a "parent" aggregation which is itself a sub-aggregation of another highter aggregations)
|
||||||
|
|
||||||
|
=== Structuring Aggregations
|
||||||
|
|
||||||
|
The following snippet captures the basic structure of aggregations:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
"aggregations" : {
|
||||||
|
"<aggregation_name>" : {
|
||||||
|
"<aggregation_type>" : {
|
||||||
|
<aggregation_body>
|
||||||
|
}
|
||||||
|
[,"aggregations" : { [<sub_aggregation>]+ } ]?
|
||||||
|
}
|
||||||
|
[,"<aggregation_name_2>" : { ... } ]*
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The `aggregations` object (a.k.a `aggs` for short) in the json holds the aggregations to be computed. Each aggregation is associated with a logical name that the user defines (e.g. if the aggregation computes the average price, then it'll make sense to name it `avg_price`). These logical names will also be used to uniquely identify the aggregations in the response. Each aggregation has a specific type (`<aggregation_type>` in the above snippet) and is typically the first key within the named aggregation body. Each type of aggregation define its own body, depending on the nature of the aggregation (eg. an `avg` aggregation on a specific field will define the field on which the avg will be calculated). At the same level of the aggregation type definition, one can optionally define a set of additional aggregations, though this only makes sense if the aggregation you defined is of a bucketing nature. In this scenario, the sub-aggregations you define on the bucketing aggregation level will be computed for all the buckets built by the bucketing aggregation. For example, if the you define a set of aggregations under the `range` aggregation, the sub-aggregations will be computed for each of the range buckets that are defined.
|
||||||
|
|
||||||
|
==== Values Source
|
||||||
|
|
||||||
|
Some aggregations work on values extracted from the aggregated documents. Typically, the values will be extracted from a sepcific document field which is set under the `field` settings for the aggrations. It is also possible to define a `<<modules-scripting,script>>` that will generate the values (per document).
|
||||||
|
|
||||||
|
When both `field` and `script` settings are configured for the aggregation, the script will be treated as a `value script`. While normal scripts are evaluated on a document level (i.e. the script has access to all the data associated with the document), value scripts are evaluated on the *value* level. In this mode, the values are extracted from the configured `field` and the `script` is used to apply a "transformation" over these value/s
|
||||||
|
|
||||||
|
["NOTE",id="metrics-script-note"]
|
||||||
|
===============================
|
||||||
|
When working with scripts, the `script_lang` and `params` settings can also be defined. The former defines the scripting language that is used (assuming the proper language is available in es either by default or as a plugin). The latter enables defining all the "dynamic" expressions in the script as parameters, and by that keep the script itself static between calls (this will ensure the use of the cached compiled scripts in elasticsearch).
|
||||||
|
===============================
|
||||||
|
|
||||||
|
Scripts can generate a single value or multiple values per documents. When generating multiple values, once can use the `script_values_sorted` settings to indicate whether these values are sorted or not. Internally, elasticsearch can perform optimizations when dealing with sorted values (for example, with the `min` aggregations, knowing the values are sorted, elasticsearch will skip the iterations over all the values and rely on the first value in the list to be the minimum value among all other values associated with the same document).
|
||||||
|
|
||||||
|
include::aggregations/metrics.asciidoc[]
|
||||||
|
|
||||||
|
include::aggregations/bucket.asciidoc[]
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,30 @@
|
||||||
|
[[search-aggregations-bucket]]
|
||||||
|
=== Bucket Aggregations
|
||||||
|
|
||||||
|
Bucket aggregations don't calculate metrics over fields like the metrics aggregations do, but instead, they create buckets of documents. Each bucket is associated with a criteria (depends on the aggregation type) that determines whether or not a document in the current context "falls" in it. In other words, the buckets effectively define document sets. In addition to the buckets themselves, the `bucket` aggregations also compute and return the number of documents that "fell in" each bucket.
|
||||||
|
|
||||||
|
Bucket aggregations, as opposed to `metrics` aggregations, can hold sub-aggregations. These sub aggregations will be aggregated for each of the buckets created by their "parent" bucket aggregation.
|
||||||
|
|
||||||
|
There are different bucket aggregators, each with a different "bucketing" strategy. Some define a single bucket, some define fixed number of multiple buckets, and others dynamically create the buckets during the aggregation process.
|
||||||
|
|
||||||
|
include::bucket/global-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/filter-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/missing-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/nested-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/terms-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/range-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/daterange-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/iprange-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/histogram-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/datehistogram-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::bucket/geodistance-aggregation.asciidoc[]
|
|
@ -0,0 +1,106 @@
|
||||||
|
[[search-aggregations-bucket-datehistogram-aggregation]]
|
||||||
|
=== Date Histogram
|
||||||
|
|
||||||
|
A multi-bucket aggregation similar to the <<search-aggregations-bucket-histogram-aggregation,histogram>> except it can only be applied on date values. Since dates are represented in elasticsearch internally as long values, it is possible to use the normal `histogram` on dates as well, though accuracy will be compromized. The reason for this is in the fact that time based intervals are not fixed (think of leap years and on the number of days in a month). For this reason, we need a spcial support for time based data. From functionality perspective, this historam supports the same features as the normal <<search-aggregations-bucket-histogram-aggregation,histogram>>. The main difference though is that the interval can be specified by date/time expressions.
|
||||||
|
|
||||||
|
Requesting a month length bucket intervals
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"articles_over_time" : {
|
||||||
|
"date_histogram" : {
|
||||||
|
"field" : "date",
|
||||||
|
"interval" : "month"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
or based on 1.5 months
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"articles_over_time" : {
|
||||||
|
"date_histogram" : {
|
||||||
|
"field" : "date",
|
||||||
|
"interval" : "1.5M"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Other available expressions for interval: `year`, `quarter`, `week`, `day`, `hour`, `minute`, `second`
|
||||||
|
|
||||||
|
==== Time Zone
|
||||||
|
|
||||||
|
By default, times are stored as UTC milliseconds since the epoch. Thus, all computation and "bucketing" / "rounding" is done on UTC. It is possible to provide a time zone (both pre rounding, and post rounding) value, which will cause all computations to take the relevant zone into account. The time returned for each bucket/entry is milliseconds since the epoch of the provided time zone.
|
||||||
|
|
||||||
|
The parameters are `pre_zone` (pre rounding based on interval) and `post_zone` (post rounding based on interval). The `time_zone` parameter simply sets the `pre_zone` parameter. By default, those are set to `UTC`.
|
||||||
|
|
||||||
|
The zone value accepts either a numeric value for the hours offset, for example: `"time_zone" : -2`. It also accepts a format of hours and minutes, like `"time_zone" : "-02:30"`. Another option is to provide a time zone accepted as one of the values listed here.
|
||||||
|
|
||||||
|
Lets take an example. For `2012-04-01T04:15:30Z`, with a `pre_zone` of `-08:00`. For day interval, the actual time by applying the time zone and rounding falls under `2012-03-31`, so the returned value will be (in millis) of `2012-03-31T00:00:00Z` (UTC). For hour interval, applying the time zone results in `2012-03-31T20:15:30`, rounding it results in `2012-03-31T20:00:00`, but, we want to return it in UTC (`post_zone` is not set), so we convert it back to UTC: `2012-04-01T04:00:00Z`. Note, we are consistent in the results, returning the rounded value in UTC.
|
||||||
|
|
||||||
|
`post_zone` simply takes the result, and adds the relevant offset.
|
||||||
|
|
||||||
|
Sometimes, we want to apply the same conversion to UTC we did above for hour also for day (and up) intervals. We can set `pre_zone_adjust_large_interval` to `true`, which will apply the same conversion done for hour interval in the example, to day and above intervals (it can be set regardless of the interval, but only kick in when using day and higher intervals).
|
||||||
|
|
||||||
|
==== Factor
|
||||||
|
|
||||||
|
The date histogram works on numeric values (since time is stored in milliseconds since the epoch in UTC). But, sometimes, systems will store a different resolution (like seconds since UTC) in a numeric field. The `factor` parameter can be used to change the value in the field to milliseconds to actual do the relevant rounding, and then be applied again to get to the original unit. For example, when storing in a numeric field seconds resolution, the factor can be set to 1000.
|
||||||
|
|
||||||
|
==== Pre/Post Offset
|
||||||
|
|
||||||
|
Specific offsets can be provided for pre rounding and post rounding. The `pre_offset` for pre rounding, and `post_offset` for post rounding. The format is the date time format (`1h`, `1d`, etc...).
|
||||||
|
|
||||||
|
==== Keys
|
||||||
|
|
||||||
|
Since internally, dates are represented as 64bit numbers, these numbers are returned as the bucket keys (each key representing a date - milliseconds since the epoch). It is also possible to define a date format, which will result in returning the dates as formatted strings next to the numeric key values:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"articles_over_time" : {
|
||||||
|
"date_histogram" : {
|
||||||
|
"field" : "date",
|
||||||
|
"interval" : "1M",
|
||||||
|
"format" : "yyyy-MM-dd" <1>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> Supports expressive date <<date-format-pattern,format pattern>>
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"articles_over_time": [
|
||||||
|
{
|
||||||
|
"key_as_string": "2013-02-02",
|
||||||
|
"key": 1328140800000,
|
||||||
|
"doc_count": 1
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key_as_string": "2013-03-02",
|
||||||
|
"key": 1330646400000,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
...
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Like with the normal <<search-aggregations-bucket-histogram-aggregation,histogram>>, both document level scripts and value level scripts are supported. It is also possilbe to control the order of the returned buckets using the `order` settings and empty buckets can also be returned by setting the `empty_buckets` field to `true` (defaults to `false`).
|
|
@ -0,0 +1,106 @@
|
||||||
|
[[search-aggregations-bucket-daterange-aggregation]]
|
||||||
|
=== Date Range
|
||||||
|
|
||||||
|
A range aggregation that is dedicated for date values. The main difference between this aggregation and the normal <<search-aggregations-bucket-range-aggregation,range>> aggregation is that the `from` and `to` values can be expressed in Date Math expressions, and it is also possible to specify a date format by which the `from` and `to` response fields will be returned:
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs": {
|
||||||
|
"range": {
|
||||||
|
"date_range": {
|
||||||
|
"field": "date",
|
||||||
|
"format": "MM-yyy",
|
||||||
|
"ranges": [
|
||||||
|
{ "to": "now-10M/M" },
|
||||||
|
{ "from": "now-10M/M" }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
In the example above, we created two range buckets, the first will "bucket" all documents dated prior to 10 months ago and
|
||||||
|
the second will "bucket" all documents dated since 10 months ago
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"range": [
|
||||||
|
{
|
||||||
|
"to": 1.3437792E+12,
|
||||||
|
"to_as_string": "08-2012",
|
||||||
|
"doc_count": 7
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 1.3437792E+12,
|
||||||
|
"from_as_string": "08-2012",
|
||||||
|
"doc_count": 2
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
[[date-format-pattern]]
|
||||||
|
==== Date Format/Pattern (copied from http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html[JodaDate])
|
||||||
|
|
||||||
|
All ASCII letters are reserved as format pattern letters, which are defined as follows:
|
||||||
|
|
||||||
|
[options="header"]
|
||||||
|
|=======
|
||||||
|
|Symbol |Meaning |Presentation |Examples
|
||||||
|
|G |era |text |AD
|
||||||
|
|C |century of era (>=0) |number |20
|
||||||
|
|Y |year of era (>=0) |year |1996
|
||||||
|
|
||||||
|
|x |weekyear |year |1996
|
||||||
|
|w |week of weekyear |number |27
|
||||||
|
|e |day of week |number |2
|
||||||
|
|E |day of week |text |Tuesday; Tue
|
||||||
|
|
||||||
|
|y |year |year |1996
|
||||||
|
|D |day of year |number |189
|
||||||
|
|M |month of year |month |July; Jul; 07
|
||||||
|
|d |day of month |number |10
|
||||||
|
|
||||||
|
|a |halfday of day |text |PM
|
||||||
|
|K |hour of halfday (0~11) |number |0
|
||||||
|
|h |clockhour of halfday (1~12) |number |12
|
||||||
|
|
||||||
|
|H |hour of day (0~23) |number |0
|
||||||
|
|k |clockhour of day (1~24) |number |24
|
||||||
|
|m |minute of hour |number |30
|
||||||
|
|s |second of minute |number |55
|
||||||
|
|S |fraction of second |number |978
|
||||||
|
|
||||||
|
|z |time zone |text |Pacific Standard Time; PST
|
||||||
|
|Z |time zone offset/id |zone |-0800; -08:00; America/Los_Angeles
|
||||||
|
|
||||||
|
|' |escape for text |delimiter
|
||||||
|
|'' |single quote |literal |'
|
||||||
|
|=======
|
||||||
|
|
||||||
|
The count of pattern letters determine the format.
|
||||||
|
|
||||||
|
Text:: If the number of pattern letters is 4 or more, the full form is used; otherwise a short or abbreviated form is used if available.
|
||||||
|
|
||||||
|
Number:: The minimum number of digits. Shorter numbers are zero-padded to this amount.
|
||||||
|
|
||||||
|
Year:: Numeric presentation for year and weekyear fields are handled specially. For example, if the count of 'y' is 2, the year will be displayed as the zero-based year of the century, which is two digits.
|
||||||
|
|
||||||
|
Month:: 3 or over, use text, otherwise use number.
|
||||||
|
|
||||||
|
Zone:: 'Z' outputs offset without a colon, 'ZZ' outputs the offset with a colon, 'ZZZ' or more outputs the zone id.
|
||||||
|
|
||||||
|
Zone names:: Time zone names ('z') cannot be parsed.
|
||||||
|
|
||||||
|
Any characters in the pattern that are not in the ranges of ['a'..'z'] and ['A'..'Z'] will be treated as quoted text. For instance, characters like ':', '.', ' ', '#' and '?' will appear in the resulting time text even they are not embraced within single quotes.
|
|
@ -0,0 +1,38 @@
|
||||||
|
[[search-aggregations-bucket-filter-aggregation]]
|
||||||
|
=== Filter
|
||||||
|
|
||||||
|
Defines a single bucket of all the documents in the current document set context that match a specified filter. Often this will be used to narrow down the current aggregation context to a specific set of documents.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"in_stock_products" : {
|
||||||
|
"filter" : { "range" : { "stock" : { "gt" : 0 } } },
|
||||||
|
"aggs" : {
|
||||||
|
"avg_price" : { "avg" : { "field" : "price" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
In the above example, we calculate the average price of all the products that are currently in-stock.
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"in_stock_products" : {
|
||||||
|
"doc_count" : 100,
|
||||||
|
"avg_price" : { "value" : 56.3 }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,103 @@
|
||||||
|
[[search-aggregations-bucket-geodistance-aggregation]]
|
||||||
|
=== Geo Distance
|
||||||
|
|
||||||
|
A multi-bucket aggregation that works on `geo_point` fields and onceptually works very similar to the <<search-aggregations-bucket-range-aggregation,range>> aggregation. The user can define a point of origin and a set of distance range buckets. The aggregation evaluate the distance of each document value from the origin point and determines the buckets it belongs to based on the ranges (a document belongs to a bucket if the distance between the document and the origin falls within the distance range of the bucket).
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"rings_around_amsterdam" : {
|
||||||
|
"geo_distance" : {
|
||||||
|
"field" : "location",
|
||||||
|
"origin" : "52.3760, 4.894",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 100 },
|
||||||
|
{ "from" : 100, "to" : 300 },
|
||||||
|
{ "from" : 300 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"rings": [
|
||||||
|
{
|
||||||
|
"unit": "km",
|
||||||
|
"to": 100.0,
|
||||||
|
"doc_count": 3
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"unit": "km",
|
||||||
|
"from": 100.0,
|
||||||
|
"to": 300.0,
|
||||||
|
"doc_count": 1
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"unit": "km",
|
||||||
|
"from": 300.0,
|
||||||
|
"doc_count": 7
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The specified field must be of type `geo_point` (which can only be set explicitly in the mappings). And it can also hold an array of `geo_point` fields, in which case all will be taken into account during aggregation. The origin point can accept all formats supported by the `geo_point` <<mapping-geo-point-type,type>>:
|
||||||
|
|
||||||
|
* Object format: `{ "lat" : 52.3760, "lon" : 4.894 }` - this is the safest format as it is the most explicit about the `lat` & `lon` values
|
||||||
|
* String format: `"52.3760, 4.894"` - where the first number is the `lat` and the second is the `lon`
|
||||||
|
* Array format: `[4.894, 52.3760]` - which is based on the `GeoJson` standard and where the first number is the `lon` and the second one is the `lat`
|
||||||
|
|
||||||
|
By default, the distance unit is `km` but it can also accept: `mi` (miles), `in` (inch), `yd` (yards), `m` (meters), `cm` (centimeters), `mm` (millimeters).
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"rings" : {
|
||||||
|
"geo_distance" : {
|
||||||
|
"field" : "location",
|
||||||
|
"origin" : "52.3760, 4.894",
|
||||||
|
"unit" : "mi", <1>
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 100 },
|
||||||
|
{ "from" : 100, "to" : 300 },
|
||||||
|
{ "from" : 300 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The distances will be computed as miles
|
||||||
|
|
||||||
|
There are two distance calculation modes: `arc` (the default) and `plane`. The `arc` calculation is the most accurate one but also the more expensive one in terms of performance. The `plane` is faster but less accurate. Consider using `plane` when your search context is "narrow" and spans smaller geographical areas (like cities or even countries). `plane` may return higher error mergins for searches across very large areas (e.g. cross continent search). The distance calculation type can be set using the `distance_type` parameter:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"rings" : {
|
||||||
|
"geo_distance" : {
|
||||||
|
"field" : "location",
|
||||||
|
"origin" : "52.3760, 4.894",
|
||||||
|
"distance_type" : "plane",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 100 },
|
||||||
|
{ "from" : 100, "to" : 300 },
|
||||||
|
{ "from" : 300 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,51 @@
|
||||||
|
[[search-aggregations-bucket-global-aggregation]]
|
||||||
|
=== Global
|
||||||
|
|
||||||
|
Defines a single bucket of all the documents within the search execution context. This context is defined by the indices and the document types you're searching on, but is *not* influenced by the search query itself.
|
||||||
|
|
||||||
|
NOTE: Global aggregators can only be placed as top level aggregators (it makes no sense to embed a global aggregator
|
||||||
|
within another bucket aggregator)
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"query" : {
|
||||||
|
"match" : { "title" : "shirt" }
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"all_products" : {
|
||||||
|
"global" : {}, <1>
|
||||||
|
"aggs" : { <2>
|
||||||
|
"avg_price" : { "avg" : { "field" : "price" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The `global` aggregation has an empty body
|
||||||
|
<2> The sub-aggregations that are registered for this `global` aggregation
|
||||||
|
|
||||||
|
The above aggregation demonstrates how one would compute aggregations (`avg_price` in this example) on all the documents in the search context, regardless of the query (in our example, it will compute the the average price over all products in our catalog, not just on the "shirts").
|
||||||
|
|
||||||
|
The response for the above aggreation:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations" : {
|
||||||
|
"all_products" : {
|
||||||
|
"doc_count" : 100, <1>
|
||||||
|
"avg_price" : {
|
||||||
|
"value" : 56.3
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The number of documents that were aggregated (in our case, all documents within the search context)
|
|
@ -0,0 +1,203 @@
|
||||||
|
[[search-aggregations-bucket-histogram-aggregation]]
|
||||||
|
=== Histogram
|
||||||
|
|
||||||
|
A multi-bucket values source based aggregation that can be applied on numeric values extracted from the documents. It dynamically builds fixed size (a.k.a. interval) buckets over the values. For example, if the documents have a field that holds a price (numeric), we can configure this aggregation to dynamically build buckets with interval `5` (in case of price it may represent $5). When the aggregation executes, the price field of every document will be evaluated and will be rounded down to its closes bucket - for example, if the price is `32` and the bucket size is `5` then the rounding will yield `30` and thus the document will "fall" into the bucket that is associated withe the key `30`. To make this more formal, here is the rounding function that is used:
|
||||||
|
|
||||||
|
[source,java]
|
||||||
|
--------------------------------------------------
|
||||||
|
rem = value % interval
|
||||||
|
if (rem < 0) {
|
||||||
|
rem += interval
|
||||||
|
}
|
||||||
|
bucket_key = value - rem
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The following snippet "buckets" the products based on their `price` by interval of `50`:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
And the following may be the response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"prices": [
|
||||||
|
{
|
||||||
|
"key": 0,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": 50,
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": 150,
|
||||||
|
"doc_count": 3
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The response above shows that non of the aggregated products has a price that falls within the range of `[100 - 150)`. By default, the response will only contain the non-empty buckets, though it is possible to also return those, by setting the `empty_buckets` flag to `true`:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50,
|
||||||
|
"empty_buckets" : true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"prices": [
|
||||||
|
{
|
||||||
|
"key": 0,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": 50,
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key" : 100,
|
||||||
|
"doc_count" : 0
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": 150,
|
||||||
|
"doc_count": 3
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Order
|
||||||
|
|
||||||
|
By default the returned buckets are sorted by their `key` ascending, though the order behaviour can be controled using the `order` setting.
|
||||||
|
|
||||||
|
Ordering the buckets by their key - descending:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50,
|
||||||
|
"order" : { "_key" : "desc" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Ordering the buckets by their `doc_count` - ascending:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50,
|
||||||
|
"order" : { "_count" : "asc" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
If the histogram aggregation has a direct metrics sub-aggregation, the latter can determine the order of the buckets:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50,
|
||||||
|
"order" : { "price_stats.min" : "asc" } <1>
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"price_stats" : { "stats" : {} } <2>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The `{ "price_stats.min" : asc" }` will sort the buckets based on `min` value of their their `price_stats` sub-aggregation.
|
||||||
|
|
||||||
|
<2> There is no need to configure the `price` field for the `price_stats` aggregation as it will inherit it by default from its parent histogram aggregation.
|
||||||
|
|
||||||
|
==== Response Format
|
||||||
|
|
||||||
|
By default, the buckets are retuned as an ordered array. It is also possilbe to request the response as a hash instead keyed by the buckets keys:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"prices" : {
|
||||||
|
"histogram" : {
|
||||||
|
"field" : "price",
|
||||||
|
"interval" : 50,
|
||||||
|
"keyed" : true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"prices": {
|
||||||
|
"0": {
|
||||||
|
"key": 0,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
"50": {
|
||||||
|
"key": 50,
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
"150": {
|
||||||
|
"key": 150,
|
||||||
|
"doc_count": 3
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,94 @@
|
||||||
|
[[search-aggregations-bucket-iprange-aggregation]]
|
||||||
|
=== IPv4 Range
|
||||||
|
|
||||||
|
Just like the dedicated <<search-aggregations-bucket-daterange-aggregation,date>> range aggregation, there is also a dedicated range aggregation for IPv4 typed fields:
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"ip_ranges" : {
|
||||||
|
"ip_range" : {
|
||||||
|
"field" : "ip",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : "10.0.0.5" },
|
||||||
|
{ "from" : "10.0.0.5" }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"ip_ranges": [
|
||||||
|
{
|
||||||
|
"to": 167772165,
|
||||||
|
"to_as_string": "10.0.0.5",
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 167772165,
|
||||||
|
"from_as_string": "10.0.0.5",
|
||||||
|
"doc_count": 6
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
IP ranges can also be defined as CIDR masks:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"ip_ranges" : {
|
||||||
|
"ip_range" : {
|
||||||
|
"field" : "ip",
|
||||||
|
"ranges" : [
|
||||||
|
{ "mask" : "10.0.0.0/25" },
|
||||||
|
{ "mask" : "10.0.0.127/25" }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"ip_ranges": [
|
||||||
|
{
|
||||||
|
"key": "10.0.0.0/25",
|
||||||
|
"from": 1.6777216E+8,
|
||||||
|
"from_as_string": "10.0.0.0",
|
||||||
|
"to": 167772287,
|
||||||
|
"to_as_string": "10.0.0.127",
|
||||||
|
"doc_count": 127
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key": "10.0.0.127/25",
|
||||||
|
"from": 1.6777216E+8,
|
||||||
|
"from_as_string": "10.0.0.0",
|
||||||
|
"to": 167772287,
|
||||||
|
"to_as_string": "10.0.0.127",
|
||||||
|
"doc_count": 127
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,34 @@
|
||||||
|
[[search-aggregations-bucket-missing-aggregation]]
|
||||||
|
=== Missing
|
||||||
|
|
||||||
|
A field data based single bucket aggregation, that creates a bucket of all documents in the current document set context that are missing a field value (effectively, missing a field). This aggregator will often be used in conjunction with other field data bucket aggregators (such as ranges) to return information for all the documents that could not be placed in any of the other buckets due to missing field data values.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"products_without_a_price" : {
|
||||||
|
"missing" : { "field" : "price" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
In the above example, we calculate the average price of all the products that are currently in-stock.
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"products_without_a_price" : {
|
||||||
|
"doc_count" : 10
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,63 @@
|
||||||
|
[[search-aggregations-bucket-nested-aggregation]]
|
||||||
|
=== Nested
|
||||||
|
|
||||||
|
A special single bucket aggregation that enables aggregating nested documents.
|
||||||
|
|
||||||
|
For example, lets say we have a index of products, and each product holds the list of resellers - each having its own price for the product. The mapping could look like:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"product" : {
|
||||||
|
"properties" : {
|
||||||
|
"resellers" : { <1>
|
||||||
|
"type" : "nested"
|
||||||
|
"properties" : {
|
||||||
|
"name" : { "type" : "string" },
|
||||||
|
"price" : { "type" : "double" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> The `resellers` is an array that holds nested documents under the `product` object.
|
||||||
|
|
||||||
|
The following aggregations will return the minimum price products can be purchased in:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"query" : {
|
||||||
|
"match" : { "name" : "led tv" }
|
||||||
|
}
|
||||||
|
"aggs" : {
|
||||||
|
"resellers" : {
|
||||||
|
"nested" : {
|
||||||
|
"path" : "resellers"
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"min_price" : { "min" : { "field" : "nested.value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
As you can see above, the nested aggregation requires the `path` of the nested documents within the top level documents. Then one can define any type of aggregation over these nested documents.
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"resellers": {
|
||||||
|
"min_price": 350
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,268 @@
|
||||||
|
[[search-aggregations-bucket-range-aggregation]]
|
||||||
|
=== Range
|
||||||
|
|
||||||
|
A multi-bucket value source based aggregation that enables the user to define a set of ranges - each representing a bucket. During the aggregation process, the values extracted from each document will be checked against each bucket range and "bucket" the relevant/matching document.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 50 },
|
||||||
|
{ "from" : 50, "to" : 100 },
|
||||||
|
{ "from" : 100 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"price_ranges": [
|
||||||
|
{
|
||||||
|
"to": 50,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 50,
|
||||||
|
"to": 100,
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 100,
|
||||||
|
"doc_count": 4
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Keyed Response
|
||||||
|
|
||||||
|
Setting the `key` flag to `true` will associate a unique string key with each bucket and return the ranges as a hash rather than an array:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"keyed" : true,
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 50 },
|
||||||
|
{ "from" : 50, "to" : 100 },
|
||||||
|
{ "from" : 100 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"price_ranges": {
|
||||||
|
"*-50.0": {
|
||||||
|
"to": 50,
|
||||||
|
"doc_count": 2
|
||||||
|
},
|
||||||
|
"50.0-100.0": {
|
||||||
|
"from": 50,
|
||||||
|
"to": 100,
|
||||||
|
"doc_count": 4
|
||||||
|
},
|
||||||
|
"100.0-*": {
|
||||||
|
"from": 100,
|
||||||
|
"doc_count": 4
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
It is also possible to customize the key for each range:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"keyed" : true,
|
||||||
|
"ranges" : [
|
||||||
|
{ "key" : "cheap", "to" : 50 },
|
||||||
|
{ "key" : "average", "from" : 50, "to" : 100 },
|
||||||
|
{ "key" : "expensive", "from" : 100 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"script" : "doc['price'].value",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 50 },
|
||||||
|
{ "from" : 50, "to" : 100 },
|
||||||
|
{ "from" : 100 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Value Script
|
||||||
|
|
||||||
|
Lets say the product prices are in USD but we would like to get the price ranges in EURO. We can use value script to convert the prices prior the aggregation (assuming conversion rate of 0.8)
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"script" : "_value * conversion_rate",
|
||||||
|
"params" : {
|
||||||
|
"conversion_rate" : 0.8
|
||||||
|
},
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 35 },
|
||||||
|
{ "from" : 35, "to" : 70 },
|
||||||
|
{ "from" : 70 }
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Sub Aggregations
|
||||||
|
|
||||||
|
The following example, not only "bucket" the documents to the different buckets but also computes statistics over the prices in each price range
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 50 },
|
||||||
|
{ "from" : 50, "to" : 100 },
|
||||||
|
{ "from" : 100 }
|
||||||
|
]
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"price_stats" : {
|
||||||
|
"stats" : { "field" : "price" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggregations": {
|
||||||
|
"price_ranges": [
|
||||||
|
{
|
||||||
|
"to": 50,
|
||||||
|
"doc_count": 2,
|
||||||
|
"price_stats": {
|
||||||
|
"count": 2,
|
||||||
|
"min": 20,
|
||||||
|
"max": 47,
|
||||||
|
"avg": 33.5,
|
||||||
|
"sum": 67
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 50,
|
||||||
|
"to": 100,
|
||||||
|
"doc_count": 4,
|
||||||
|
"price_stats": {
|
||||||
|
"count": 4,
|
||||||
|
"min": 60,
|
||||||
|
"max": 98,
|
||||||
|
"avg": 82.5,
|
||||||
|
"sum": 330
|
||||||
|
}
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"from": 100,
|
||||||
|
"doc_count": 4,
|
||||||
|
"price_stats": {
|
||||||
|
"count": 4,
|
||||||
|
"min": 134,
|
||||||
|
"max": 367,
|
||||||
|
"avg": 216,
|
||||||
|
"sum": 864
|
||||||
|
}
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
If a sub aggregation is also based on the same value source as the range aggregation (like the `stats` aggregation in the example above) it is possible to leave out the value source definition for it. The following will return the same response as above:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"price_ranges" : {
|
||||||
|
"range" : {
|
||||||
|
"field" : "price",
|
||||||
|
"ranges" : [
|
||||||
|
{ "to" : 50 },
|
||||||
|
{ "from" : 50, "to" : 100 },
|
||||||
|
{ "from" : 100 }
|
||||||
|
]
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"price_stats" : {
|
||||||
|
"stats" : {} <1>
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
<1> We don't need to specify the `price` as we "inherit" it by default from the parent `range` aggregation
|
|
@ -0,0 +1,156 @@
|
||||||
|
[[search-aggregations-bucket-terms-aggregation]]
|
||||||
|
=== Terms
|
||||||
|
|
||||||
|
A multi-bucket value source based aggregation where buckets are dynamically built - one per unique value.
|
||||||
|
|
||||||
|
Example:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : { "field" : "gender" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations" : {
|
||||||
|
"genders" : {
|
||||||
|
"buckets" : [
|
||||||
|
{
|
||||||
|
"key" : "male",
|
||||||
|
"doc_count" : 10
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"key" : "female",
|
||||||
|
"doc_count" : 10
|
||||||
|
},
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
By default, the `terms` aggregation will return the buckets for the top ten terms ordered by the `doc_count`. One can change this default behaviour by setting the `size` parameter.
|
||||||
|
|
||||||
|
==== Size
|
||||||
|
|
||||||
|
The `size` parameter can be set to define how many term buckets should be returned out of the overall terms list. By default, the node coordinating the search process will request each shard to provide its own top `size` term buckets and once all shards respond, it will reduces the results to the final list that will then be returned to the client. This means that if the number of unique terms is greater than `size`, the returned list is slightly off and not accurate (it could be that the term counts are slightly off and it could even be that a term that should have been in the top size buckets was not returned). The higher the `size` is, the more accurate the response at the cost of aggregation performance.
|
||||||
|
|
||||||
|
==== Order
|
||||||
|
|
||||||
|
The order of the buckets can be customized by setting the `order` parameter. By default, the buckets are ordered by their `doc_count` descending. It is also possible to change this behaviour as follows:
|
||||||
|
|
||||||
|
Ordering the buckets by their `doc_count` in an ascending manner:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"field" : "gender",
|
||||||
|
"order" : { "_count" : "asc" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Ordering the buckets alphabetically by their terms in an ascending manner:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"field" : "gender",
|
||||||
|
"order" : { "_term" : "asc" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
Ordering the buckets by single value metrics sub-aggregation (identified by the aggregation name):
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"field" : "gender",
|
||||||
|
"order" : { "avg_height" : "desc" }
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"avg_height" : { "avg" : { "field" : "height" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Ordering the buckets by multi value metrics sub-aggregation (identified by the aggregation name):
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"field" : "gender",
|
||||||
|
"order" : { "stats.avg" : "desc" }
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"height_stats" : { "stats" : { "field" : "height" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Generating the terms using a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"script" : "doc['gender'].value"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
==== Value Script
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"genders" : {
|
||||||
|
"terms" : {
|
||||||
|
"field" : "gender",
|
||||||
|
"script" : "doc['gender'].value"
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
|
@ -0,0 +1,16 @@
|
||||||
|
[[search-aggregations-metrics]]
|
||||||
|
=== Metrics Aggregations
|
||||||
|
|
||||||
|
The aggregations in this family compute metrics based on values extracted in one way or another from the documents that are being aggregated. The values are typically extracted from the fields of the document (using the field data), but can also be generated using scripts. Some aggregations output a single metric (e.g. `avg`) and are called `single-value metrics aggregation`, others generate multiple metrics (e.g. `stats`) and are called `multi-value metrics aggregation`. The distinction between single-value and multi-value metrics aggregations plays a role when these aggregations serve as direct sub-aggregations of some bucket aggregations (some bucket aggregation enable you to sort the returned buckets based on the metrics in each bucket).
|
||||||
|
|
||||||
|
include::metrics/min-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/max-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/sum-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/avg-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/stats-aggregation.asciidoc[]
|
||||||
|
|
||||||
|
include::metrics/extendedstats-aggregation.asciidoc[]
|
|
@ -0,0 +1,73 @@
|
||||||
|
[[search-aggregations-metrics-avg-aggregation]]
|
||||||
|
=== Avg
|
||||||
|
|
||||||
|
A `single-value` metrics aggregation that computes the average of numeric values that are extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
Assuming the data consists of documents representing exams grades (between 0 and 100) of students
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"avg_grade" : { "avg" : { "field" : "grade" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The above aggregation computes the average grade over all documents. The aggregation type is `avg` and the `field` setting defines the numeric field of the documents the average will be computed on. The above will return the following:
|
||||||
|
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"avg_grade": {
|
||||||
|
"value": 75
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The name of the aggregation (`avg_grade` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the average grade based on a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...,
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"avg_grade" : { "avg" : { "script" : "doc['grade'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
===== Value Script
|
||||||
|
|
||||||
|
It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use value script to get the new average
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"avg_corrected_grade" : {
|
||||||
|
"avg" : {
|
||||||
|
"field" : "grade",
|
||||||
|
"script" : "_value * correction",
|
||||||
|
"params" : {
|
||||||
|
"correction" : 1.2
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,82 @@
|
||||||
|
[[search-aggregations-metrics-extendedstats-aggregation]]
|
||||||
|
=== Extended Stats
|
||||||
|
|
||||||
|
A `multi-value` metrics aggregation that computes stats over numeric values extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
The `exteded_stats` aggregations is an extended version of the `<<search-aggregations-metrics-stats-aggregation,stats>>` aggregation, where additional metrics are added such as `sum_of_squares`, `variance` and `std_deviation`.
|
||||||
|
|
||||||
|
Assuming the data consists of documents representing exams grades (between 0 and 100) of students
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : { "extended_stats" : { "field" : "grade" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The above aggregation computes the grades statistics over all documents. The aggregation type is `extended_stats` and the `field` setting defines the numeric field of the documents the stats will be computed on. The above will return the following:
|
||||||
|
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"grades_stats": {
|
||||||
|
"count": 6,
|
||||||
|
"min": 72,
|
||||||
|
"max": 117.6,
|
||||||
|
"avg": 94.2,
|
||||||
|
"sum": 565.2,
|
||||||
|
"sum_of_squares": 54551.51999999999,
|
||||||
|
"variance": 218.2799999999976,
|
||||||
|
"std_deviation": 14.774302013969987
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The name of the aggregation (`grades_stats` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the grades stats based on a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...,
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : { "extended_stats" : { "script" : "doc['grade'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
===== Value Script
|
||||||
|
|
||||||
|
It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use value script to get the new stats
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : {
|
||||||
|
"extended_stats" : {
|
||||||
|
"field" : "grade",
|
||||||
|
"script" : "_value * correction",
|
||||||
|
"params" : {
|
||||||
|
"correction" : 1.2
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,68 @@
|
||||||
|
[[search-aggregations-metrics-max-aggregation]]
|
||||||
|
=== Max
|
||||||
|
|
||||||
|
A `single-value` metrics aggregation that keeps track and returns the maximum value among the numeric values extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
Computing the max price value across all documents
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"max_price" : { "max" : { "field" : "price" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"max_price": {
|
||||||
|
"value": 35
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
As can be seen, the name of the aggregation (`max_price` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the max price value across all document, this time using a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"max_price" : { "max" : { "script" : "doc['price'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
==== Value Script
|
||||||
|
|
||||||
|
Let's say that the prices of the documents in our index are in USD, but we would like to compute the max in EURO (and for the sake of this example, lets say the conversion rate is 1.2). We can use a value script to apply the conversion rate to every value before it's aggregated:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"max_price_in_euros" : {
|
||||||
|
"max" : {
|
||||||
|
"field" : "price",
|
||||||
|
"script" : "_value * conversion_rate",
|
||||||
|
"params" : {
|
||||||
|
"conversion_rate" : 1.2
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
|
@ -0,0 +1,67 @@
|
||||||
|
[[search-aggregations-metrics-min-aggregation]]
|
||||||
|
=== Min
|
||||||
|
|
||||||
|
A `single-value` metrics aggregation that keeps track and returns the minimum value among numeric values extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
Computing the min price value across all documents
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"min_price" : { "min" : { "field" : "price" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
Response:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"min_price": {
|
||||||
|
"value": 10
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
As can be seen, the name of the aggregation (`min_price` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the min price value across all document, this time using a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"min_price" : { "min" : { "script" : "doc['price'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
|
||||||
|
==== Value Script
|
||||||
|
|
||||||
|
Let's say that the prices of the documents in our index are in USD, but we would like to compute the min in EURO (and for the sake of this example, lets say the conversion rate is 1.2). We can use a value script to apply the conversion rate to every value before it's aggregated:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"min_price_in_euros" : {
|
||||||
|
"min" : {
|
||||||
|
"field" : "price",
|
||||||
|
"script" : "_value * conversion_rate",
|
||||||
|
"params" : {
|
||||||
|
"conversion_rate" : 1.2
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,79 @@
|
||||||
|
[[search-aggregations-metrics-stats-aggregation]]
|
||||||
|
=== Stats
|
||||||
|
|
||||||
|
A `multi-value` metrics aggregation that computes stats over numeric values extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
The stats that are returned consist of: `min`, `max`, `sum`, `count` and `avg`.
|
||||||
|
|
||||||
|
Assuming the data consists of documents representing exams grades (between 0 and 100) of students
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : { "stats" : { "field" : "grade" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The above aggregation computes the grades statistics over all documents. The aggregation type is `stats` and the `field` setting defines the numeric field of the documents the stats will be computed on. The above will return the following:
|
||||||
|
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"grades_stats": {
|
||||||
|
"count": 6,
|
||||||
|
"min": 60,
|
||||||
|
"max": 98,
|
||||||
|
"avg": 78.5,
|
||||||
|
"sum": 471
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The name of the aggregation (`grades_stats` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the grades stats based on a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...,
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : { "stats" : { "script" : "doc['grade'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
===== Value Script
|
||||||
|
|
||||||
|
It turned out that the exam was way above the level of the students and a grade correction needs to be applied. We can use value script to get the new stats
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"grades_stats" : {
|
||||||
|
"stats" : {
|
||||||
|
"field" : "grade",
|
||||||
|
"script" : "_value * correction",
|
||||||
|
"params" : {
|
||||||
|
"correction" : 1.2
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -0,0 +1,77 @@
|
||||||
|
[[search-aggregations-metrics-sum-aggregation]]
|
||||||
|
=== Sum
|
||||||
|
|
||||||
|
A `single-value` metrics aggregation that sums up numeric values that are extracted from the aggregated documents. These values can be extracted either from specific numeric fields in the documents, or be generated by a provided script.
|
||||||
|
|
||||||
|
Assuming the data consists of documents representing stock ticks, where each tick holds the change in the stock price from the previous tick.
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"query" : {
|
||||||
|
"filtered" : {
|
||||||
|
"query" : { "match_all" : {}},
|
||||||
|
"filter" : {
|
||||||
|
"range" : { "timestamp" : { "from" : "now/1d+9.5h", "to" : "now/1d+16h" }}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
},
|
||||||
|
"aggs" : {
|
||||||
|
"intraday_return" : { "sum" : { "field" : "change" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The above aggregation sums up all changes in the today's trading stock ticks which accounts for the intraday return. The aggregation type is `sum` and the `field` setting defines the numeric field of the documents of which values will be summed up. The above will return the following:
|
||||||
|
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggregations": {
|
||||||
|
"intraday_return": {
|
||||||
|
"value": 2.18
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
The name of the aggregation (`intraday_return` above) also serves as the key by which the aggreagtion result can be retrieved from the returned response.
|
||||||
|
|
||||||
|
==== Script
|
||||||
|
|
||||||
|
Computing the intraday return based on a script:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
...,
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"intraday_return" : { "sum" : { "script" : "doc['change'].value" } }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
||||||
|
|
||||||
|
===== Value Script
|
||||||
|
|
||||||
|
Computing the sum of squares over all stock tick changes:
|
||||||
|
|
||||||
|
[source,js]
|
||||||
|
--------------------------------------------------
|
||||||
|
{
|
||||||
|
"aggs" : {
|
||||||
|
...
|
||||||
|
|
||||||
|
"aggs" : {
|
||||||
|
"daytime_return" : {
|
||||||
|
"sum" : {
|
||||||
|
"field" : "change",
|
||||||
|
"script" : "_value * _value" }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
--------------------------------------------------
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.index.query.FilterBuilder;
|
import org.elasticsearch.index.query.FilterBuilder;
|
||||||
import org.elasticsearch.index.query.QueryBuilder;
|
import org.elasticsearch.index.query.QueryBuilder;
|
||||||
import org.elasticsearch.search.Scroll;
|
import org.elasticsearch.search.Scroll;
|
||||||
|
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
|
||||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||||
import org.elasticsearch.search.facet.FacetBuilder;
|
import org.elasticsearch.search.facet.FacetBuilder;
|
||||||
import org.elasticsearch.search.highlight.HighlightBuilder;
|
import org.elasticsearch.search.highlight.HighlightBuilder;
|
||||||
|
@ -566,6 +567,54 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds an get to the search operation.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder addAggregation(AbstractAggregationBuilder aggregation) {
|
||||||
|
sourceBuilder().aggregation(aggregation);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent) binary representation of addAggregation to use.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder setAggregations(BytesReference aggregations) {
|
||||||
|
sourceBuilder().aggregations(aggregations);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent) binary representation of addAggregation to use.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder setAggregations(byte[] aggregations) {
|
||||||
|
sourceBuilder().aggregations(aggregations);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent) binary representation of addAggregation to use.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder setAggregations(byte[] aggregations, int aggregationsOffset, int aggregationsLength) {
|
||||||
|
sourceBuilder().facets(aggregations, aggregationsOffset, aggregationsLength);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent) binary representation of addAggregation to use.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder setAggregations(XContentBuilder aggregations) {
|
||||||
|
sourceBuilder().aggregations(aggregations);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent) binary representation of addAggregation to use.
|
||||||
|
*/
|
||||||
|
public SearchRequestBuilder setAggregations(Map aggregations) {
|
||||||
|
sourceBuilder().aggregations(aggregations);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds a field to be highlighted with default fragment size of 100 characters, and
|
* Adds a field to be highlighted with default fragment size of 100 characters, and
|
||||||
* default number of fragments of 5.
|
* default number of fragments of 5.
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.search.SearchHits;
|
import org.elasticsearch.search.SearchHits;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregations;
|
||||||
import org.elasticsearch.search.facet.Facets;
|
import org.elasticsearch.search.facet.Facets;
|
||||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||||
import org.elasticsearch.search.suggest.Suggest;
|
import org.elasticsearch.search.suggest.Suggest;
|
||||||
|
@ -102,6 +103,11 @@ public class SearchResponse extends ActionResponse implements ToXContent {
|
||||||
return internalResponse.facets();
|
return internalResponse.facets();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Aggregations getAggregations() {
|
||||||
|
return internalResponse.aggregations();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public Suggest getSuggest() {
|
public Suggest getSuggest() {
|
||||||
return internalResponse.suggest();
|
return internalResponse.suggest();
|
||||||
}
|
}
|
||||||
|
|
|
@ -122,7 +122,7 @@ public class TransportSearchScrollScanAction extends AbstractComponent {
|
||||||
|
|
||||||
public void start() {
|
public void start() {
|
||||||
if (scrollId.getContext().length == 0) {
|
if (scrollId.getContext().length == 0) {
|
||||||
final InternalSearchResponse internalResponse = new InternalSearchResponse(new InternalSearchHits(InternalSearchHits.EMPTY, Long.parseLong(this.scrollId.getAttributes().get("total_hits")), 0.0f), null, null, false);
|
final InternalSearchResponse internalResponse = new InternalSearchResponse(new InternalSearchHits(InternalSearchHits.EMPTY, Long.parseLong(this.scrollId.getAttributes().get("total_hits")), 0.0f), null, null, null, false);
|
||||||
listener.onResponse(new SearchResponse(internalResponse, request.scrollId(), 0, 0, 0l, buildShardFailures()));
|
listener.onResponse(new SearchResponse(internalResponse, request.scrollId(), 0, 0, 0l, buildShardFailures()));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,8 @@
|
||||||
package org.elasticsearch.common.compress;
|
package org.elasticsearch.common.compress;
|
||||||
|
|
||||||
import org.apache.lucene.store.IndexInput;
|
import org.apache.lucene.store.IndexInput;
|
||||||
import org.elasticsearch.common.util.BigLongArray;
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.LongArray;
|
||||||
|
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -36,7 +37,7 @@ public abstract class CompressedIndexInput<T extends CompressorContext> extends
|
||||||
|
|
||||||
private int version;
|
private int version;
|
||||||
private long totalUncompressedLength;
|
private long totalUncompressedLength;
|
||||||
private BigLongArray offsets;
|
private LongArray offsets;
|
||||||
|
|
||||||
private boolean closed;
|
private boolean closed;
|
||||||
|
|
||||||
|
@ -58,7 +59,7 @@ public abstract class CompressedIndexInput<T extends CompressorContext> extends
|
||||||
in.seek(metaDataPosition);
|
in.seek(metaDataPosition);
|
||||||
this.totalUncompressedLength = in.readVLong();
|
this.totalUncompressedLength = in.readVLong();
|
||||||
int size = in.readVInt();
|
int size = in.readVInt();
|
||||||
offsets = new BigLongArray(size);
|
offsets = BigArrays.newLongArray(size);
|
||||||
for (int i = 0; i < size; i++) {
|
for (int i = 0; i < size; i++) {
|
||||||
offsets.set(i, in.readVLong());
|
offsets.set(i, in.readVLong());
|
||||||
}
|
}
|
||||||
|
@ -139,7 +140,7 @@ public abstract class CompressedIndexInput<T extends CompressorContext> extends
|
||||||
@Override
|
@Override
|
||||||
public void seek(long pos) throws IOException {
|
public void seek(long pos) throws IOException {
|
||||||
int idx = (int) (pos / uncompressedLength);
|
int idx = (int) (pos / uncompressedLength);
|
||||||
if (idx >= offsets.size) {
|
if (idx >= offsets.size()) {
|
||||||
// set the next "readyBuffer" to EOF
|
// set the next "readyBuffer" to EOF
|
||||||
currentOffsetIdx = idx;
|
currentOffsetIdx = idx;
|
||||||
position = 0;
|
position = 0;
|
||||||
|
@ -184,7 +185,7 @@ public abstract class CompressedIndexInput<T extends CompressorContext> extends
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
// we reached the end...
|
// we reached the end...
|
||||||
if (currentOffsetIdx + 1 >= offsets.size) {
|
if (currentOffsetIdx + 1 >= offsets.size()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
valid = uncompress(in, uncompressed);
|
valid = uncompress(in, uncompressed);
|
||||||
|
|
|
@ -135,6 +135,12 @@ public class GeoPoint {
|
||||||
return "[" + lat + ", " + lon + "]";
|
return "[" + lat + ", " + lon + "]";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static GeoPoint parseFromLatLon(String latLon) {
|
||||||
|
GeoPoint point = new GeoPoint();
|
||||||
|
point.resetFromString(latLon);
|
||||||
|
return point;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parse a {@link GeoPoint} with a {@link XContentParser}:
|
* Parse a {@link GeoPoint} with a {@link XContentParser}:
|
||||||
*
|
*
|
||||||
|
|
|
@ -1,282 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to ElasticSearch and Shay Banon 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.common.joda;
|
|
||||||
|
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
|
||||||
import org.joda.time.DateTimeConstants;
|
|
||||||
import org.joda.time.DateTimeField;
|
|
||||||
import org.joda.time.DateTimeZone;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public abstract class TimeZoneRounding {
|
|
||||||
|
|
||||||
public abstract long calc(long utcMillis);
|
|
||||||
|
|
||||||
public static Builder builder(DateTimeField field) {
|
|
||||||
return new Builder(field);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Builder builder(TimeValue interval) {
|
|
||||||
return new Builder(interval);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class Builder {
|
|
||||||
|
|
||||||
private DateTimeField field;
|
|
||||||
private long interval = -1;
|
|
||||||
|
|
||||||
private DateTimeZone preTz = DateTimeZone.UTC;
|
|
||||||
private DateTimeZone postTz = DateTimeZone.UTC;
|
|
||||||
|
|
||||||
private float factor = 1.0f;
|
|
||||||
|
|
||||||
private long preOffset;
|
|
||||||
private long postOffset;
|
|
||||||
|
|
||||||
private boolean preZoneAdjustLargeInterval = false;
|
|
||||||
|
|
||||||
public Builder(DateTimeField field) {
|
|
||||||
this.field = field;
|
|
||||||
this.interval = -1;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder(TimeValue interval) {
|
|
||||||
this.field = null;
|
|
||||||
this.interval = interval.millis();
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder preZone(DateTimeZone preTz) {
|
|
||||||
this.preTz = preTz;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder preZoneAdjustLargeInterval(boolean preZoneAdjustLargeInterval) {
|
|
||||||
this.preZoneAdjustLargeInterval = preZoneAdjustLargeInterval;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder postZone(DateTimeZone postTz) {
|
|
||||||
this.postTz = postTz;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder preOffset(long preOffset) {
|
|
||||||
this.preOffset = preOffset;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder postOffset(long postOffset) {
|
|
||||||
this.postOffset = postOffset;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder factor(float factor) {
|
|
||||||
this.factor = factor;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public TimeZoneRounding build() {
|
|
||||||
TimeZoneRounding timeZoneRounding;
|
|
||||||
if (field != null) {
|
|
||||||
if (preTz.equals(DateTimeZone.UTC) && postTz.equals(DateTimeZone.UTC)) {
|
|
||||||
timeZoneRounding = new UTCTimeZoneRoundingFloor(field);
|
|
||||||
} else if (preZoneAdjustLargeInterval || field.getDurationField().getUnitMillis() < DateTimeConstants.MILLIS_PER_HOUR * 12) {
|
|
||||||
timeZoneRounding = new TimeTimeZoneRoundingFloor(field, preTz, postTz);
|
|
||||||
} else {
|
|
||||||
timeZoneRounding = new DayTimeZoneRoundingFloor(field, preTz, postTz);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if (preTz.equals(DateTimeZone.UTC) && postTz.equals(DateTimeZone.UTC)) {
|
|
||||||
timeZoneRounding = new UTCIntervalTimeZoneRounding(interval);
|
|
||||||
} else if (preZoneAdjustLargeInterval || interval < DateTimeConstants.MILLIS_PER_HOUR * 12) {
|
|
||||||
timeZoneRounding = new TimeIntervalTimeZoneRounding(interval, preTz, postTz);
|
|
||||||
} else {
|
|
||||||
timeZoneRounding = new DayIntervalTimeZoneRounding(interval, preTz, postTz);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (preOffset != 0 || postOffset != 0) {
|
|
||||||
timeZoneRounding = new PrePostTimeZoneRounding(timeZoneRounding, preOffset, postOffset);
|
|
||||||
}
|
|
||||||
if (factor != 1.0f) {
|
|
||||||
timeZoneRounding = new FactorTimeZoneRounding(timeZoneRounding, factor);
|
|
||||||
}
|
|
||||||
return timeZoneRounding;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class TimeTimeZoneRoundingFloor extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final DateTimeField field;
|
|
||||||
private final DateTimeZone preTz;
|
|
||||||
private final DateTimeZone postTz;
|
|
||||||
|
|
||||||
TimeTimeZoneRoundingFloor(DateTimeField field, DateTimeZone preTz, DateTimeZone postTz) {
|
|
||||||
this.field = field;
|
|
||||||
this.preTz = preTz;
|
|
||||||
this.postTz = postTz;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
long time = utcMillis + preTz.getOffset(utcMillis);
|
|
||||||
time = field.roundFloor(time);
|
|
||||||
// now, time is still in local, move it to UTC (or the adjustLargeInterval flag is set)
|
|
||||||
time = time - preTz.getOffset(time);
|
|
||||||
// now apply post Tz
|
|
||||||
time = time + postTz.getOffset(time);
|
|
||||||
return time;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class UTCTimeZoneRoundingFloor extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final DateTimeField field;
|
|
||||||
|
|
||||||
UTCTimeZoneRoundingFloor(DateTimeField field) {
|
|
||||||
this.field = field;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
return field.roundFloor(utcMillis);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class DayTimeZoneRoundingFloor extends TimeZoneRounding {
|
|
||||||
private final DateTimeField field;
|
|
||||||
private final DateTimeZone preTz;
|
|
||||||
private final DateTimeZone postTz;
|
|
||||||
|
|
||||||
DayTimeZoneRoundingFloor(DateTimeField field, DateTimeZone preTz, DateTimeZone postTz) {
|
|
||||||
this.field = field;
|
|
||||||
this.preTz = preTz;
|
|
||||||
this.postTz = postTz;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
long time = utcMillis + preTz.getOffset(utcMillis);
|
|
||||||
time = field.roundFloor(time);
|
|
||||||
// after rounding, since its day level (and above), its actually UTC!
|
|
||||||
// now apply post Tz
|
|
||||||
time = time + postTz.getOffset(time);
|
|
||||||
return time;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class UTCIntervalTimeZoneRounding extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final long interval;
|
|
||||||
|
|
||||||
UTCIntervalTimeZoneRounding(long interval) {
|
|
||||||
this.interval = interval;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
return ((utcMillis / interval) * interval);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
static class TimeIntervalTimeZoneRounding extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final long interval;
|
|
||||||
private final DateTimeZone preTz;
|
|
||||||
private final DateTimeZone postTz;
|
|
||||||
|
|
||||||
TimeIntervalTimeZoneRounding(long interval, DateTimeZone preTz, DateTimeZone postTz) {
|
|
||||||
this.interval = interval;
|
|
||||||
this.preTz = preTz;
|
|
||||||
this.postTz = postTz;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
long time = utcMillis + preTz.getOffset(utcMillis);
|
|
||||||
time = ((time / interval) * interval);
|
|
||||||
// now, time is still in local, move it to UTC
|
|
||||||
time = time - preTz.getOffset(time);
|
|
||||||
// now apply post Tz
|
|
||||||
time = time + postTz.getOffset(time);
|
|
||||||
return time;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class DayIntervalTimeZoneRounding extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final long interval;
|
|
||||||
private final DateTimeZone preTz;
|
|
||||||
private final DateTimeZone postTz;
|
|
||||||
|
|
||||||
DayIntervalTimeZoneRounding(long interval, DateTimeZone preTz, DateTimeZone postTz) {
|
|
||||||
this.interval = interval;
|
|
||||||
this.preTz = preTz;
|
|
||||||
this.postTz = postTz;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
long time = utcMillis + preTz.getOffset(utcMillis);
|
|
||||||
time = ((time / interval) * interval);
|
|
||||||
// after rounding, since its day level (and above), its actually UTC!
|
|
||||||
// now apply post Tz
|
|
||||||
time = time + postTz.getOffset(time);
|
|
||||||
return time;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class FactorTimeZoneRounding extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final TimeZoneRounding timeZoneRounding;
|
|
||||||
|
|
||||||
private final float factor;
|
|
||||||
|
|
||||||
FactorTimeZoneRounding(TimeZoneRounding timeZoneRounding, float factor) {
|
|
||||||
this.timeZoneRounding = timeZoneRounding;
|
|
||||||
this.factor = factor;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
return timeZoneRounding.calc((long) (factor * utcMillis));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class PrePostTimeZoneRounding extends TimeZoneRounding {
|
|
||||||
|
|
||||||
private final TimeZoneRounding timeZoneRounding;
|
|
||||||
|
|
||||||
private final long preOffset;
|
|
||||||
private final long postOffset;
|
|
||||||
|
|
||||||
PrePostTimeZoneRounding(TimeZoneRounding timeZoneRounding, long preOffset, long postOffset) {
|
|
||||||
this.timeZoneRounding = timeZoneRounding;
|
|
||||||
this.preOffset = preOffset;
|
|
||||||
this.postOffset = postOffset;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long calc(long utcMillis) {
|
|
||||||
return postOffset + timeZoneRounding.calc(utcMillis + preOffset);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -0,0 +1,30 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.lucene;
|
||||||
|
|
||||||
|
import org.apache.lucene.index.AtomicReaderContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface ReaderContextAware {
|
||||||
|
|
||||||
|
public void setNextReader(AtomicReaderContext reader);
|
||||||
|
}
|
|
@ -0,0 +1,31 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.lucene;
|
||||||
|
|
||||||
|
import org.apache.lucene.search.Scorer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface ScorerAware {
|
||||||
|
|
||||||
|
void setScorer(Scorer scorer);
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.rounding;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticSearchException;
|
||||||
|
import org.elasticsearch.common.joda.Joda;
|
||||||
|
import org.joda.time.DateTimeField;
|
||||||
|
import org.joda.time.chrono.ISOChronology;
|
||||||
|
|
||||||
|
import java.math.BigInteger;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public enum DateTimeUnit {
|
||||||
|
|
||||||
|
WEEK_OF_WEEKYEAR( (byte) 1, ISOChronology.getInstanceUTC().weekOfWeekyear()),
|
||||||
|
YEAR_OF_CENTURY( (byte) 2, ISOChronology.getInstanceUTC().yearOfCentury()),
|
||||||
|
QUARTER( (byte) 3, Joda.QuarterOfYear.getField(ISOChronology.getInstanceUTC())),
|
||||||
|
MONTH_OF_YEAR( (byte) 4, ISOChronology.getInstanceUTC().monthOfYear()),
|
||||||
|
DAY_OF_MONTH( (byte) 5, ISOChronology.getInstanceUTC().dayOfMonth()),
|
||||||
|
HOUR_OF_DAY( (byte) 6, ISOChronology.getInstanceUTC().hourOfDay()),
|
||||||
|
MINUTES_OF_HOUR( (byte) 7, ISOChronology.getInstanceUTC().minuteOfHour()),
|
||||||
|
SECOND_OF_MINUTE( (byte) 8, ISOChronology.getInstanceUTC().secondOfMinute());
|
||||||
|
|
||||||
|
private final byte id;
|
||||||
|
private final DateTimeField field;
|
||||||
|
|
||||||
|
private DateTimeUnit(byte id, DateTimeField field) {
|
||||||
|
this.id = id;
|
||||||
|
this.field = field;
|
||||||
|
}
|
||||||
|
|
||||||
|
public byte id() {
|
||||||
|
return id;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateTimeField field() {
|
||||||
|
return field;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DateTimeUnit resolve(byte id) {
|
||||||
|
switch (id) {
|
||||||
|
case 1: return WEEK_OF_WEEKYEAR;
|
||||||
|
case 2: return YEAR_OF_CENTURY;
|
||||||
|
case 3: return QUARTER;
|
||||||
|
case 4: return MONTH_OF_YEAR;
|
||||||
|
case 5: return DAY_OF_MONTH;
|
||||||
|
case 6: return HOUR_OF_DAY;
|
||||||
|
case 7: return MINUTES_OF_HOUR;
|
||||||
|
case 8: return SECOND_OF_MINUTE;
|
||||||
|
default: throw new ElasticSearchException("Unknown date time unit id [" + id + "]");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,143 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.rounding;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticSearchException;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Streamable;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A strategy for rounding long values.
|
||||||
|
*/
|
||||||
|
public interface Rounding extends Streamable {
|
||||||
|
|
||||||
|
byte id();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rounds the given value.
|
||||||
|
*
|
||||||
|
* @param value The value to round.
|
||||||
|
* @return The rounded value.
|
||||||
|
*/
|
||||||
|
long round(long value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given the rounded value (which was potentially generated by {@link #round(long)}, returns the next rounding value. For example, with
|
||||||
|
* interval based rounding, if the interval is 3, {@code nextRoundValue(6) = 9 }.
|
||||||
|
*
|
||||||
|
* @param value The current rounding value
|
||||||
|
* @return The next rounding value;
|
||||||
|
*/
|
||||||
|
long nextRoundingValue(long value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rounding strategy which is based on an interval
|
||||||
|
*
|
||||||
|
* {@code rounded = value - (value % interval) }
|
||||||
|
*/
|
||||||
|
public static class Interval implements Rounding {
|
||||||
|
|
||||||
|
final static byte ID = 0;
|
||||||
|
|
||||||
|
private long interval;
|
||||||
|
|
||||||
|
public Interval() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new interval rounding.
|
||||||
|
*
|
||||||
|
* @param interval The interval
|
||||||
|
*/
|
||||||
|
public Interval(long interval) {
|
||||||
|
this.interval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
static long round(long value, long interval) {
|
||||||
|
long rem = value % interval;
|
||||||
|
// We need this condition because % may return a negative result on negative numbers
|
||||||
|
// According to Google caliper's IntModBenchmark, using a condition is faster than attempts to use tricks to avoid
|
||||||
|
// the condition. Moreover, in our case, the condition is very likely to be always true (dates, prices, distances),
|
||||||
|
// so easily predictable by the CPU
|
||||||
|
if (rem < 0) {
|
||||||
|
rem += interval;
|
||||||
|
}
|
||||||
|
return value - rem;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long value) {
|
||||||
|
return round(value, interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
assert value == round(value);
|
||||||
|
return value + interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
interval = in.readVLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeVLong(interval);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Streams {
|
||||||
|
|
||||||
|
public static void write(Rounding rounding, StreamOutput out) throws IOException {
|
||||||
|
out.writeByte(rounding.id());
|
||||||
|
rounding.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Rounding read(StreamInput in) throws IOException {
|
||||||
|
Rounding rounding = null;
|
||||||
|
byte id = in.readByte();
|
||||||
|
switch (id) {
|
||||||
|
case Interval.ID: rounding = new Interval(); break;
|
||||||
|
case TimeZoneRounding.TimeTimeZoneRoundingFloor.ID: rounding = new TimeZoneRounding.TimeTimeZoneRoundingFloor(); break;
|
||||||
|
case TimeZoneRounding.UTCTimeZoneRoundingFloor.ID: rounding = new TimeZoneRounding.UTCTimeZoneRoundingFloor(); break;
|
||||||
|
case TimeZoneRounding.DayTimeZoneRoundingFloor.ID: rounding = new TimeZoneRounding.DayTimeZoneRoundingFloor(); break;
|
||||||
|
case TimeZoneRounding.UTCIntervalTimeZoneRounding.ID: rounding = new TimeZoneRounding.UTCIntervalTimeZoneRounding(); break;
|
||||||
|
case TimeZoneRounding.TimeIntervalTimeZoneRounding.ID: rounding = new TimeZoneRounding.TimeIntervalTimeZoneRounding(); break;
|
||||||
|
case TimeZoneRounding.DayIntervalTimeZoneRounding.ID: rounding = new TimeZoneRounding.DayIntervalTimeZoneRounding(); break;
|
||||||
|
case TimeZoneRounding.FactorTimeZoneRounding.ID: rounding = new TimeZoneRounding.FactorTimeZoneRounding(); break;
|
||||||
|
case TimeZoneRounding.PrePostTimeZoneRounding.ID: rounding = new TimeZoneRounding.PrePostTimeZoneRounding(); break;
|
||||||
|
default: throw new ElasticSearchException("unknown rounding id [" + id + "]");
|
||||||
|
}
|
||||||
|
rounding.readFrom(in);
|
||||||
|
return rounding;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,509 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.rounding;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.joda.time.DateTimeConstants;
|
||||||
|
import org.joda.time.DateTimeZone;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public abstract class TimeZoneRounding implements Rounding {
|
||||||
|
|
||||||
|
public abstract long round(long utcMillis);
|
||||||
|
|
||||||
|
public static Builder builder(DateTimeUnit unit) {
|
||||||
|
return new Builder(unit);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Builder builder(TimeValue interval) {
|
||||||
|
return new Builder(interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private DateTimeUnit unit;
|
||||||
|
private long interval = -1;
|
||||||
|
|
||||||
|
private DateTimeZone preTz = DateTimeZone.UTC;
|
||||||
|
private DateTimeZone postTz = DateTimeZone.UTC;
|
||||||
|
|
||||||
|
private float factor = 1.0f;
|
||||||
|
|
||||||
|
private long preOffset;
|
||||||
|
private long postOffset;
|
||||||
|
|
||||||
|
private boolean preZoneAdjustLargeInterval = false;
|
||||||
|
|
||||||
|
public Builder(DateTimeUnit unit) {
|
||||||
|
this.unit = unit;
|
||||||
|
this.interval = -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder(TimeValue interval) {
|
||||||
|
this.unit = null;
|
||||||
|
this.interval = interval.millis();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder preZone(DateTimeZone preTz) {
|
||||||
|
this.preTz = preTz;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder preZoneAdjustLargeInterval(boolean preZoneAdjustLargeInterval) {
|
||||||
|
this.preZoneAdjustLargeInterval = preZoneAdjustLargeInterval;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder postZone(DateTimeZone postTz) {
|
||||||
|
this.postTz = postTz;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder preOffset(long preOffset) {
|
||||||
|
this.preOffset = preOffset;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder postOffset(long postOffset) {
|
||||||
|
this.postOffset = postOffset;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder factor(float factor) {
|
||||||
|
this.factor = factor;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimeZoneRounding build() {
|
||||||
|
TimeZoneRounding timeZoneRounding;
|
||||||
|
if (unit != null) {
|
||||||
|
if (preTz.equals(DateTimeZone.UTC) && postTz.equals(DateTimeZone.UTC)) {
|
||||||
|
timeZoneRounding = new UTCTimeZoneRoundingFloor(unit);
|
||||||
|
} else if (preZoneAdjustLargeInterval || unit.field().getDurationField().getUnitMillis() < DateTimeConstants.MILLIS_PER_HOUR * 12) {
|
||||||
|
timeZoneRounding = new TimeTimeZoneRoundingFloor(unit, preTz, postTz);
|
||||||
|
} else {
|
||||||
|
timeZoneRounding = new DayTimeZoneRoundingFloor(unit, preTz, postTz);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
if (preTz.equals(DateTimeZone.UTC) && postTz.equals(DateTimeZone.UTC)) {
|
||||||
|
timeZoneRounding = new UTCIntervalTimeZoneRounding(interval);
|
||||||
|
} else if (preZoneAdjustLargeInterval || interval < DateTimeConstants.MILLIS_PER_HOUR * 12) {
|
||||||
|
timeZoneRounding = new TimeIntervalTimeZoneRounding(interval, preTz, postTz);
|
||||||
|
} else {
|
||||||
|
timeZoneRounding = new DayIntervalTimeZoneRounding(interval, preTz, postTz);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (preOffset != 0 || postOffset != 0) {
|
||||||
|
timeZoneRounding = new PrePostTimeZoneRounding(timeZoneRounding, preOffset, postOffset);
|
||||||
|
}
|
||||||
|
if (factor != 1.0f) {
|
||||||
|
timeZoneRounding = new FactorTimeZoneRounding(timeZoneRounding, factor);
|
||||||
|
}
|
||||||
|
return timeZoneRounding;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class TimeTimeZoneRoundingFloor extends TimeZoneRounding {
|
||||||
|
|
||||||
|
static final byte ID = 1;
|
||||||
|
|
||||||
|
private DateTimeUnit unit;
|
||||||
|
private DateTimeZone preTz;
|
||||||
|
private DateTimeZone postTz;
|
||||||
|
|
||||||
|
TimeTimeZoneRoundingFloor() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeTimeZoneRoundingFloor(DateTimeUnit unit, DateTimeZone preTz, DateTimeZone postTz) {
|
||||||
|
this.unit = unit;
|
||||||
|
this.preTz = preTz;
|
||||||
|
this.postTz = postTz;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
long time = utcMillis + preTz.getOffset(utcMillis);
|
||||||
|
time = unit.field().roundFloor(time);
|
||||||
|
// now, time is still in local, move it to UTC (or the adjustLargeInterval flag is set)
|
||||||
|
time = time - preTz.getOffset(time);
|
||||||
|
// now apply post Tz
|
||||||
|
time = time + postTz.getOffset(time);
|
||||||
|
return time;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
// return value + unit.field().getDurationField().getUnitMillis();
|
||||||
|
return unit.field().roundCeiling(value + 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
unit = DateTimeUnit.resolve(in.readByte());
|
||||||
|
preTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
postTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeByte(unit.id());
|
||||||
|
out.writeSharedString(preTz.getID());
|
||||||
|
out.writeSharedString(postTz.getID());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class UTCTimeZoneRoundingFloor extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 2;
|
||||||
|
|
||||||
|
private DateTimeUnit unit;
|
||||||
|
|
||||||
|
UTCTimeZoneRoundingFloor() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
UTCTimeZoneRoundingFloor(DateTimeUnit unit) {
|
||||||
|
this.unit = unit;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
return unit.field().roundFloor(utcMillis);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return unit.field().roundCeiling(value + 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
unit = DateTimeUnit.resolve(in.readByte());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeByte(unit.id());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class DayTimeZoneRoundingFloor extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 3;
|
||||||
|
|
||||||
|
private DateTimeUnit unit;
|
||||||
|
private DateTimeZone preTz;
|
||||||
|
private DateTimeZone postTz;
|
||||||
|
|
||||||
|
DayTimeZoneRoundingFloor() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
DayTimeZoneRoundingFloor(DateTimeUnit unit, DateTimeZone preTz, DateTimeZone postTz) {
|
||||||
|
this.unit = unit;
|
||||||
|
this.preTz = preTz;
|
||||||
|
this.postTz = postTz;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
long time = utcMillis + preTz.getOffset(utcMillis);
|
||||||
|
time = unit.field().roundFloor(time);
|
||||||
|
// after rounding, since its day level (and above), its actually UTC!
|
||||||
|
// now apply post Tz
|
||||||
|
time = time + postTz.getOffset(time);
|
||||||
|
return time;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return unit.field().getDurationField().getUnitMillis() + value;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
unit = DateTimeUnit.resolve(in.readByte());
|
||||||
|
preTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
postTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeByte(unit.id());
|
||||||
|
out.writeSharedString(preTz.getID());
|
||||||
|
out.writeSharedString(postTz.getID());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class UTCIntervalTimeZoneRounding extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 4;
|
||||||
|
|
||||||
|
private long interval;
|
||||||
|
|
||||||
|
UTCIntervalTimeZoneRounding() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
UTCIntervalTimeZoneRounding(long interval) {
|
||||||
|
this.interval = interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
return Rounding.Interval.round(utcMillis, interval);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return value + interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
interval = in.readVLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeVLong(interval);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
static class TimeIntervalTimeZoneRounding extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 5;
|
||||||
|
|
||||||
|
private long interval;
|
||||||
|
private DateTimeZone preTz;
|
||||||
|
private DateTimeZone postTz;
|
||||||
|
|
||||||
|
TimeIntervalTimeZoneRounding() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeIntervalTimeZoneRounding(long interval, DateTimeZone preTz, DateTimeZone postTz) {
|
||||||
|
this.interval = interval;
|
||||||
|
this.preTz = preTz;
|
||||||
|
this.postTz = postTz;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
long time = utcMillis + preTz.getOffset(utcMillis);
|
||||||
|
time = Rounding.Interval.round(time, interval);
|
||||||
|
// now, time is still in local, move it to UTC
|
||||||
|
time = time - preTz.getOffset(time);
|
||||||
|
// now apply post Tz
|
||||||
|
time = time + postTz.getOffset(time);
|
||||||
|
return time;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return value + interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
interval = in.readVLong();
|
||||||
|
preTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
postTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeVLong(interval);
|
||||||
|
out.writeSharedString(preTz.getID());
|
||||||
|
out.writeSharedString(postTz.getID());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class DayIntervalTimeZoneRounding extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 6;
|
||||||
|
|
||||||
|
private long interval;
|
||||||
|
private DateTimeZone preTz;
|
||||||
|
private DateTimeZone postTz;
|
||||||
|
|
||||||
|
DayIntervalTimeZoneRounding() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
DayIntervalTimeZoneRounding(long interval, DateTimeZone preTz, DateTimeZone postTz) {
|
||||||
|
this.interval = interval;
|
||||||
|
this.preTz = preTz;
|
||||||
|
this.postTz = postTz;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
long time = utcMillis + preTz.getOffset(utcMillis);
|
||||||
|
time = Rounding.Interval.round(time, interval);
|
||||||
|
// after rounding, since its day level (and above), its actually UTC!
|
||||||
|
// now apply post Tz
|
||||||
|
time = time + postTz.getOffset(time);
|
||||||
|
return time;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return value + interval;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
interval = in.readVLong();
|
||||||
|
preTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
postTz = DateTimeZone.forID(in.readSharedString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeVLong(interval);
|
||||||
|
out.writeSharedString(preTz.getID());
|
||||||
|
out.writeSharedString(postTz.getID());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class FactorTimeZoneRounding extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 7;
|
||||||
|
|
||||||
|
private TimeZoneRounding timeZoneRounding;
|
||||||
|
|
||||||
|
private float factor;
|
||||||
|
|
||||||
|
FactorTimeZoneRounding() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
FactorTimeZoneRounding(TimeZoneRounding timeZoneRounding, float factor) {
|
||||||
|
this.timeZoneRounding = timeZoneRounding;
|
||||||
|
this.factor = factor;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
return timeZoneRounding.round((long) (factor * utcMillis));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return timeZoneRounding.nextRoundingValue(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
timeZoneRounding = (TimeZoneRounding) Rounding.Streams.read(in);
|
||||||
|
factor = in.readFloat();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
Rounding.Streams.write(timeZoneRounding, out);
|
||||||
|
out.writeFloat(factor);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class PrePostTimeZoneRounding extends TimeZoneRounding {
|
||||||
|
|
||||||
|
final static byte ID = 8;
|
||||||
|
|
||||||
|
private TimeZoneRounding timeZoneRounding;
|
||||||
|
|
||||||
|
private long preOffset;
|
||||||
|
private long postOffset;
|
||||||
|
|
||||||
|
PrePostTimeZoneRounding() { // for serialization
|
||||||
|
}
|
||||||
|
|
||||||
|
PrePostTimeZoneRounding(TimeZoneRounding timeZoneRounding, long preOffset, long postOffset) {
|
||||||
|
this.timeZoneRounding = timeZoneRounding;
|
||||||
|
this.preOffset = preOffset;
|
||||||
|
this.postOffset = postOffset;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte id() {
|
||||||
|
return ID;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long round(long utcMillis) {
|
||||||
|
return postOffset + timeZoneRounding.round(utcMillis + preOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long nextRoundingValue(long value) {
|
||||||
|
return postOffset + timeZoneRounding.nextRoundingValue(value - postOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
timeZoneRounding = (TimeZoneRounding) Rounding.Streams.read(in);
|
||||||
|
preOffset = in.readVLong();
|
||||||
|
postOffset = in.readVLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
Rounding.Streams.write(timeZoneRounding, out);
|
||||||
|
out.writeVLong(preOffset);
|
||||||
|
out.writeVLong(postOffset);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -234,7 +234,7 @@ public class TimeValue implements Serializable, Streamable {
|
||||||
if (sValue.endsWith("S")) {
|
if (sValue.endsWith("S")) {
|
||||||
millis = Long.parseLong(sValue.substring(0, sValue.length() - 1));
|
millis = Long.parseLong(sValue.substring(0, sValue.length() - 1));
|
||||||
} else if (sValue.endsWith("ms")) {
|
} else if (sValue.endsWith("ms")) {
|
||||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - "ms".length())));
|
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 2)));
|
||||||
} else if (sValue.endsWith("s")) {
|
} else if (sValue.endsWith("s")) {
|
||||||
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 1000);
|
millis = (long) (Double.parseDouble(sValue.substring(0, sValue.length() - 1)) * 1000);
|
||||||
} else if (sValue.endsWith("m")) {
|
} else if (sValue.endsWith("m")) {
|
||||||
|
|
|
@ -0,0 +1,28 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
/** Base abstraction of an array. */
|
||||||
|
interface BigArray {
|
||||||
|
|
||||||
|
/** Return the length of this array. */
|
||||||
|
public long size();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,337 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
/** Utility class to work with arrays. */
|
||||||
|
public enum BigArrays {
|
||||||
|
;
|
||||||
|
|
||||||
|
/** Page size in bytes: 16KB */
|
||||||
|
public static final int PAGE_SIZE_IN_BYTES = 1 << 14;
|
||||||
|
|
||||||
|
/** Returns the next size to grow when working with parallel arrays that may have different page sizes or number of bytes per element. */
|
||||||
|
public static long overSize(long minTargetSize) {
|
||||||
|
return overSize(minTargetSize, PAGE_SIZE_IN_BYTES / 8, 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return the next size to grow to that is >= <code>minTargetSize</code>.
|
||||||
|
* Inspired from {@link ArrayUtil#oversize(int, int)} and adapted to play nicely with paging. */
|
||||||
|
public static long overSize(long minTargetSize, int pageSize, int bytesPerElement) {
|
||||||
|
Preconditions.checkArgument(minTargetSize >= 0, "minTargetSize must be >= 0");
|
||||||
|
Preconditions.checkArgument(pageSize >= 0, "pageSize must be > 0");
|
||||||
|
Preconditions.checkArgument(bytesPerElement > 0, "bytesPerElement must be > 0");
|
||||||
|
|
||||||
|
long newSize;
|
||||||
|
if (minTargetSize < pageSize) {
|
||||||
|
newSize = ArrayUtil.oversize((int)minTargetSize, bytesPerElement);
|
||||||
|
} else {
|
||||||
|
newSize = minTargetSize + (minTargetSize >>> 3);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (newSize > pageSize) {
|
||||||
|
// round to a multiple of pageSize
|
||||||
|
newSize = newSize - (newSize % pageSize) + pageSize;
|
||||||
|
assert newSize % pageSize == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
return newSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
static boolean indexIsInt(long index) {
|
||||||
|
return index == (int) index;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class IntArrayWrapper implements IntArray {
|
||||||
|
|
||||||
|
private final int[] array;
|
||||||
|
|
||||||
|
IntArrayWrapper(int[] array) {
|
||||||
|
this.array = array;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long size() {
|
||||||
|
return array.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int get(long index) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int set(long index, int value) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
final int ret = array[(int) index];
|
||||||
|
array[(int) index] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int increment(long index, int inc) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index] += inc;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class LongArrayWrapper implements LongArray {
|
||||||
|
|
||||||
|
private final long[] array;
|
||||||
|
|
||||||
|
LongArrayWrapper(long[] array) {
|
||||||
|
this.array = array;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long size() {
|
||||||
|
return array.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long get(long index) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long set(long index, long value) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
final long ret = array[(int) index];
|
||||||
|
array[(int) index] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long increment(long index, long inc) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index] += inc;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class DoubleArrayWrapper implements DoubleArray {
|
||||||
|
|
||||||
|
private final double[] array;
|
||||||
|
|
||||||
|
DoubleArrayWrapper(double[] array) {
|
||||||
|
this.array = array;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long size() {
|
||||||
|
return array.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double get(long index) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double set(long index, double value) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
double ret = array[(int) index];
|
||||||
|
array[(int) index] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double increment(long index, double inc) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return array[(int) index] += inc;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(long fromIndex, long toIndex, double value) {
|
||||||
|
assert indexIsInt(fromIndex);
|
||||||
|
assert indexIsInt(toIndex);
|
||||||
|
Arrays.fill(array, (int) fromIndex, (int) toIndex, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class ObjectArrayWrapper<T> implements ObjectArray<T> {
|
||||||
|
|
||||||
|
private final Object[] array;
|
||||||
|
|
||||||
|
ObjectArrayWrapper(Object[] array) {
|
||||||
|
this.array = array;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long size() {
|
||||||
|
return array.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
public T get(long index) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
return (T) array[(int) index];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public T set(long index, T value) {
|
||||||
|
assert indexIsInt(index);
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
T ret = (T) array[(int) index];
|
||||||
|
array[(int) index] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Allocate a new {@link IntArray} of the given capacity. */
|
||||||
|
public static IntArray newIntArray(long size) {
|
||||||
|
if (size <= BigIntArray.PAGE_SIZE) {
|
||||||
|
return new IntArrayWrapper(new int[(int) size]);
|
||||||
|
} else {
|
||||||
|
return new BigIntArray(size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Resize the array to the exact provided size. */
|
||||||
|
public static IntArray resize(IntArray array, long size) {
|
||||||
|
if (array instanceof BigIntArray) {
|
||||||
|
((BigIntArray) array).resize(size);
|
||||||
|
return array;
|
||||||
|
} else {
|
||||||
|
final IntArray newArray = newIntArray(size);
|
||||||
|
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
|
||||||
|
newArray.set(i, array.get(i));
|
||||||
|
}
|
||||||
|
return newArray;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Grow an array to a size that is larger than <code>minSize</code>, preserving content, and potentially reusing part of the provided array. */
|
||||||
|
public static IntArray grow(IntArray array, long minSize) {
|
||||||
|
if (minSize <= array.size()) {
|
||||||
|
return array;
|
||||||
|
}
|
||||||
|
final long newSize = overSize(minSize, BigIntArray.PAGE_SIZE, RamUsageEstimator.NUM_BYTES_INT);
|
||||||
|
return resize(array, newSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Allocate a new {@link LongArray} of the given capacity. */
|
||||||
|
public static LongArray newLongArray(long size) {
|
||||||
|
if (size <= BigLongArray.PAGE_SIZE) {
|
||||||
|
return new LongArrayWrapper(new long[(int) size]);
|
||||||
|
} else {
|
||||||
|
return new BigLongArray(size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Resize the array to the exact provided size. */
|
||||||
|
public static LongArray resize(LongArray array, long size) {
|
||||||
|
if (array instanceof BigLongArray) {
|
||||||
|
((BigLongArray) array).resize(size);
|
||||||
|
return array;
|
||||||
|
} else {
|
||||||
|
final LongArray newArray = newLongArray(size);
|
||||||
|
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
|
||||||
|
newArray.set(i, array.get(i));
|
||||||
|
}
|
||||||
|
return newArray;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Grow an array to a size that is larger than <code>minSize</code>, preserving content, and potentially reusing part of the provided array. */
|
||||||
|
public static LongArray grow(LongArray array, long minSize) {
|
||||||
|
if (minSize <= array.size()) {
|
||||||
|
return array;
|
||||||
|
}
|
||||||
|
final long newSize = overSize(minSize, BigLongArray.PAGE_SIZE, RamUsageEstimator.NUM_BYTES_LONG);
|
||||||
|
return resize(array, newSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Allocate a new {@link LongArray} of the given capacity. */
|
||||||
|
public static DoubleArray newDoubleArray(long size) {
|
||||||
|
if (size <= BigLongArray.PAGE_SIZE) {
|
||||||
|
return new DoubleArrayWrapper(new double[(int) size]);
|
||||||
|
} else {
|
||||||
|
return new BigDoubleArray(size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Resize the array to the exact provided size. */
|
||||||
|
public static DoubleArray resize(DoubleArray array, long size) {
|
||||||
|
if (array instanceof BigDoubleArray) {
|
||||||
|
((BigDoubleArray) array).resize(size);
|
||||||
|
return array;
|
||||||
|
} else {
|
||||||
|
final DoubleArray newArray = newDoubleArray(size);
|
||||||
|
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
|
||||||
|
newArray.set(i, array.get(i));
|
||||||
|
}
|
||||||
|
return newArray;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Grow an array to a size that is larger than <code>minSize</code>, preserving content, and potentially reusing part of the provided array. */
|
||||||
|
public static DoubleArray grow(DoubleArray array, long minSize) {
|
||||||
|
if (minSize <= array.size()) {
|
||||||
|
return array;
|
||||||
|
}
|
||||||
|
final long newSize = overSize(minSize, BigDoubleArray.PAGE_SIZE, RamUsageEstimator.NUM_BYTES_DOUBLE);
|
||||||
|
return resize(array, newSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Allocate a new {@link LongArray} of the given capacity. */
|
||||||
|
public static <T> ObjectArray<T> newObjectArray(long size) {
|
||||||
|
if (size <= BigLongArray.PAGE_SIZE) {
|
||||||
|
return new ObjectArrayWrapper<T>(new Object[(int) size]);
|
||||||
|
} else {
|
||||||
|
return new BigObjectArray<T>(size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Resize the array to the exact provided size. */
|
||||||
|
public static <T> ObjectArray<T> resize(ObjectArray<T> array, long size) {
|
||||||
|
if (array instanceof BigObjectArray) {
|
||||||
|
((BigObjectArray<?>) array).resize(size);
|
||||||
|
return array;
|
||||||
|
} else {
|
||||||
|
final ObjectArray<T> newArray = newObjectArray(size);
|
||||||
|
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
|
||||||
|
newArray.set(i, array.get(i));
|
||||||
|
}
|
||||||
|
return newArray;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Grow an array to a size that is larger than <code>minSize</code>, preserving content, and potentially reusing part of the provided array. */
|
||||||
|
public static <T> ObjectArray<T> grow(ObjectArray<T> array, long minSize) {
|
||||||
|
if (minSize <= array.size()) {
|
||||||
|
return array;
|
||||||
|
}
|
||||||
|
final long newSize = overSize(minSize, BigObjectArray.PAGE_SIZE, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
|
||||||
|
return resize(array, newSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,112 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Double array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
|
||||||
|
* configurable length.
|
||||||
|
*/
|
||||||
|
final class BigDoubleArray extends AbstractBigArray implements DoubleArray {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Page size, 16KB of memory per page.
|
||||||
|
*/
|
||||||
|
public static final int PAGE_SIZE = BigArrays.PAGE_SIZE_IN_BYTES / RamUsageEstimator.NUM_BYTES_DOUBLE;
|
||||||
|
|
||||||
|
|
||||||
|
private double[][] pages;
|
||||||
|
|
||||||
|
/** Constructor. */
|
||||||
|
public BigDoubleArray(long size) {
|
||||||
|
super(PAGE_SIZE);
|
||||||
|
this.size = size;
|
||||||
|
pages = new double[numPages(size)][];
|
||||||
|
for (int i = 0; i < pages.length; ++i) {
|
||||||
|
pages[i] = new double[pageSize()];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double get(long index) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
return pages[pageIndex][indexInPage];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double set(long index, double value) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
final double[] page = pages[pageIndex];
|
||||||
|
final double ret = page[indexInPage];
|
||||||
|
page[indexInPage] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double increment(long index, double inc) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
return pages[pageIndex][indexInPage] += inc;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numBytesPerElement() {
|
||||||
|
return RamUsageEstimator.NUM_BYTES_INT;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Change the size of this array. Content between indexes <code>0</code> and <code>min(size(), newSize)</code> will be preserved. */
|
||||||
|
public void resize(long newSize) {
|
||||||
|
final int numPages = numPages(newSize);
|
||||||
|
if (numPages > pages.length) {
|
||||||
|
pages = Arrays.copyOf(pages, ArrayUtil.oversize(numPages, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
|
||||||
|
}
|
||||||
|
for (int i = numPages - 1; i >= 0 && pages[i] == null; --i) {
|
||||||
|
pages[i] = new double[pageSize()];
|
||||||
|
}
|
||||||
|
for (int i = numPages; i < pages.length && pages[i] != null; ++i) {
|
||||||
|
pages[i] = null;
|
||||||
|
}
|
||||||
|
this.size = newSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void fill(long fromIndex, long toIndex, double value) {
|
||||||
|
Preconditions.checkArgument(fromIndex <= toIndex);
|
||||||
|
final int fromPage = pageIndex(fromIndex);
|
||||||
|
final int toPage = pageIndex(toIndex - 1);
|
||||||
|
if (fromPage == toPage) {
|
||||||
|
Arrays.fill(pages[fromPage], indexInPage(fromIndex), indexInPage(toIndex - 1) + 1, value);
|
||||||
|
} else {
|
||||||
|
Arrays.fill(pages[fromPage], indexInPage(fromIndex), pages[fromPage].length, value);
|
||||||
|
for (int i = fromPage + 1; i < toPage; ++i) {
|
||||||
|
Arrays.fill(pages[i], value);
|
||||||
|
}
|
||||||
|
Arrays.fill(pages[toPage], 0, indexInPage(toIndex - 1) + 1, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.common.util;
|
package org.elasticsearch.common.util;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.RamUsageEstimator;
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -27,17 +28,19 @@ import java.util.Arrays;
|
||||||
* Int array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
|
* Int array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
|
||||||
* configurable length.
|
* configurable length.
|
||||||
*/
|
*/
|
||||||
public final class BigIntArray extends AbstractBigArray implements IntArray {
|
final class BigIntArray extends AbstractBigArray implements IntArray {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default page size, 16KB of memory per page.
|
* Page size, 16KB of memory per page.
|
||||||
*/
|
*/
|
||||||
public static final int DEFAULT_PAGE_SIZE = 1 << 12;
|
public static final int PAGE_SIZE = BigArrays.PAGE_SIZE_IN_BYTES / RamUsageEstimator.NUM_BYTES_INT;
|
||||||
|
|
||||||
|
|
||||||
private int[][] pages;
|
private int[][] pages;
|
||||||
|
|
||||||
public BigIntArray(int pageSize, long size) {
|
/** Constructor. */
|
||||||
super(pageSize);
|
public BigIntArray(long size) {
|
||||||
|
super(PAGE_SIZE);
|
||||||
this.size = size;
|
this.size = size;
|
||||||
pages = new int[numPages(size)][];
|
pages = new int[numPages(size)][];
|
||||||
for (int i = 0; i < pages.length; ++i) {
|
for (int i = 0; i < pages.length; ++i) {
|
||||||
|
@ -45,22 +48,24 @@ public final class BigIntArray extends AbstractBigArray implements IntArray {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public BigIntArray(long size) {
|
@Override
|
||||||
this(DEFAULT_PAGE_SIZE, size);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int get(long index) {
|
public int get(long index) {
|
||||||
final int pageIndex = pageIndex(index);
|
final int pageIndex = pageIndex(index);
|
||||||
final int indexInPage = indexInPage(index);
|
final int indexInPage = indexInPage(index);
|
||||||
return pages[pageIndex][indexInPage];
|
return pages[pageIndex][indexInPage];
|
||||||
}
|
}
|
||||||
|
|
||||||
public void set(long index, int value) {
|
@Override
|
||||||
|
public int set(long index, int value) {
|
||||||
final int pageIndex = pageIndex(index);
|
final int pageIndex = pageIndex(index);
|
||||||
final int indexInPage = indexInPage(index);
|
final int indexInPage = indexInPage(index);
|
||||||
pages[pageIndex][indexInPage] = value;
|
final int[] page = pages[pageIndex];
|
||||||
|
final int ret = page[indexInPage];
|
||||||
|
page[indexInPage] = value;
|
||||||
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
public int increment(long index, int inc) {
|
public int increment(long index, int inc) {
|
||||||
final int pageIndex = pageIndex(index);
|
final int pageIndex = pageIndex(index);
|
||||||
final int indexInPage = indexInPage(index);
|
final int indexInPage = indexInPage(index);
|
||||||
|
@ -72,10 +77,19 @@ public final class BigIntArray extends AbstractBigArray implements IntArray {
|
||||||
return RamUsageEstimator.NUM_BYTES_INT;
|
return RamUsageEstimator.NUM_BYTES_INT;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/** Change the size of this array. Content between indexes <code>0</code> and <code>min(size(), newSize)</code> will be preserved. */
|
||||||
public void clear(int sentinal) {
|
public void resize(long newSize) {
|
||||||
for (int[] page : pages) {
|
final int numPages = numPages(newSize);
|
||||||
Arrays.fill(page, sentinal);
|
if (numPages > pages.length) {
|
||||||
|
pages = Arrays.copyOf(pages, ArrayUtil.oversize(numPages, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
|
||||||
}
|
}
|
||||||
|
for (int i = numPages - 1; i >= 0 && pages[i] == null; --i) {
|
||||||
|
pages[i] = new int[pageSize()];
|
||||||
|
}
|
||||||
|
for (int i = numPages; i < pages.length && pages[i] != null; ++i) {
|
||||||
|
pages[i] = null;
|
||||||
|
}
|
||||||
|
this.size = newSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,59 +19,77 @@
|
||||||
|
|
||||||
package org.elasticsearch.common.util;
|
package org.elasticsearch.common.util;
|
||||||
|
|
||||||
import java.util.Locale;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A GC friendly long[].
|
* Long array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
|
||||||
* Allocating large arrays (that are not short-lived) generate fragmentation
|
* configurable length.
|
||||||
* in old-gen space. This breaks such large long array into fixed size pages
|
|
||||||
* to avoid that problem.
|
|
||||||
*/
|
*/
|
||||||
public class BigLongArray {
|
final class BigLongArray extends AbstractBigArray implements LongArray {
|
||||||
|
|
||||||
private static final int DEFAULT_PAGE_SIZE = 4096;
|
/**
|
||||||
|
* Page size, 16KB of memory per page.
|
||||||
|
*/
|
||||||
|
public static final int PAGE_SIZE = BigArrays.PAGE_SIZE_IN_BYTES / RamUsageEstimator.NUM_BYTES_LONG;
|
||||||
|
|
||||||
|
|
||||||
private final long[][] pages;
|
private long[][] pages;
|
||||||
public final int size;
|
|
||||||
|
|
||||||
private final int pageSize;
|
/** Constructor. */
|
||||||
private final int pageCount;
|
public BigLongArray(long size) {
|
||||||
|
super(PAGE_SIZE);
|
||||||
public BigLongArray(int size) {
|
|
||||||
this(size, DEFAULT_PAGE_SIZE);
|
|
||||||
}
|
|
||||||
|
|
||||||
public BigLongArray(int size, int pageSize) {
|
|
||||||
this.size = size;
|
this.size = size;
|
||||||
this.pageSize = pageSize;
|
pages = new long[numPages(size)][];
|
||||||
|
for (int i = 0; i < pages.length; ++i) {
|
||||||
int lastPageSize = size % pageSize;
|
pages[i] = new long[pageSize()];
|
||||||
int fullPageCount = size / pageSize;
|
}
|
||||||
pageCount = fullPageCount + (lastPageSize == 0 ? 0 : 1);
|
|
||||||
pages = new long[pageCount][];
|
|
||||||
|
|
||||||
for (int i = 0; i < fullPageCount; ++i)
|
|
||||||
pages[i] = new long[pageSize];
|
|
||||||
|
|
||||||
if (lastPageSize != 0)
|
|
||||||
pages[pages.length - 1] = new long[lastPageSize];
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void set(int idx, long value) {
|
@Override
|
||||||
if (idx < 0 || idx > size)
|
public long get(long index) {
|
||||||
throw new IndexOutOfBoundsException(String.format(Locale.ROOT, "%d is not whithin [0, %d)", idx, size));
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
int page = idx / pageSize;
|
return pages[pageIndex][indexInPage];
|
||||||
int pageIdx = idx % pageSize;
|
|
||||||
pages[page][pageIdx] = value;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public long get(int idx) {
|
@Override
|
||||||
if (idx < 0 || idx > size)
|
public long set(long index, long value) {
|
||||||
throw new IndexOutOfBoundsException(String.format(Locale.ROOT, "%d is not whithin [0, %d)", idx, size));
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
int page = idx / pageSize;
|
final long[] page = pages[pageIndex];
|
||||||
int pageIdx = idx % pageSize;
|
final long ret = page[indexInPage];
|
||||||
return pages[page][pageIdx];
|
page[indexInPage] = value;
|
||||||
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long increment(long index, long inc) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
return pages[pageIndex][indexInPage] += inc;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numBytesPerElement() {
|
||||||
|
return RamUsageEstimator.NUM_BYTES_LONG;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Change the size of this array. Content between indexes <code>0</code> and <code>min(size(), newSize)</code> will be preserved. */
|
||||||
|
public void resize(long newSize) {
|
||||||
|
final int numPages = numPages(newSize);
|
||||||
|
if (numPages > pages.length) {
|
||||||
|
pages = Arrays.copyOf(pages, ArrayUtil.oversize(numPages, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
|
||||||
|
}
|
||||||
|
for (int i = numPages - 1; i >= 0 && pages[i] == null; --i) {
|
||||||
|
pages[i] = new long[pageSize()];
|
||||||
|
}
|
||||||
|
for (int i = numPages; i < pages.length && pages[i] != null; ++i) {
|
||||||
|
pages[i] = null;
|
||||||
|
}
|
||||||
|
this.size = newSize;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,90 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.RamUsageEstimator;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Int array abstraction able to support more than 2B values. This implementation slices data into fixed-sized blocks of
|
||||||
|
* configurable length.
|
||||||
|
*/
|
||||||
|
final class BigObjectArray<T> extends AbstractBigArray implements ObjectArray<T> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Page size, 16KB of memory per page.
|
||||||
|
*/
|
||||||
|
public static final int PAGE_SIZE = BigArrays.PAGE_SIZE_IN_BYTES / RamUsageEstimator.NUM_BYTES_OBJECT_REF;
|
||||||
|
|
||||||
|
|
||||||
|
private Object[][] pages;
|
||||||
|
|
||||||
|
/** Constructor. */
|
||||||
|
public BigObjectArray(long size) {
|
||||||
|
super(PAGE_SIZE);
|
||||||
|
this.size = size;
|
||||||
|
pages = new Object[numPages(size)][];
|
||||||
|
for (int i = 0; i < pages.length; ++i) {
|
||||||
|
pages[i] = new Object[pageSize()];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
public T get(long index) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
return (T) pages[pageIndex][indexInPage];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public T set(long index, T value) {
|
||||||
|
final int pageIndex = pageIndex(index);
|
||||||
|
final int indexInPage = indexInPage(index);
|
||||||
|
final Object[] page = pages[pageIndex];
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final T ret = (T) page[indexInPage];
|
||||||
|
page[indexInPage] = value;
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numBytesPerElement() {
|
||||||
|
return RamUsageEstimator.NUM_BYTES_INT;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Change the size of this array. Content between indexes <code>0</code> and <code>min(size(), newSize)</code> will be preserved. */
|
||||||
|
public void resize(long newSize) {
|
||||||
|
final int numPages = numPages(newSize);
|
||||||
|
if (numPages > pages.length) {
|
||||||
|
pages = Arrays.copyOf(pages, ArrayUtil.oversize(numPages, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
|
||||||
|
}
|
||||||
|
for (int i = numPages - 1; i >= 0 && pages[i] == null; --i) {
|
||||||
|
pages[i] = new Object[pageSize()];
|
||||||
|
}
|
||||||
|
for (int i = numPages; i < pages.length && pages[i] != null; ++i) {
|
||||||
|
pages[i] = null;
|
||||||
|
}
|
||||||
|
this.size = newSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,47 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstraction of an array of double values.
|
||||||
|
*/
|
||||||
|
public interface DoubleArray extends BigArray {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an element given its index.
|
||||||
|
*/
|
||||||
|
public abstract double get(long index);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set a value at the given index and return the previous value.
|
||||||
|
*/
|
||||||
|
public abstract double set(long index, double value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment value at the given index by <code>inc</code> and return the value.
|
||||||
|
*/
|
||||||
|
public abstract double increment(long index, double inc);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Fill slots between <code>fromIndex</code> inclusive to <code>toIndex</code> exclusive with <code>value</code>.
|
||||||
|
*/
|
||||||
|
public abstract void fill(long fromIndex, long toIndex, double value);
|
||||||
|
|
||||||
|
}
|
|
@ -22,7 +22,7 @@ package org.elasticsearch.common.util;
|
||||||
/**
|
/**
|
||||||
* Abstraction of an array of integer values.
|
* Abstraction of an array of integer values.
|
||||||
*/
|
*/
|
||||||
public interface IntArray {
|
public interface IntArray extends BigArray {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get an element given its index.
|
* Get an element given its index.
|
||||||
|
@ -30,14 +30,13 @@ public interface IntArray {
|
||||||
public abstract int get(long index);
|
public abstract int get(long index);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set a value at the given index.
|
* Set a value at the given index and return the previous value.
|
||||||
*/
|
*/
|
||||||
public abstract void set(long index, int value);
|
public abstract int set(long index, int value);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Increment value at the given index by <code>inc</code> and return the value.
|
* Increment value at the given index by <code>inc</code> and return the value.
|
||||||
*/
|
*/
|
||||||
public abstract int increment(long index, int inc);
|
public abstract int increment(long index, int inc);
|
||||||
|
|
||||||
void clear(int sentinal);
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,80 +0,0 @@
|
||||||
/*
|
|
||||||
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
|
||||||
|
|
||||||
import java.util.Arrays;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Utility methods to work with {@link IntArray}s.
|
|
||||||
*/
|
|
||||||
public class IntArrays {
|
|
||||||
|
|
||||||
private IntArrays() {
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return a {@link IntArray} view over the provided array.
|
|
||||||
*/
|
|
||||||
public static IntArray wrap(final int[] array) {
|
|
||||||
return new IntArray() {
|
|
||||||
|
|
||||||
private void checkIndex(long index) {
|
|
||||||
if (index > Integer.MAX_VALUE) {
|
|
||||||
throw new IndexOutOfBoundsException(Long.toString(index));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void set(long index, int value) {
|
|
||||||
checkIndex(index);
|
|
||||||
array[(int) index] = value;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int increment(long index, int inc) {
|
|
||||||
checkIndex(index);
|
|
||||||
return array[(int) index] += inc;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int get(long index) {
|
|
||||||
checkIndex(index);
|
|
||||||
return array[(int) index];
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void clear(int sentinal) {
|
|
||||||
Arrays.fill(array, sentinal);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return a newly allocated {@link IntArray} of the given length or more.
|
|
||||||
*/
|
|
||||||
public static IntArray allocate(long length) {
|
|
||||||
if (length <= BigIntArray.DEFAULT_PAGE_SIZE) {
|
|
||||||
return wrap(new int[(int) length]);
|
|
||||||
} else {
|
|
||||||
return new BigIntArray(length);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -0,0 +1,42 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstraction of an array of long values.
|
||||||
|
*/
|
||||||
|
public interface LongArray extends BigArray {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an element given its index.
|
||||||
|
*/
|
||||||
|
public abstract long get(long index);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set a value at the given index and return the previous value.
|
||||||
|
*/
|
||||||
|
public abstract long set(long index, long value);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Increment value at the given index by <code>inc</code> and return the value.
|
||||||
|
*/
|
||||||
|
public abstract long increment(long index, long inc);
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,37 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.common.util;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstraction of an array of object values.
|
||||||
|
*/
|
||||||
|
public interface ObjectArray<T> extends BigArray {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an element given its index.
|
||||||
|
*/
|
||||||
|
public abstract T get(long index);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set a value at the given index and return the previous value.
|
||||||
|
*/
|
||||||
|
public abstract T set(long index, T value);
|
||||||
|
|
||||||
|
}
|
|
@ -1148,6 +1148,8 @@ public final class XContentBuilder implements BytesStream {
|
||||||
generator.writeString(XContentBuilder.defaultDatePrinter.print(((Date) value).getTime()));
|
generator.writeString(XContentBuilder.defaultDatePrinter.print(((Date) value).getTime()));
|
||||||
} else if (value instanceof Calendar) {
|
} else if (value instanceof Calendar) {
|
||||||
generator.writeString(XContentBuilder.defaultDatePrinter.print((((Calendar) value)).getTimeInMillis()));
|
generator.writeString(XContentBuilder.defaultDatePrinter.print((((Calendar) value)).getTimeInMillis()));
|
||||||
|
} else if (value instanceof ReadableInstant) {
|
||||||
|
generator.writeString(XContentBuilder.defaultDatePrinter.print((((ReadableInstant) value)).getMillis()));
|
||||||
} else if (value instanceof BytesReference) {
|
} else if (value instanceof BytesReference) {
|
||||||
BytesReference bytes = (BytesReference) value;
|
BytesReference bytes = (BytesReference) value;
|
||||||
if (!bytes.hasArray()) {
|
if (!bytes.hasArray()) {
|
||||||
|
|
|
@ -24,7 +24,8 @@ import org.apache.lucene.util.IntsRef;
|
||||||
import org.apache.lucene.util.fst.*;
|
import org.apache.lucene.util.fst.*;
|
||||||
import org.apache.lucene.util.fst.FST.Arc;
|
import org.apache.lucene.util.fst.FST.Arc;
|
||||||
import org.apache.lucene.util.fst.FST.BytesReader;
|
import org.apache.lucene.util.fst.FST.BytesReader;
|
||||||
import org.elasticsearch.common.util.BigIntArray;
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.IntArray;
|
||||||
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
||||||
import org.elasticsearch.index.fielddata.ScriptDocValues;
|
import org.elasticsearch.index.fielddata.ScriptDocValues;
|
||||||
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
|
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
|
||||||
|
@ -44,7 +45,7 @@ public class FSTBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<Scr
|
||||||
// 0 ordinal in values means no value (its null)
|
// 0 ordinal in values means no value (its null)
|
||||||
protected final Ordinals ordinals;
|
protected final Ordinals ordinals;
|
||||||
|
|
||||||
private volatile BigIntArray hashes;
|
private volatile IntArray hashes;
|
||||||
private long size = -1;
|
private long size = -1;
|
||||||
|
|
||||||
private final FST<Long> fst;
|
private final FST<Long> fst;
|
||||||
|
@ -95,7 +96,7 @@ public class FSTBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<Scr
|
||||||
if (needsHashes) {
|
if (needsHashes) {
|
||||||
if (hashes == null) {
|
if (hashes == null) {
|
||||||
BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
|
BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
|
||||||
BigIntArray hashes = new BigIntArray(ordinals.getMaxOrd());
|
IntArray hashes = BigArrays.newIntArray(ordinals.getMaxOrd());
|
||||||
// we don't store an ord 0 in the FST since we could have an empty string in there and FST don't support
|
// we don't store an ord 0 in the FST since we could have an empty string in there and FST don't support
|
||||||
// empty strings twice. ie. them merge fails for long output.
|
// empty strings twice. ie. them merge fails for long output.
|
||||||
hashes.set(0, new BytesRef().hashCode());
|
hashes.set(0, new BytesRef().hashCode());
|
||||||
|
@ -164,9 +165,9 @@ public class FSTBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<Scr
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class HashedBytesValues extends BytesValues {
|
static final class HashedBytesValues extends BytesValues {
|
||||||
private final BigIntArray hashes;
|
private final IntArray hashes;
|
||||||
|
|
||||||
HashedBytesValues(FST<Long> fst, Docs ordinals, BigIntArray hashes) {
|
HashedBytesValues(FST<Long> fst, Docs ordinals, IntArray hashes) {
|
||||||
super(fst, ordinals);
|
super(fst, ordinals);
|
||||||
this.hashes = hashes;
|
this.hashes = hashes;
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,8 @@ import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.PagedBytes;
|
import org.apache.lucene.util.PagedBytes;
|
||||||
import org.apache.lucene.util.PagedBytes.Reader;
|
import org.apache.lucene.util.PagedBytes.Reader;
|
||||||
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
|
import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
|
||||||
import org.elasticsearch.common.util.BigIntArray;
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.IntArray;
|
||||||
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
||||||
import org.elasticsearch.index.fielddata.ScriptDocValues;
|
import org.elasticsearch.index.fielddata.ScriptDocValues;
|
||||||
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
|
import org.elasticsearch.index.fielddata.ordinals.EmptyOrdinals;
|
||||||
|
@ -43,7 +44,7 @@ public class PagedBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<S
|
||||||
private final MonotonicAppendingLongBuffer termOrdToBytesOffset;
|
private final MonotonicAppendingLongBuffer termOrdToBytesOffset;
|
||||||
protected final Ordinals ordinals;
|
protected final Ordinals ordinals;
|
||||||
|
|
||||||
private volatile BigIntArray hashes;
|
private volatile IntArray hashes;
|
||||||
private long size = -1;
|
private long size = -1;
|
||||||
private final long readerBytesSize;
|
private final long readerBytesSize;
|
||||||
|
|
||||||
|
@ -91,10 +92,10 @@ public class PagedBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<S
|
||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
private final BigIntArray getHashes() {
|
private final IntArray getHashes() {
|
||||||
if (hashes == null) {
|
if (hashes == null) {
|
||||||
long numberOfValues = termOrdToBytesOffset.size();
|
long numberOfValues = termOrdToBytesOffset.size();
|
||||||
BigIntArray hashes = new BigIntArray(numberOfValues);
|
IntArray hashes = BigArrays.newIntArray(numberOfValues);
|
||||||
BytesRef scratch = new BytesRef();
|
BytesRef scratch = new BytesRef();
|
||||||
for (long i = 0; i < numberOfValues; i++) {
|
for (long i = 0; i < numberOfValues; i++) {
|
||||||
bytes.fill(scratch, termOrdToBytesOffset.get(i));
|
bytes.fill(scratch, termOrdToBytesOffset.get(i));
|
||||||
|
@ -108,7 +109,7 @@ public class PagedBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<S
|
||||||
@Override
|
@Override
|
||||||
public BytesValues.WithOrdinals getBytesValues(boolean needsHashes) {
|
public BytesValues.WithOrdinals getBytesValues(boolean needsHashes) {
|
||||||
if (needsHashes) {
|
if (needsHashes) {
|
||||||
final BigIntArray hashes = getHashes();
|
final IntArray hashes = getHashes();
|
||||||
return new BytesValues.HashedBytesValues(hashes, bytes, termOrdToBytesOffset, ordinals.ordinals());
|
return new BytesValues.HashedBytesValues(hashes, bytes, termOrdToBytesOffset, ordinals.ordinals());
|
||||||
} else {
|
} else {
|
||||||
return new BytesValues(bytes, termOrdToBytesOffset, ordinals.ordinals());
|
return new BytesValues(bytes, termOrdToBytesOffset, ordinals.ordinals());
|
||||||
|
@ -159,10 +160,10 @@ public class PagedBytesAtomicFieldData implements AtomicFieldData.WithOrdinals<S
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class HashedBytesValues extends BytesValues {
|
static final class HashedBytesValues extends BytesValues {
|
||||||
private final BigIntArray hashes;
|
private final IntArray hashes;
|
||||||
|
|
||||||
|
|
||||||
HashedBytesValues(BigIntArray hashes, Reader bytes, MonotonicAppendingLongBuffer termOrdToBytesOffset, Docs ordinals) {
|
HashedBytesValues(IntArray hashes, Reader bytes, MonotonicAppendingLongBuffer termOrdToBytesOffset, Docs ordinals) {
|
||||||
super(bytes, termOrdToBytesOffset, ordinals);
|
super(bytes, termOrdToBytesOffset, ordinals);
|
||||||
this.hashes = hashes;
|
this.hashes = hashes;
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,8 +25,8 @@ import org.apache.lucene.util.ArrayUtil;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.LongsRef;
|
import org.apache.lucene.util.LongsRef;
|
||||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||||
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
import org.elasticsearch.common.util.IntArray;
|
import org.elasticsearch.common.util.IntArray;
|
||||||
import org.elasticsearch.common.util.IntArrays;
|
|
||||||
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
||||||
import org.elasticsearch.index.fielddata.BytesValues;
|
import org.elasticsearch.index.fielddata.BytesValues;
|
||||||
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
|
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
|
||||||
|
@ -85,7 +85,7 @@ abstract class SortedSetDVAtomicFieldData {
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
if (hashes == null) {
|
if (hashes == null) {
|
||||||
final long valueCount = values.getValueCount();
|
final long valueCount = values.getValueCount();
|
||||||
final IntArray hashes = IntArrays.allocate(1L + valueCount);
|
final IntArray hashes = BigArrays.newIntArray(1L + valueCount);
|
||||||
BytesRef scratch = new BytesRef(16);
|
BytesRef scratch = new BytesRef(16);
|
||||||
hashes.set(0, scratch.hashCode());
|
hashes.set(0, scratch.hashCode());
|
||||||
for (long i = 0; i < valueCount; ++i) {
|
for (long i = 0; i < valueCount; ++i) {
|
||||||
|
|
|
@ -215,6 +215,14 @@ public class DateFieldMapper extends NumberFieldMapper<Long> {
|
||||||
this.dateMathParser = new DateMathParser(dateTimeFormatter, timeUnit);
|
this.dateMathParser = new DateMathParser(dateTimeFormatter, timeUnit);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public FormatDateTimeFormatter dateTimeFormatter() {
|
||||||
|
return dateTimeFormatter;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateMathParser dateMathParser() {
|
||||||
|
return dateMathParser;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FieldType defaultFieldType() {
|
public FieldType defaultFieldType() {
|
||||||
return Defaults.FIELD_TYPE;
|
return Defaults.FIELD_TYPE;
|
||||||
|
|
|
@ -55,6 +55,7 @@ import org.elasticsearch.script.ScriptService;
|
||||||
import org.elasticsearch.search.Scroll;
|
import org.elasticsearch.search.Scroll;
|
||||||
import org.elasticsearch.search.SearchHitField;
|
import org.elasticsearch.search.SearchHitField;
|
||||||
import org.elasticsearch.search.SearchShardTarget;
|
import org.elasticsearch.search.SearchShardTarget;
|
||||||
|
import org.elasticsearch.search.aggregations.SearchContextAggregations;
|
||||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||||
import org.elasticsearch.search.facet.SearchContextFacets;
|
import org.elasticsearch.search.facet.SearchContextFacets;
|
||||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||||
|
@ -283,6 +284,16 @@ public class PercolateContext extends SearchContext {
|
||||||
return fieldDataService;
|
return fieldDataService;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SearchContextAggregations aggregations() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SearchContext aggregations(SearchContextAggregations aggregations) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SearchContextFacets facets() {
|
public SearchContextFacets facets() {
|
||||||
return facets;
|
return facets;
|
||||||
|
|
|
@ -19,10 +19,11 @@
|
||||||
|
|
||||||
package org.elasticsearch.script;
|
package org.elasticsearch.script;
|
||||||
|
|
||||||
import org.apache.lucene.index.AtomicReader;
|
import org.elasticsearch.common.lucene.ReaderContextAware;
|
||||||
import org.apache.lucene.index.AtomicReaderContext;
|
import org.elasticsearch.common.lucene.ScorerAware;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.elasticsearch.search.SearchService;
|
||||||
import org.apache.lucene.search.Scorer;
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
import org.elasticsearch.search.lookup.SearchLookup;
|
||||||
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
|
@ -31,11 +32,7 @@ import java.util.Map;
|
||||||
*
|
*
|
||||||
* @see ExplainableSearchScript for script which can explain a score
|
* @see ExplainableSearchScript for script which can explain a score
|
||||||
*/
|
*/
|
||||||
public interface SearchScript extends ExecutableScript {
|
public interface SearchScript extends ExecutableScript, ReaderContextAware, ScorerAware {
|
||||||
|
|
||||||
void setScorer(Scorer scorer);
|
|
||||||
|
|
||||||
void setNextReader(AtomicReaderContext context);
|
|
||||||
|
|
||||||
void setNextDocId(int doc);
|
void setNextDocId(int doc);
|
||||||
|
|
||||||
|
@ -48,4 +45,34 @@ public interface SearchScript extends ExecutableScript {
|
||||||
long runAsLong();
|
long runAsLong();
|
||||||
|
|
||||||
double runAsDouble();
|
double runAsDouble();
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private String script;
|
||||||
|
private String lang;
|
||||||
|
private Map<String, Object> params;
|
||||||
|
|
||||||
|
public Builder script(String script) {
|
||||||
|
this.script = script;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder lang(String lang) {
|
||||||
|
this.lang = lang;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder params(Map<String, Object> params) {
|
||||||
|
this.params = params;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public SearchScript build(SearchContext context) {
|
||||||
|
return build(context.scriptService(), context.lookup());
|
||||||
|
}
|
||||||
|
|
||||||
|
public SearchScript build(ScriptService service, SearchLookup lookup) {
|
||||||
|
return service.search(lookup, lang, script, params);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -54,7 +54,7 @@ public class MvelScriptEngineService extends AbstractComponent implements Script
|
||||||
|
|
||||||
parserConfiguration = new ParserConfiguration();
|
parserConfiguration = new ParserConfiguration();
|
||||||
parserConfiguration.addPackageImport("java.util");
|
parserConfiguration.addPackageImport("java.util");
|
||||||
parserConfiguration.addPackageImport("org.joda");
|
parserConfiguration.addPackageImport("org.joda.time");
|
||||||
parserConfiguration.addImport("time", MVEL.getStaticMethod(System.class, "currentTimeMillis", new Class[0]));
|
parserConfiguration.addImport("time", MVEL.getStaticMethod(System.class, "currentTimeMillis", new Class[0]));
|
||||||
// unboxed version of Math, better performance since conversion from boxed to unboxed my mvel is not needed
|
// unboxed version of Math, better performance since conversion from boxed to unboxed my mvel is not needed
|
||||||
for (Method m : UnboxedMathUtils.class.getMethods()) {
|
for (Method m : UnboxedMathUtils.class.getMethods()) {
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.common.inject.Module;
|
||||||
import org.elasticsearch.common.inject.SpawnModules;
|
import org.elasticsearch.common.inject.SpawnModules;
|
||||||
import org.elasticsearch.index.query.functionscore.FunctionScoreModule;
|
import org.elasticsearch.index.query.functionscore.FunctionScoreModule;
|
||||||
import org.elasticsearch.search.action.SearchServiceTransportAction;
|
import org.elasticsearch.search.action.SearchServiceTransportAction;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationModule;
|
||||||
import org.elasticsearch.search.controller.SearchPhaseController;
|
import org.elasticsearch.search.controller.SearchPhaseController;
|
||||||
import org.elasticsearch.search.dfs.DfsPhase;
|
import org.elasticsearch.search.dfs.DfsPhase;
|
||||||
import org.elasticsearch.search.facet.FacetModule;
|
import org.elasticsearch.search.facet.FacetModule;
|
||||||
|
@ -47,7 +48,7 @@ public class SearchModule extends AbstractModule implements SpawnModules {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterable<? extends Module> spawnModules() {
|
public Iterable<? extends Module> spawnModules() {
|
||||||
return ImmutableList.of(new TransportSearchModule(), new FacetModule(), new HighlightModule(), new SuggestModule(), new FunctionScoreModule());
|
return ImmutableList.of(new TransportSearchModule(), new FacetModule(), new HighlightModule(), new SuggestModule(), new FunctionScoreModule(), new AggregationModule());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList;
|
||||||
import org.elasticsearch.common.inject.AbstractModule;
|
import org.elasticsearch.common.inject.AbstractModule;
|
||||||
import org.elasticsearch.common.inject.Module;
|
import org.elasticsearch.common.inject.Module;
|
||||||
import org.elasticsearch.common.inject.SpawnModules;
|
import org.elasticsearch.common.inject.SpawnModules;
|
||||||
|
import org.elasticsearch.search.aggregations.TransportAggregationModule;
|
||||||
import org.elasticsearch.search.facet.TransportFacetModule;
|
import org.elasticsearch.search.facet.TransportFacetModule;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,7 +33,7 @@ public class TransportSearchModule extends AbstractModule implements SpawnModule
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterable<? extends Module> spawnModules() {
|
public Iterable<? extends Module> spawnModules() {
|
||||||
return ImmutableList.of(new TransportFacetModule());
|
return ImmutableList.of(new TransportFacetModule(), new TransportAggregationModule());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,18 @@
|
||||||
|
package org.elasticsearch.search.aggregations;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class AbstractAggregationBuilder implements ToXContent {
|
||||||
|
|
||||||
|
protected final String name;
|
||||||
|
protected final String type;
|
||||||
|
|
||||||
|
protected AbstractAggregationBuilder(String name, String type) {
|
||||||
|
this.name = name;
|
||||||
|
this.type = type;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,32 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An aggregation
|
||||||
|
*/
|
||||||
|
public interface Aggregation {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The name of this aggregation.
|
||||||
|
*/
|
||||||
|
String getName();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,48 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class AggregationBinaryParseElement extends AggregationParseElement {
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public AggregationBinaryParseElement(AggregatorParsers aggregatorParsers) {
|
||||||
|
super(aggregatorParsers);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void parse(XContentParser parser, SearchContext context) throws Exception {
|
||||||
|
byte[] facetSource = parser.binaryValue();
|
||||||
|
XContentParser aSourceParser = XContentFactory.xContent(facetSource).createParser(facetSource);
|
||||||
|
try {
|
||||||
|
aSourceParser.nextToken(); // move past the first START_OBJECT
|
||||||
|
super.parse(aSourceParser, context);
|
||||||
|
} finally {
|
||||||
|
aSourceParser.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,113 @@
|
||||||
|
package org.elasticsearch.search.aggregations;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.elasticsearch.ElasticSearchGenerationException;
|
||||||
|
import org.elasticsearch.client.Requests;
|
||||||
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A base class for all bucket aggregation builders.
|
||||||
|
*/
|
||||||
|
public abstract class AggregationBuilder<B extends AggregationBuilder<B>> extends AbstractAggregationBuilder {
|
||||||
|
|
||||||
|
private List<AbstractAggregationBuilder> aggregations;
|
||||||
|
private BytesReference aggregationsBinary;
|
||||||
|
|
||||||
|
protected AggregationBuilder(String name, String type) {
|
||||||
|
super(name, type);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a sub get to this bucket get.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B subAggregation(AbstractAggregationBuilder aggregation) {
|
||||||
|
if (aggregations == null) {
|
||||||
|
aggregations = Lists.newArrayList();
|
||||||
|
}
|
||||||
|
aggregations.add(aggregation);
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent / json) sub addAggregation.
|
||||||
|
*/
|
||||||
|
public B subAggregation(byte[] aggregationsBinary) {
|
||||||
|
return subAggregation(aggregationsBinary, 0, aggregationsBinary.length);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent / json) sub addAggregation.
|
||||||
|
*/
|
||||||
|
public B subAggregation(byte[] aggregationsBinary, int aggregationsBinaryOffset, int aggregationsBinaryLength) {
|
||||||
|
return subAggregation(new BytesArray(aggregationsBinary, aggregationsBinaryOffset, aggregationsBinaryLength));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent / json) sub addAggregation.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B subAggregation(BytesReference aggregationsBinary) {
|
||||||
|
this.aggregationsBinary = aggregationsBinary;
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent / json) sub addAggregation.
|
||||||
|
*/
|
||||||
|
public B subAggregation(XContentBuilder facets) {
|
||||||
|
return subAggregation(facets.bytes());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets a raw (xcontent / json) sub addAggregation.
|
||||||
|
*/
|
||||||
|
public B subAggregation(Map<String, Object> facets) {
|
||||||
|
try {
|
||||||
|
XContentBuilder builder = XContentFactory.contentBuilder(Requests.CONTENT_TYPE);
|
||||||
|
builder.map(facets);
|
||||||
|
return subAggregation(builder);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new ElasticSearchGenerationException("Failed to generate [" + facets + "]", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject(name);
|
||||||
|
|
||||||
|
builder.field(type);
|
||||||
|
internalXContent(builder, params);
|
||||||
|
|
||||||
|
if (aggregations != null || aggregationsBinary != null) {
|
||||||
|
builder.startObject("aggregations");
|
||||||
|
|
||||||
|
if (aggregations != null) {
|
||||||
|
for (AbstractAggregationBuilder subAgg : aggregations) {
|
||||||
|
subAgg.toXContent(builder, params);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (aggregationsBinary != null) {
|
||||||
|
if (XContentFactory.xContentType(aggregationsBinary) == builder.contentType()) {
|
||||||
|
builder.rawField("aggregations", aggregationsBinary);
|
||||||
|
} else {
|
||||||
|
builder.field("aggregations_binary", aggregationsBinary);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
return builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException;
|
||||||
|
}
|
|
@ -0,0 +1,101 @@
|
||||||
|
package org.elasticsearch.search.aggregations;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.RangeBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.date.DateRangeBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.ipv4.IPv4RangeBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.TermsBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.global.GlobalBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.missing.MissingBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.nested.NestedBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.avg.AvgBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.max.MaxBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.min.MinBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.StatsBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.sum.SumBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class AggregationBuilders {
|
||||||
|
|
||||||
|
protected AggregationBuilders() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ValueCountBuilder count(String name) {
|
||||||
|
return new ValueCountBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static AvgBuilder avg(String name) {
|
||||||
|
return new AvgBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MaxBuilder max(String name) {
|
||||||
|
return new MaxBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MinBuilder min(String name) {
|
||||||
|
return new MinBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static SumBuilder sum(String name) {
|
||||||
|
return new SumBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static StatsBuilder stats(String name) {
|
||||||
|
return new StatsBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ExtendedStatsBuilder extendedStats(String name) {
|
||||||
|
return new ExtendedStatsBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static FilterAggregationBuilder filter(String name) {
|
||||||
|
return new FilterAggregationBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static GlobalBuilder global(String name) {
|
||||||
|
return new GlobalBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MissingBuilder missing(String name) {
|
||||||
|
return new MissingBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static NestedBuilder nested(String name) {
|
||||||
|
return new NestedBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static GeoDistanceBuilder geoDistance(String name) {
|
||||||
|
return new GeoDistanceBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HistogramBuilder histogram(String name) {
|
||||||
|
return new HistogramBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DateHistogramBuilder dateHistogram(String name) {
|
||||||
|
return new DateHistogramBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static RangeBuilder range(String name) {
|
||||||
|
return new RangeBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DateRangeBuilder dateRange(String name) {
|
||||||
|
return new DateRangeBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static IPv4RangeBuilder ipRange(String name) {
|
||||||
|
return new IPv4RangeBuilder(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static TermsBuilder terms(String name) {
|
||||||
|
return new TermsBuilder(name);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticSearchException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown when failing to execute an aggregation
|
||||||
|
*/
|
||||||
|
public class AggregationExecutionException extends ElasticSearchException {
|
||||||
|
|
||||||
|
public AggregationExecutionException(String msg) {
|
||||||
|
super(msg);
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregationExecutionException(String msg, Throwable cause) {
|
||||||
|
super(msg, cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,36 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticSearchException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown when failing to execute an aggregation
|
||||||
|
*/
|
||||||
|
public class AggregationInitializationException extends ElasticSearchException {
|
||||||
|
|
||||||
|
public AggregationInitializationException(String msg) {
|
||||||
|
super(msg);
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregationInitializationException(String msg, Throwable cause) {
|
||||||
|
super(msg, cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,95 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.elasticsearch.common.inject.AbstractModule;
|
||||||
|
import org.elasticsearch.common.inject.multibindings.Multibinder;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.HistogramParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.RangeParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.date.DateRangeParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.geodistance.GeoDistanceParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.ipv4.IpRangeParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.TermsParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.filter.FilterParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.global.GlobalParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.missing.MissingParser;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.nested.NestedParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.avg.AvgParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.max.MaxParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.min.MinParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.StatsParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStatsParser;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.sum.SumParser;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The main module for the get (binding all get components together)
|
||||||
|
*/
|
||||||
|
public class AggregationModule extends AbstractModule {
|
||||||
|
|
||||||
|
private List<Class<? extends Aggregator.Parser>> parsers = Lists.newArrayList();
|
||||||
|
|
||||||
|
public AggregationModule() {
|
||||||
|
parsers.add(AvgParser.class);
|
||||||
|
parsers.add(SumParser.class);
|
||||||
|
parsers.add(MinParser.class);
|
||||||
|
parsers.add(MaxParser.class);
|
||||||
|
parsers.add(StatsParser.class);
|
||||||
|
parsers.add(ExtendedStatsParser.class);
|
||||||
|
parsers.add(ValueCountParser.class);
|
||||||
|
|
||||||
|
parsers.add(GlobalParser.class);
|
||||||
|
parsers.add(MissingParser.class);
|
||||||
|
parsers.add(FilterParser.class);
|
||||||
|
parsers.add(TermsParser.class);
|
||||||
|
parsers.add(RangeParser.class);
|
||||||
|
parsers.add(DateRangeParser.class);
|
||||||
|
parsers.add(IpRangeParser.class);
|
||||||
|
parsers.add(HistogramParser.class);
|
||||||
|
parsers.add(DateHistogramParser.class);
|
||||||
|
parsers.add(GeoDistanceParser.class);
|
||||||
|
parsers.add(NestedParser.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Enabling extending the get module by adding a custom aggregation parser.
|
||||||
|
*
|
||||||
|
* @param parser The parser for the custom aggregator.
|
||||||
|
*/
|
||||||
|
public void addAggregatorParser(Class<? extends Aggregator.Parser> parser) {
|
||||||
|
parsers.add(parser);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configure() {
|
||||||
|
Multibinder<Aggregator.Parser> multibinder = Multibinder.newSetBinder(binder(), Aggregator.Parser.class);
|
||||||
|
for (Class<? extends Aggregator.Parser> parser : parsers) {
|
||||||
|
multibinder.addBinding().to(parser);
|
||||||
|
}
|
||||||
|
bind(AggregatorParsers.class).asEagerSingleton();
|
||||||
|
bind(AggregationParseElement.class).asEagerSingleton();
|
||||||
|
bind(AggregationPhase.class).asEagerSingleton();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,64 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.SearchParseElement;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The search parse element that is responsible for parsing the get part of the request.
|
||||||
|
*
|
||||||
|
* For example (in bold):
|
||||||
|
* <pre>
|
||||||
|
* curl -XGET 'localhost:9200/_search?search_type=count' -d '{
|
||||||
|
* query: {
|
||||||
|
* match_all : {}
|
||||||
|
* },
|
||||||
|
* addAggregation : {
|
||||||
|
* avg_price: {
|
||||||
|
* avg : { field : price }
|
||||||
|
* },
|
||||||
|
* categories: {
|
||||||
|
* terms : { field : category, size : 12 },
|
||||||
|
* addAggregation: {
|
||||||
|
* avg_price : { avg : { field : price }}
|
||||||
|
* }
|
||||||
|
* }
|
||||||
|
* }
|
||||||
|
* }'
|
||||||
|
* </pre>
|
||||||
|
*/
|
||||||
|
public class AggregationParseElement implements SearchParseElement {
|
||||||
|
|
||||||
|
private final AggregatorParsers aggregatorParsers;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public AggregationParseElement(AggregatorParsers aggregatorParsers) {
|
||||||
|
this.aggregatorParsers = aggregatorParsers;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void parse(XContentParser parser, SearchContext context) throws Exception {
|
||||||
|
AggregatorFactories factories = aggregatorParsers.parseAggregators(parser, context);
|
||||||
|
context.aggregations(new SearchContextAggregations(factories));
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,178 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.apache.lucene.index.AtomicReaderContext;
|
||||||
|
import org.apache.lucene.search.Filter;
|
||||||
|
import org.apache.lucene.search.Query;
|
||||||
|
import org.apache.lucene.search.Scorer;
|
||||||
|
import org.elasticsearch.ElasticSearchException;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.lucene.search.Queries;
|
||||||
|
import org.elasticsearch.common.lucene.search.XCollector;
|
||||||
|
import org.elasticsearch.common.lucene.search.XConstantScoreQuery;
|
||||||
|
import org.elasticsearch.common.lucene.search.XFilteredQuery;
|
||||||
|
import org.elasticsearch.search.SearchParseElement;
|
||||||
|
import org.elasticsearch.search.SearchPhase;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
import org.elasticsearch.search.query.QueryPhaseExecutionException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class AggregationPhase implements SearchPhase {
|
||||||
|
|
||||||
|
private final AggregationParseElement parseElement;
|
||||||
|
|
||||||
|
private final AggregationBinaryParseElement binaryParseElement;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public AggregationPhase(AggregationParseElement parseElement, AggregationBinaryParseElement binaryParseElement) {
|
||||||
|
this.parseElement = parseElement;
|
||||||
|
this.binaryParseElement = binaryParseElement;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, ? extends SearchParseElement> parseElements() {
|
||||||
|
return ImmutableMap.<String, SearchParseElement>builder()
|
||||||
|
.put("aggregations", parseElement)
|
||||||
|
.put("aggs", parseElement)
|
||||||
|
.put("aggregations_binary", binaryParseElement)
|
||||||
|
.put("aggregationsBinary", binaryParseElement)
|
||||||
|
.put("aggs_binary", binaryParseElement)
|
||||||
|
.put("aggsBinary", binaryParseElement)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void preProcess(SearchContext context) {
|
||||||
|
if (context.aggregations() != null) {
|
||||||
|
AggregationContext aggregationContext = new AggregationContext(context);
|
||||||
|
context.aggregations().aggregationContext(aggregationContext);
|
||||||
|
|
||||||
|
List<Aggregator> collectors = new ArrayList<Aggregator>();
|
||||||
|
Aggregator[] aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext);
|
||||||
|
for (int i = 0; i < aggregators.length; i++) {
|
||||||
|
if (!(aggregators[i] instanceof GlobalAggregator)) {
|
||||||
|
Aggregator aggregator = aggregators[i];
|
||||||
|
if (aggregator.shouldCollect()) {
|
||||||
|
collectors.add(aggregator);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
context.aggregations().aggregators(aggregators);
|
||||||
|
if (!collectors.isEmpty()) {
|
||||||
|
context.searcher().addMainQueryCollector(new AggregationsCollector(collectors, aggregationContext));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void execute(SearchContext context) throws ElasticSearchException {
|
||||||
|
if (context.aggregations() == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (context.queryResult().aggregations() != null) {
|
||||||
|
// no need to compute the facets twice, they should be computed on a per context basis
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Aggregator[] aggregators = context.aggregations().aggregators();
|
||||||
|
List<Aggregator> globals = new ArrayList<Aggregator>();
|
||||||
|
for (int i = 0; i < aggregators.length; i++) {
|
||||||
|
if (aggregators[i] instanceof GlobalAggregator) {
|
||||||
|
globals.add(aggregators[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// optimize the global collector based execution
|
||||||
|
if (!globals.isEmpty()) {
|
||||||
|
AggregationsCollector collector = new AggregationsCollector(globals, context.aggregations().aggregationContext());
|
||||||
|
Query query = new XConstantScoreQuery(Queries.MATCH_ALL_FILTER);
|
||||||
|
Filter searchFilter = context.searchFilter(context.types());
|
||||||
|
if (searchFilter != null) {
|
||||||
|
query = new XFilteredQuery(query, searchFilter);
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
context.searcher().search(query, collector);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new QueryPhaseExecutionException(context, "Failed to execute global aggregators", e);
|
||||||
|
}
|
||||||
|
collector.postCollection();
|
||||||
|
}
|
||||||
|
|
||||||
|
List<InternalAggregation> aggregations = new ArrayList<InternalAggregation>(aggregators.length);
|
||||||
|
for (Aggregator aggregator : context.aggregations().aggregators()) {
|
||||||
|
aggregations.add(aggregator.buildAggregation(0));
|
||||||
|
}
|
||||||
|
context.queryResult().aggregations(new InternalAggregations(aggregations));
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
static class AggregationsCollector extends XCollector {
|
||||||
|
|
||||||
|
private final AggregationContext aggregationContext;
|
||||||
|
private final List<Aggregator> collectors;
|
||||||
|
|
||||||
|
AggregationsCollector(List<Aggregator> collectors, AggregationContext aggregationContext) {
|
||||||
|
this.collectors = collectors;
|
||||||
|
this.aggregationContext = aggregationContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setScorer(Scorer scorer) throws IOException {
|
||||||
|
aggregationContext.setScorer(scorer);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc) throws IOException {
|
||||||
|
for (int i = 0; i < collectors.size(); i++) {
|
||||||
|
collectors.get(i).collect(doc, 0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setNextReader(AtomicReaderContext context) throws IOException {
|
||||||
|
aggregationContext.setNextReader(context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean acceptsDocsOutOfOrder() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void postCollection() {
|
||||||
|
for (int i = 0; i < collectors.size(); i++) {
|
||||||
|
collectors.get(i).postCollection();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,68 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.collect.MapBuilder;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A registry for all the dedicated streams in the aggregation module. This is to support dynamic addAggregation that
|
||||||
|
* know how to stream themselves.
|
||||||
|
*/
|
||||||
|
public class AggregationStreams {
|
||||||
|
|
||||||
|
private static ImmutableMap<BytesReference, Stream> streams = ImmutableMap.of();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A stream that knows how to read an aggregation from the input.
|
||||||
|
*/
|
||||||
|
public static interface Stream {
|
||||||
|
InternalAggregation readResult(StreamInput in) throws IOException;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Registers the given stream and associate it with the given types.
|
||||||
|
*
|
||||||
|
* @param stream The streams to register
|
||||||
|
* @param types The types associated with the streams
|
||||||
|
*/
|
||||||
|
public static synchronized void registerStream(Stream stream, BytesReference... types) {
|
||||||
|
MapBuilder<BytesReference, Stream> uStreams = MapBuilder.newMapBuilder(streams);
|
||||||
|
for (BytesReference type : types) {
|
||||||
|
uStreams.put(type, stream);
|
||||||
|
}
|
||||||
|
streams = uStreams.immutableMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the stream that is registered for the given type
|
||||||
|
*
|
||||||
|
* @param type The given type
|
||||||
|
* @return The associated stream
|
||||||
|
*/
|
||||||
|
public static Stream stream(BytesReference type) {
|
||||||
|
return streams.get(type);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Represents a set of computed addAggregation.
|
||||||
|
*/
|
||||||
|
public interface Aggregations extends Iterable<Aggregation> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The list of {@link Aggregation}s.
|
||||||
|
*/
|
||||||
|
List<Aggregation> asList();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the {@link Aggregation}s keyed by aggregation name.
|
||||||
|
*/
|
||||||
|
Map<String, Aggregation> asMap();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the {@link Aggregation}s keyed by aggregation name.
|
||||||
|
*/
|
||||||
|
Map<String, Aggregation> getAsMap();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the aggregation that is associated with the specified name.
|
||||||
|
*/
|
||||||
|
<A extends Aggregation> A get(String name);
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,198 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class Aggregator {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Defines the nature of the aggregator's aggregation execution when nested in other aggregators and the buckets they create.
|
||||||
|
*/
|
||||||
|
public static enum BucketAggregationMode {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In this mode, a new aggregator instance will be created per bucket (created by the parent aggregator)
|
||||||
|
*/
|
||||||
|
PER_BUCKET,
|
||||||
|
|
||||||
|
/**
|
||||||
|
* In this mode, a single aggregator instance will be created per parent aggregator, that will handle the aggregations of all its buckets.
|
||||||
|
*/
|
||||||
|
MULTI_BUCKETS
|
||||||
|
}
|
||||||
|
|
||||||
|
protected final String name;
|
||||||
|
protected final Aggregator parent;
|
||||||
|
protected final AggregationContext context;
|
||||||
|
protected final int depth;
|
||||||
|
protected final long estimatedBucketCount;
|
||||||
|
|
||||||
|
protected final BucketAggregationMode bucketAggregationMode;
|
||||||
|
protected final AggregatorFactories factories;
|
||||||
|
protected final Aggregator[] subAggregators;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a new Aggregator.
|
||||||
|
*
|
||||||
|
* @param name The name of the aggregation
|
||||||
|
* @param bucketAggregationMode The nature of execution as a sub-aggregator (see {@link BucketAggregationMode})
|
||||||
|
* @param factories The factories for all the sub-aggregators under this aggregator
|
||||||
|
* @param estimatedBucketsCount When served as a sub-aggregator, indicate how many buckets the parent aggregator will generate.
|
||||||
|
* @param context The aggregation context
|
||||||
|
* @param parent The parent aggregator (may be {@code null} for top level aggregators)
|
||||||
|
*/
|
||||||
|
protected Aggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories, long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||||
|
this.name = name;
|
||||||
|
this.parent = parent;
|
||||||
|
this.estimatedBucketCount = estimatedBucketsCount;
|
||||||
|
this.context = context;
|
||||||
|
this.depth = parent == null ? 0 : 1 + parent.depth();
|
||||||
|
this.bucketAggregationMode = bucketAggregationMode;
|
||||||
|
assert factories != null : "sub-factories provided to BucketAggregator must not be null, use AggragatorFactories.EMPTY instead";
|
||||||
|
this.factories = factories;
|
||||||
|
this.subAggregators = factories.createSubAggregators(this, estimatedBucketsCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The name of the aggregation.
|
||||||
|
*/
|
||||||
|
public String name() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return the estimated number of buckets. */
|
||||||
|
public final long estimatedBucketCount() {
|
||||||
|
return estimatedBucketCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return the depth of this aggregator in the aggregation tree. */
|
||||||
|
public final int depth() {
|
||||||
|
return depth;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The parent aggregator of this aggregator. The addAggregation are hierarchical in the sense that some can
|
||||||
|
* be composed out of others (more specifically, bucket addAggregation can define other addAggregation that will
|
||||||
|
* be aggregated per bucket). This method returns the direct parent aggregator that contains this aggregator, or
|
||||||
|
* {@code null} if there is none (meaning, this aggregator is a top level one)
|
||||||
|
*/
|
||||||
|
public Aggregator parent() {
|
||||||
|
return parent;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The current aggregation context.
|
||||||
|
*/
|
||||||
|
public AggregationContext context() {
|
||||||
|
return context;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The bucket aggregation mode of this aggregator. This mode defines the nature in which the aggregation is executed
|
||||||
|
* @see BucketAggregationMode
|
||||||
|
*/
|
||||||
|
public BucketAggregationMode bucketAggregationMode() {
|
||||||
|
return bucketAggregationMode;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return Whether this aggregator is in the state where it can collect documents. Some aggregators can do their aggregations without
|
||||||
|
* actually collecting documents, for example, an aggregator that computes stats over unmapped fields doesn't need to collect
|
||||||
|
* anything as it knows to just return "empty" stats as the aggregation result.
|
||||||
|
*/
|
||||||
|
public abstract boolean shouldCollect();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called during the query phase, to collect & aggregate the given document.
|
||||||
|
*
|
||||||
|
* @param doc The document to be collected/aggregated
|
||||||
|
* @param owningBucketOrdinal The ordinal of the bucket this aggregator belongs to, assuming this aggregator is not a top level aggregator.
|
||||||
|
* Typically, aggregators with {@code #bucketAggregationMode} set to {@link BucketAggregationMode#MULTI_BUCKETS}
|
||||||
|
* will heavily depend on this ordinal. Other aggregators may or may not use it and can see this ordinal as just
|
||||||
|
* an extra information for the aggregation context. For top level aggregators, the ordinal will always be
|
||||||
|
* equal to 0.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public abstract void collect(int doc, long owningBucketOrdinal) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called after collection of all document is done.
|
||||||
|
*/
|
||||||
|
public final void postCollection() {
|
||||||
|
for (int i = 0; i < subAggregators.length; i++) {
|
||||||
|
subAggregators[i].postCollection();
|
||||||
|
}
|
||||||
|
doPostCollection();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Can be overriden by aggregator implementation to be called back when the collection phase ends.
|
||||||
|
*/
|
||||||
|
protected void doPostCollection() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The aggregated & built aggregation
|
||||||
|
*/
|
||||||
|
public abstract InternalAggregation buildAggregation(long owningBucketOrdinal);
|
||||||
|
|
||||||
|
public abstract InternalAggregation buildEmptyAggregation();
|
||||||
|
|
||||||
|
protected final InternalAggregations buildEmptySubAggregations() {
|
||||||
|
List<InternalAggregation> aggs = new ArrayList<InternalAggregation>();
|
||||||
|
for (Aggregator aggregator : subAggregators) {
|
||||||
|
aggs.add(aggregator.buildEmptyAggregation());
|
||||||
|
}
|
||||||
|
return new InternalAggregations(aggs);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parses the aggregation request and creates the appropriate aggregator factory for it.
|
||||||
|
*
|
||||||
|
* @see {@link AggregatorFactory}
|
||||||
|
*/
|
||||||
|
public static interface Parser {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The aggregation type this parser is associated with.
|
||||||
|
*/
|
||||||
|
String type();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the aggregator factory with which this parser is associated, may return {@code null} indicating the
|
||||||
|
* aggregation should be skipped (e.g. when trying to aggregate on unmapped fields).
|
||||||
|
*
|
||||||
|
* @param aggregationName The name of the aggregation
|
||||||
|
* @param parser The xcontent parser
|
||||||
|
* @param context The search context
|
||||||
|
* @return The resolved aggregator factory or {@code null} in case the aggregation should be skipped
|
||||||
|
* @throws java.io.IOException When parsing fails
|
||||||
|
*/
|
||||||
|
AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException;
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,176 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.ObjectArray;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator.BucketAggregationMode;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class AggregatorFactories {
|
||||||
|
|
||||||
|
public static final AggregatorFactories EMPTY = new Empty();
|
||||||
|
|
||||||
|
private final AggregatorFactory[] factories;
|
||||||
|
|
||||||
|
public static Builder builder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
private AggregatorFactories(AggregatorFactory[] factories) {
|
||||||
|
this.factories = factories;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create all aggregators so that they can be consumed with multiple buckets.
|
||||||
|
*/
|
||||||
|
public Aggregator[] createSubAggregators(Aggregator parent, final long estimatedBucketsCount) {
|
||||||
|
Aggregator[] aggregators = new Aggregator[count()];
|
||||||
|
for (int i = 0; i < factories.length; ++i) {
|
||||||
|
final AggregatorFactory factory = factories[i];
|
||||||
|
final Aggregator first = factory.create(parent.context(), parent, estimatedBucketsCount);
|
||||||
|
if (first.bucketAggregationMode() == BucketAggregationMode.MULTI_BUCKETS) {
|
||||||
|
// This aggregator already supports multiple bucket ordinals, can be used directly
|
||||||
|
aggregators[i] = first;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
// the aggregator doesn't support multiple ordinals, let's wrap it so that it does.
|
||||||
|
aggregators[i] = new Aggregator(first.name(), BucketAggregationMode.MULTI_BUCKETS, AggregatorFactories.EMPTY, 1, first.context(), first.parent()) {
|
||||||
|
|
||||||
|
ObjectArray<Aggregator> aggregators;
|
||||||
|
|
||||||
|
{
|
||||||
|
aggregators = BigArrays.newObjectArray(estimatedBucketsCount);
|
||||||
|
aggregators.set(0, first);
|
||||||
|
for (long i = 1; i < estimatedBucketsCount; ++i) {
|
||||||
|
aggregators.set(i, factory.create(parent.context(), parent, estimatedBucketsCount));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldCollect() {
|
||||||
|
return first.shouldCollect();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void doPostCollection() {
|
||||||
|
for (long i = 0; i < aggregators.size(); ++i) {
|
||||||
|
final Aggregator aggregator = aggregators.get(i);
|
||||||
|
if (aggregator != null) {
|
||||||
|
aggregator.postCollection();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, long owningBucketOrdinal) throws IOException {
|
||||||
|
aggregators = BigArrays.grow(aggregators, owningBucketOrdinal + 1);
|
||||||
|
Aggregator aggregator = aggregators.get(owningBucketOrdinal);
|
||||||
|
if (aggregator == null) {
|
||||||
|
aggregator = factory.create(parent.context(), parent, estimatedBucketsCount);
|
||||||
|
aggregators.set(owningBucketOrdinal, aggregator);
|
||||||
|
}
|
||||||
|
aggregator.collect(doc, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||||
|
return aggregators.get(owningBucketOrdinal).buildAggregation(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildEmptyAggregation() {
|
||||||
|
return first.buildEmptyAggregation();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Aggregator[] createTopLevelAggregators(AggregationContext ctx) {
|
||||||
|
// These aggregators are going to be used with a single bucket ordinal, no need to wrap the PER_BUCKET ones
|
||||||
|
Aggregator[] aggregators = new Aggregator[factories.length];
|
||||||
|
for (int i = 0; i < factories.length; i++) {
|
||||||
|
aggregators[i] = factories[i].create(ctx, null, 0);
|
||||||
|
}
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int count() {
|
||||||
|
return factories.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
void setParent(AggregatorFactory parent) {
|
||||||
|
for (AggregatorFactory factory : factories) {
|
||||||
|
factory.parent = parent;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void validate() {
|
||||||
|
for (AggregatorFactory factory : factories) {
|
||||||
|
factory.validate();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final static class Empty extends AggregatorFactories {
|
||||||
|
|
||||||
|
private static final AggregatorFactory[] EMPTY_FACTORIES = new AggregatorFactory[0];
|
||||||
|
private static final Aggregator[] EMPTY_AGGREGATORS = new Aggregator[0];
|
||||||
|
|
||||||
|
private Empty() {
|
||||||
|
super(EMPTY_FACTORIES);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator[] createSubAggregators(Aggregator parent, long estimatedBucketsCount) {
|
||||||
|
return EMPTY_AGGREGATORS;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator[] createTopLevelAggregators(AggregationContext ctx) {
|
||||||
|
return EMPTY_AGGREGATORS;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private List<AggregatorFactory> factories = new ArrayList<AggregatorFactory>();
|
||||||
|
|
||||||
|
public Builder add(AggregatorFactory factory) {
|
||||||
|
factories.add(factory);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregatorFactories build() {
|
||||||
|
if (factories.isEmpty()) {
|
||||||
|
return EMPTY;
|
||||||
|
}
|
||||||
|
return new AggregatorFactories(factories.toArray(new AggregatorFactory[factories.size()]));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,88 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A factory that knows how to create an {@link Aggregator} of a specific type.
|
||||||
|
*/
|
||||||
|
public abstract class AggregatorFactory {
|
||||||
|
|
||||||
|
protected String name;
|
||||||
|
protected String type;
|
||||||
|
protected AggregatorFactory parent;
|
||||||
|
protected AggregatorFactories factories = AggregatorFactories.EMPTY;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a new aggregator factory.
|
||||||
|
*
|
||||||
|
* @param name The aggregation name
|
||||||
|
* @param type The aggregation type
|
||||||
|
*/
|
||||||
|
public AggregatorFactory(String name, String type) {
|
||||||
|
this.name = name;
|
||||||
|
this.type = type;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Registers sub-factories with this factory. The sub-factory will be responsible for the creation of sub-aggregators under the
|
||||||
|
* aggregator created by this factory.
|
||||||
|
*
|
||||||
|
* @param subFactories The sub-factories
|
||||||
|
* @return this factory (fluent interface)
|
||||||
|
*/
|
||||||
|
public AggregatorFactory subFactories(AggregatorFactories subFactories) {
|
||||||
|
this.factories = subFactories;
|
||||||
|
this.factories.setParent(this);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Validates the state of this factory (makes sure the factory is properly configured)
|
||||||
|
*/
|
||||||
|
public final void validate() {
|
||||||
|
doValidate();
|
||||||
|
factories.validate();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The parent factory if one exists (will always return {@code null} for top level aggregator factories).
|
||||||
|
*/
|
||||||
|
public AggregatorFactory parent() {
|
||||||
|
return parent;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates the aggregator
|
||||||
|
*
|
||||||
|
* @param context The aggregation context
|
||||||
|
* @param parent The parent aggregator (if this is a top level factory, the parent will be {@code null})
|
||||||
|
* @param expectedBucketsCount If this is a sub-factory of another factory, this will indicate the number of bucket the parent aggregator
|
||||||
|
* may generate (this is an estimation only). For top level factories, this will always be 0
|
||||||
|
*
|
||||||
|
* @return The created aggregator
|
||||||
|
*/
|
||||||
|
public abstract Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount);
|
||||||
|
|
||||||
|
public void doValidate() {
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.elasticsearch.common.collect.MapBuilder;
|
||||||
|
import org.elasticsearch.common.inject.Inject;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.SearchParseException;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A registry for all the aggregator parser, also servers as the main parser for the aggregations module
|
||||||
|
*/
|
||||||
|
public class AggregatorParsers {
|
||||||
|
|
||||||
|
private final ImmutableMap<String, Aggregator.Parser> parsers;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs the AggregatorParsers out of all the given parsers
|
||||||
|
*
|
||||||
|
* @param parsers The available aggregator parsers (dynamically injected by the {@link org.elasticsearch.search.aggregations.AggregationModule}).
|
||||||
|
*/
|
||||||
|
@Inject
|
||||||
|
public AggregatorParsers(Set<Aggregator.Parser> parsers) {
|
||||||
|
MapBuilder<String, Aggregator.Parser> builder = MapBuilder.newMapBuilder();
|
||||||
|
for (Aggregator.Parser parser : parsers) {
|
||||||
|
builder.put(parser.type(), parser);
|
||||||
|
}
|
||||||
|
this.parsers = builder.immutableMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the parser that is registered under the given aggregation type.
|
||||||
|
*
|
||||||
|
* @param type The aggregation type
|
||||||
|
* @return The parser associated with the given aggregation type.
|
||||||
|
*/
|
||||||
|
public Aggregator.Parser parser(String type) {
|
||||||
|
return parsers.get(type);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parses the aggregation request recursively generating aggregator factories in turn.
|
||||||
|
*
|
||||||
|
* @param parser The input xcontent that will be parsed.
|
||||||
|
* @param context The search context.
|
||||||
|
*
|
||||||
|
* @return The parsed aggregator factories.
|
||||||
|
*
|
||||||
|
* @throws IOException When parsing fails for unknown reasons.
|
||||||
|
*/
|
||||||
|
public AggregatorFactories parseAggregators(XContentParser parser, SearchContext context) throws IOException {
|
||||||
|
return parseAggregators(parser, context, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private AggregatorFactories parseAggregators(XContentParser parser, SearchContext context, int level) throws IOException {
|
||||||
|
XContentParser.Token token = null;
|
||||||
|
String currentFieldName = null;
|
||||||
|
|
||||||
|
AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
|
||||||
|
|
||||||
|
String aggregationName = null;
|
||||||
|
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||||
|
if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
|
aggregationName = parser.currentName();
|
||||||
|
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||||
|
String aggregatorType = null;
|
||||||
|
AggregatorFactory factory = null;
|
||||||
|
AggregatorFactories subFactories = null;
|
||||||
|
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||||
|
if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
|
currentFieldName = parser.currentName();
|
||||||
|
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||||
|
if ("aggregations".equals(currentFieldName) || "aggs".equals(currentFieldName)) {
|
||||||
|
subFactories = parseAggregators(parser, context, level+1);
|
||||||
|
} else {
|
||||||
|
aggregatorType = currentFieldName;
|
||||||
|
Aggregator.Parser aggregatorParser = parser(aggregatorType);
|
||||||
|
if (aggregatorParser == null) {
|
||||||
|
throw new SearchParseException(context, "Could not find aggregator type [" + currentFieldName + "]");
|
||||||
|
}
|
||||||
|
factory = aggregatorParser.parse(aggregationName, parser, context);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (factory == null) {
|
||||||
|
// skipping the aggregation
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (subFactories != null) {
|
||||||
|
factory.subFactories(subFactories);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (level == 0) {
|
||||||
|
factory.validate();
|
||||||
|
}
|
||||||
|
|
||||||
|
factories.add(factory);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return factories.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,146 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.cache.recycler.CacheRecycler;
|
||||||
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.Streamable;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An internal implementation of {@link Aggregation}. Serves as a base class for all aggregation implementations.
|
||||||
|
*/
|
||||||
|
public abstract class InternalAggregation implements Aggregation, ToXContent, Streamable {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The aggregation type that holds all the string types that are associated with an aggregation:
|
||||||
|
* <ul>
|
||||||
|
* <li>name - used as the parser type</li>
|
||||||
|
* <li>stream - used as the stream type</li>
|
||||||
|
* </ul>
|
||||||
|
*/
|
||||||
|
public static class Type {
|
||||||
|
|
||||||
|
private String name;
|
||||||
|
private BytesReference stream;
|
||||||
|
|
||||||
|
public Type(String name) {
|
||||||
|
this(name, new BytesArray(name));
|
||||||
|
}
|
||||||
|
|
||||||
|
public Type(String name, String stream) {
|
||||||
|
this(name, new BytesArray(stream));
|
||||||
|
}
|
||||||
|
|
||||||
|
public Type(String name, BytesReference stream) {
|
||||||
|
this.name = name;
|
||||||
|
this.stream = stream;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The name of the type (mainly used for registering the parser for the aggregator (see {@link org.elasticsearch.search.aggregations.Aggregator.Parser#type()}).
|
||||||
|
*/
|
||||||
|
public String name() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The name of the stream type (used for registering the aggregation stream
|
||||||
|
* (see {@link AggregationStreams#registerStream(AggregationStreams.Stream, org.elasticsearch.common.bytes.BytesReference...)}).
|
||||||
|
*/
|
||||||
|
public BytesReference stream() {
|
||||||
|
return stream;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static class ReduceContext {
|
||||||
|
|
||||||
|
private final List<InternalAggregation> aggregations;
|
||||||
|
private final CacheRecycler cacheRecycler;
|
||||||
|
|
||||||
|
public ReduceContext(List<InternalAggregation> aggregations, CacheRecycler cacheRecycler) {
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
this.cacheRecycler = cacheRecycler;
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<InternalAggregation> aggregations() {
|
||||||
|
return aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
public CacheRecycler cacheRecycler() {
|
||||||
|
return cacheRecycler;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected String name;
|
||||||
|
|
||||||
|
/** Constructs an un initialized addAggregation (used for serialization) **/
|
||||||
|
protected InternalAggregation() {}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs an get with a given name.
|
||||||
|
*
|
||||||
|
* @param name The name of the get.
|
||||||
|
*/
|
||||||
|
protected InternalAggregation(String name) {
|
||||||
|
this.name = name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getName() {
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The {@link Type} of this aggregation
|
||||||
|
*/
|
||||||
|
public abstract Type type();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reduces the given addAggregation to a single one and returns it. In <b>most</b> cases, the assumption will be the all given
|
||||||
|
* addAggregation are of the same type (the same type as this aggregation). For best efficiency, when implementing,
|
||||||
|
* try reusing an existing get instance (typically the first in the given list) to save on redundant object
|
||||||
|
* construction.
|
||||||
|
*/
|
||||||
|
public abstract InternalAggregation reduce(ReduceContext reduceContext);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Common xcontent fields that are shared among addAggregation
|
||||||
|
*/
|
||||||
|
public static final class CommonFields {
|
||||||
|
public static final XContentBuilderString BUCKETS = new XContentBuilderString("buckets");
|
||||||
|
public static final XContentBuilderString VALUE = new XContentBuilderString("value");
|
||||||
|
public static final XContentBuilderString VALUE_AS_STRING = new XContentBuilderString("value_as_string");
|
||||||
|
public static final XContentBuilderString DOC_COUNT = new XContentBuilderString("doc_count");
|
||||||
|
public static final XContentBuilderString KEY = new XContentBuilderString("key");
|
||||||
|
public static final XContentBuilderString KEY_AS_STRING = new XContentBuilderString("key_as_string");
|
||||||
|
public static final XContentBuilderString FROM = new XContentBuilderString("from");
|
||||||
|
public static final XContentBuilderString FROM_AS_STRING = new XContentBuilderString("from_as_string");
|
||||||
|
public static final XContentBuilderString TO = new XContentBuilderString("to");
|
||||||
|
public static final XContentBuilderString TO_AS_STRING = new XContentBuilderString("to_as_string");
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,211 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import com.google.common.base.Function;
|
||||||
|
import com.google.common.collect.*;
|
||||||
|
import org.elasticsearch.cache.recycler.CacheRecycler;
|
||||||
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Streamable;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilderString;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.*;
|
||||||
|
|
||||||
|
import static com.google.common.collect.Maps.newHashMap;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An internal implementation of {@link Aggregations}.
|
||||||
|
*/
|
||||||
|
public class InternalAggregations implements Aggregations, ToXContent, Streamable {
|
||||||
|
|
||||||
|
public final static InternalAggregations EMPTY = new InternalAggregations();
|
||||||
|
private static final Function<InternalAggregation, Aggregation> SUPERTYPE_CAST = new Function<InternalAggregation, Aggregation>() {
|
||||||
|
@Override
|
||||||
|
public Aggregation apply(InternalAggregation input) {
|
||||||
|
return input;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
private List<InternalAggregation> aggregations = ImmutableList.of();
|
||||||
|
|
||||||
|
private Map<String, InternalAggregation> aggregationsAsMap;
|
||||||
|
|
||||||
|
private InternalAggregations() {
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a new addAggregation.
|
||||||
|
*/
|
||||||
|
public InternalAggregations(List<InternalAggregation> aggregations) {
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Resets the internal addAggregation */
|
||||||
|
void reset(List<InternalAggregation> aggregations) {
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
this.aggregationsAsMap = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Iterates over the {@link Aggregation}s.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Iterator<Aggregation> iterator() {
|
||||||
|
return Iterators.transform(aggregations.iterator(), SUPERTYPE_CAST);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The list of {@link Aggregation}s.
|
||||||
|
*/
|
||||||
|
public List<Aggregation> asList() {
|
||||||
|
return Lists.transform(aggregations, SUPERTYPE_CAST);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the {@link Aggregation}s keyed by map.
|
||||||
|
*/
|
||||||
|
public Map<String, Aggregation> asMap() {
|
||||||
|
return getAsMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the {@link Aggregation}s keyed by map.
|
||||||
|
*/
|
||||||
|
public Map<String, Aggregation> getAsMap() {
|
||||||
|
if (aggregationsAsMap == null) {
|
||||||
|
Map<String, InternalAggregation> aggregationsAsMap = newHashMap();
|
||||||
|
for (InternalAggregation aggregation : aggregations) {
|
||||||
|
aggregationsAsMap.put(aggregation.getName(), aggregation);
|
||||||
|
}
|
||||||
|
this.aggregationsAsMap = aggregationsAsMap;
|
||||||
|
}
|
||||||
|
return Maps.transformValues(aggregationsAsMap, SUPERTYPE_CAST);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the aggregation of the specified name.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Override
|
||||||
|
public <A extends Aggregation> A get(String name) {
|
||||||
|
return (A) asMap().get(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reduces the given lists of addAggregation.
|
||||||
|
*
|
||||||
|
* @param aggregationsList A list of addAggregation to reduce
|
||||||
|
* @return The reduced addAggregation
|
||||||
|
*/
|
||||||
|
public static InternalAggregations reduce(List<InternalAggregations> aggregationsList, CacheRecycler cacheRecycler) {
|
||||||
|
if (aggregationsList.isEmpty()) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
// first we collect all addAggregation of the same type and list them together
|
||||||
|
|
||||||
|
Map<String, List<InternalAggregation>> aggByName = new HashMap<String, List<InternalAggregation>>();
|
||||||
|
for (InternalAggregations aggregations : aggregationsList) {
|
||||||
|
for (InternalAggregation aggregation : aggregations.aggregations) {
|
||||||
|
List<InternalAggregation> aggs = aggByName.get(aggregation.getName());
|
||||||
|
if (aggs == null) {
|
||||||
|
aggs = new ArrayList<InternalAggregation>(aggregationsList.size());
|
||||||
|
aggByName.put(aggregation.getName(), aggs);
|
||||||
|
}
|
||||||
|
aggs.add(aggregation);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// now we can use the first aggregation of each list to handle the reduce of its list
|
||||||
|
|
||||||
|
List<InternalAggregation> reducedAggregations = new ArrayList<InternalAggregation>();
|
||||||
|
for (Map.Entry<String, List<InternalAggregation>> entry : aggByName.entrySet()) {
|
||||||
|
List<InternalAggregation> aggregations = entry.getValue();
|
||||||
|
InternalAggregation first = aggregations.get(0); // the list can't be empty as it's created on demand
|
||||||
|
reducedAggregations.add(first.reduce(new InternalAggregation.ReduceContext(aggregations, cacheRecycler)));
|
||||||
|
}
|
||||||
|
InternalAggregations result = aggregationsList.get(0);
|
||||||
|
result.reset(reducedAggregations);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** The fields required to write this addAggregation to xcontent */
|
||||||
|
static class Fields {
|
||||||
|
public static final XContentBuilderString AGGREGATIONS = new XContentBuilderString("aggregations");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
if (aggregations.isEmpty()) {
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
builder.startObject(Fields.AGGREGATIONS);
|
||||||
|
toXContentInternal(builder, params);
|
||||||
|
return builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Directly write all the addAggregation without their bounding object. Used by sub-addAggregation (non top level addAggregation)
|
||||||
|
*/
|
||||||
|
public XContentBuilder toXContentInternal(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
for (Aggregation aggregation : aggregations) {
|
||||||
|
((InternalAggregation) aggregation).toXContent(builder, params);
|
||||||
|
}
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static InternalAggregations readAggregations(StreamInput in) throws IOException {
|
||||||
|
InternalAggregations result = new InternalAggregations();
|
||||||
|
result.readFrom(in);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
int size = in.readVInt();
|
||||||
|
if (size == 0) {
|
||||||
|
aggregations = ImmutableList.of();
|
||||||
|
aggregationsAsMap = ImmutableMap.of();
|
||||||
|
} else {
|
||||||
|
aggregations = Lists.newArrayListWithCapacity(size);
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
BytesReference type = in.readBytesReference();
|
||||||
|
InternalAggregation aggregation = AggregationStreams.stream(type).readResult(in);
|
||||||
|
aggregations.add(aggregation);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeVInt(aggregations.size());
|
||||||
|
for (Aggregation aggregation : aggregations) {
|
||||||
|
InternalAggregation internal = (InternalAggregation) aggregation;
|
||||||
|
out.writeBytesReference(internal.type().stream());
|
||||||
|
internal.writeTo(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The aggregation context that is part of the search context.
|
||||||
|
*/
|
||||||
|
public class SearchContextAggregations {
|
||||||
|
|
||||||
|
private final AggregatorFactories factories;
|
||||||
|
private Aggregator[] aggregators;
|
||||||
|
private AggregationContext aggregationContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a new aggregation context with the parsed aggregator factories
|
||||||
|
*/
|
||||||
|
public SearchContextAggregations(AggregatorFactories factories) {
|
||||||
|
this.factories = factories;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregatorFactories factories() {
|
||||||
|
return factories;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Aggregator[] aggregators() {
|
||||||
|
return aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
public AggregationContext aggregationContext() {
|
||||||
|
return aggregationContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void aggregationContext(AggregationContext aggregationContext) {
|
||||||
|
this.aggregationContext = aggregationContext;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Registers all the created aggregators (top level aggregators) for the search execution context.
|
||||||
|
*
|
||||||
|
* @param aggregators The top level aggregators of the search execution.
|
||||||
|
*/
|
||||||
|
public void aggregators(Aggregator[] aggregators) {
|
||||||
|
this.aggregators = aggregators;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,78 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.inject.AbstractModule;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.InternalDateHistogram;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.date.InternalDateRange;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.geodistance.InternalGeoDistance;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.range.ipv4.InternalIPv4Range;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.terms.UnmappedTerms;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.missing.InternalMissing;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.nested.InternalNested;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.valuecount.InternalValueCount;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.avg.InternalAvg;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.min.InternalMin;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.InternalStats;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.stats.extended.InternalExtendedStats;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.sum.InternalSum;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A module that registers all the transport streams for the addAggregation
|
||||||
|
*/
|
||||||
|
public class TransportAggregationModule extends AbstractModule {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void configure() {
|
||||||
|
|
||||||
|
// calcs
|
||||||
|
InternalAvg.registerStreams();
|
||||||
|
InternalSum.registerStreams();
|
||||||
|
InternalMin.registerStreams();
|
||||||
|
InternalMax.registerStreams();
|
||||||
|
InternalStats.registerStreams();
|
||||||
|
InternalExtendedStats.registerStreams();
|
||||||
|
InternalValueCount.registerStreams();
|
||||||
|
|
||||||
|
// buckets
|
||||||
|
InternalGlobal.registerStreams();
|
||||||
|
InternalFilter.registerStreams();
|
||||||
|
InternalMissing.registerStreams();
|
||||||
|
StringTerms.registerStreams();
|
||||||
|
LongTerms.registerStreams();
|
||||||
|
DoubleTerms.registerStreams();
|
||||||
|
UnmappedTerms.registerStreams();
|
||||||
|
InternalRange.registerStream();
|
||||||
|
InternalDateRange.registerStream();
|
||||||
|
InternalIPv4Range.registerStream();
|
||||||
|
InternalHistogram.registerStream();
|
||||||
|
InternalDateHistogram.registerStream();
|
||||||
|
InternalGeoDistance.registerStream();
|
||||||
|
InternalNested.registerStream();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,122 @@
|
||||||
|
package org.elasticsearch.search.aggregations;
|
||||||
|
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A base class for all bucket aggregation builders that are based on values (either script generated or field data values)
|
||||||
|
*/
|
||||||
|
public abstract class ValuesSourceAggregationBuilder<B extends ValuesSourceAggregationBuilder<B>> extends AggregationBuilder<B> {
|
||||||
|
|
||||||
|
private String field;
|
||||||
|
private String script;
|
||||||
|
private String scriptLang;
|
||||||
|
private Map<String, Object> params;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs a new builder.
|
||||||
|
*
|
||||||
|
* @param name The name of the aggregation.
|
||||||
|
* @param type The type of the aggregation.
|
||||||
|
*/
|
||||||
|
protected ValuesSourceAggregationBuilder(String name, String type) {
|
||||||
|
super(name, type);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the field from which the values will be extracted.
|
||||||
|
*
|
||||||
|
* @param field The name of the field
|
||||||
|
* @return This builder (fluent interface support)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B field(String field) {
|
||||||
|
this.field = field;
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the script which generates the values. If the script is configured along with the field (as in {@link #field(String)}), then
|
||||||
|
* this script will be treated as a {@code value script}. A <i>value script</i> will be applied on the values that are extracted from
|
||||||
|
* the field data (you can refer to that value in the script using the {@code _value} reserved variable). If only the script is configured
|
||||||
|
* (and the no field is configured next to it), then the script will be responsible to generate the values that will be aggregated.
|
||||||
|
*
|
||||||
|
* @param script The configured script.
|
||||||
|
* @return This builder (fluent interface support)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B script(String script) {
|
||||||
|
this.script = script;
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the language of the script (if one is defined).
|
||||||
|
* <p/>
|
||||||
|
* Also see {@link #script(String)}.
|
||||||
|
*
|
||||||
|
* @param scriptLang The language of the script.
|
||||||
|
* @return This builder (fluent interface support)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B scriptLang(String scriptLang) {
|
||||||
|
this.scriptLang = scriptLang;
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the value of a parameter that is used in the script (if one is configured).
|
||||||
|
*
|
||||||
|
* @param name The name of the parameter.
|
||||||
|
* @param value The value of the parameter.
|
||||||
|
* @return This builder (fluent interface support)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B param(String name, Object value) {
|
||||||
|
if (params == null) {
|
||||||
|
params = Maps.newHashMap();
|
||||||
|
}
|
||||||
|
params.put(name, value);
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the values of a parameters that are used in the script (if one is configured).
|
||||||
|
*
|
||||||
|
* @param params The the parameters.
|
||||||
|
* @return This builder (fluent interface support)
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public B params(Map<String, Object> params) {
|
||||||
|
if (this.params == null) {
|
||||||
|
this.params = Maps.newHashMap();
|
||||||
|
}
|
||||||
|
this.params.putAll(params);
|
||||||
|
return (B) this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected final XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject();
|
||||||
|
if (field != null) {
|
||||||
|
builder.field("field", field);
|
||||||
|
}
|
||||||
|
if (script != null) {
|
||||||
|
builder.field("script", script);
|
||||||
|
}
|
||||||
|
if (scriptLang != null) {
|
||||||
|
builder.field("script_lang", scriptLang);
|
||||||
|
}
|
||||||
|
if (this.params != null) {
|
||||||
|
builder.field("params").map(this.params);
|
||||||
|
}
|
||||||
|
|
||||||
|
doInternalXContent(builder, params);
|
||||||
|
return builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException;
|
||||||
|
}
|
|
@ -0,0 +1,106 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket;
|
||||||
|
|
||||||
|
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.metrics.MetricsAggregation;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface Bucket {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The number of documents that fall within this bucket
|
||||||
|
*/
|
||||||
|
long getDocCount();
|
||||||
|
|
||||||
|
Aggregations getAggregations();
|
||||||
|
|
||||||
|
static class Comparator<B extends Bucket> implements java.util.Comparator<B> {
|
||||||
|
|
||||||
|
private final String aggName;
|
||||||
|
private final String valueName;
|
||||||
|
private final boolean asc;
|
||||||
|
|
||||||
|
public Comparator(String expression, boolean asc) {
|
||||||
|
this.asc = asc;
|
||||||
|
int i = expression.indexOf('.');
|
||||||
|
if (i < 0) {
|
||||||
|
this.aggName = expression;
|
||||||
|
this.valueName = null;
|
||||||
|
} else {
|
||||||
|
this.aggName = expression.substring(0, i);
|
||||||
|
this.valueName = expression.substring(i+1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Comparator(String aggName, String valueName, boolean asc) {
|
||||||
|
this.aggName = aggName;
|
||||||
|
this.valueName = valueName;
|
||||||
|
this.asc = asc;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean asc() {
|
||||||
|
return asc;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String aggName() {
|
||||||
|
return aggName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String valueName() {
|
||||||
|
return valueName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(B b1, B b2) {
|
||||||
|
double v1 = value(b1);
|
||||||
|
double v2 = value(b2);
|
||||||
|
if (v1 > v2) {
|
||||||
|
return asc ? 1 : -1;
|
||||||
|
} else if (v1 < v2) {
|
||||||
|
return asc ? -1 : 1;
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
private double value(B bucket) {
|
||||||
|
MetricsAggregation aggregation = bucket.getAggregations().get(aggName);
|
||||||
|
if (aggregation == null) {
|
||||||
|
throw new ElasticSearchIllegalArgumentException("Unknown aggregation named [" + aggName + "]");
|
||||||
|
}
|
||||||
|
if (aggregation instanceof MetricsAggregation.SingleValue) {
|
||||||
|
//TODO should we throw an exception if the value name is specified?
|
||||||
|
return ((MetricsAggregation.SingleValue) aggregation).value();
|
||||||
|
}
|
||||||
|
if (aggregation instanceof MetricsAggregation.MultiValue) {
|
||||||
|
if (valueName == null) {
|
||||||
|
throw new ElasticSearchIllegalArgumentException("Cannot sort on multi valued aggregation [" + aggName + "]. A value name is required");
|
||||||
|
}
|
||||||
|
return ((MetricsAggregation.MultiValue) aggregation).value(valueName);
|
||||||
|
}
|
||||||
|
|
||||||
|
throw new ElasticSearchIllegalArgumentException("A mal attempt to sort terms by aggregation [" + aggregation.getName() +
|
||||||
|
"]. Terms can only be ordered by either standard order or direct calc aggregators of the terms");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,93 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.LongArray;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public abstract class BucketsAggregator extends Aggregator {
|
||||||
|
|
||||||
|
protected LongArray docCounts;
|
||||||
|
|
||||||
|
public BucketsAggregator(String name, BucketAggregationMode bucketAggregationMode, AggregatorFactories factories,
|
||||||
|
long estimatedBucketsCount, AggregationContext context, Aggregator parent) {
|
||||||
|
super(name, bucketAggregationMode, factories, estimatedBucketsCount, context, parent);
|
||||||
|
docCounts = BigArrays.newLongArray(estimatedBucketsCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to collect the given doc in the given bucket (identified by the bucket ordinal)
|
||||||
|
*/
|
||||||
|
protected final void collectBucket(int doc, long bucketOrd) throws IOException {
|
||||||
|
docCounts = BigArrays.grow(docCounts, bucketOrd + 1);
|
||||||
|
docCounts.increment(bucketOrd, 1);
|
||||||
|
for (int i = 0; i < subAggregators.length; i++) {
|
||||||
|
subAggregators[i].collect(doc, bucketOrd);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to collect the given doc in the given bucket but not to update the doc counts of the bucket
|
||||||
|
*/
|
||||||
|
protected final void collectBucketNoCounts(int doc, long bucketOrd) throws IOException {
|
||||||
|
for (int i = 0; i < subAggregators.length; i++) {
|
||||||
|
subAggregators[i].collect(doc, bucketOrd);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to increment the doc counts of the given bucket (identified by the bucket ordinal)
|
||||||
|
*/
|
||||||
|
protected final void incrementBucketDocCount(int inc, long bucketOrd) throws IOException {
|
||||||
|
docCounts = BigArrays.grow(docCounts, bucketOrd + 1);
|
||||||
|
docCounts.increment(bucketOrd, inc);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to return the number of documents that fell in the given bucket (identified by the bucket ordinal)
|
||||||
|
*/
|
||||||
|
protected final long bucketDocCount(long bucketOrd) {
|
||||||
|
assert bucketOrd < docCounts.size();
|
||||||
|
return docCounts.get(bucketOrd);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility method to build the aggregations of the given bucket (identified by the bucket ordinal)
|
||||||
|
*/
|
||||||
|
protected final InternalAggregations bucketAggregations(long bucketOrd) {
|
||||||
|
InternalAggregation[] aggregations = new InternalAggregation[subAggregators.length];
|
||||||
|
for (int i = 0; i < subAggregators.length; i++) {
|
||||||
|
aggregations[i] = subAggregators[i].buildAggregation(bucketOrd);
|
||||||
|
}
|
||||||
|
return new InternalAggregations(Arrays.asList(aggregations));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,194 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket;
|
||||||
|
|
||||||
|
import com.carrotsearch.hppc.hash.MurmurHash3;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.elasticsearch.common.util.BigArrays;
|
||||||
|
import org.elasticsearch.common.util.LongArray;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Specialized hash table implementation similar to BytesRefHash that maps
|
||||||
|
* long values to ids. Collisions are resolved with open addressing and linear
|
||||||
|
* probing, growth is smooth thanks to {@link BigArrays} and capacity is always
|
||||||
|
* a multiple of 2 for faster identification of buckets.
|
||||||
|
*/
|
||||||
|
// IDs are internally stored as id + 1 so that 0 encodes for an empty slot
|
||||||
|
public final class LongHash {
|
||||||
|
|
||||||
|
// Open addressing typically requires having smaller load factors compared to linked lists because
|
||||||
|
// collisions may result into worse lookup performance.
|
||||||
|
private static final float DEFAULT_MAX_LOAD_FACTOR = 0.6f;
|
||||||
|
|
||||||
|
private final float maxLoadFactor;
|
||||||
|
private long size, maxSize;
|
||||||
|
private LongArray keys;
|
||||||
|
private LongArray ids;
|
||||||
|
private long mask;
|
||||||
|
|
||||||
|
// Constructor with configurable capacity and default maximum load factor.
|
||||||
|
public LongHash(long capacity) {
|
||||||
|
this(capacity, DEFAULT_MAX_LOAD_FACTOR);
|
||||||
|
}
|
||||||
|
|
||||||
|
//Constructor with configurable capacity and load factor.
|
||||||
|
public LongHash(long capacity, float maxLoadFactor) {
|
||||||
|
Preconditions.checkArgument(capacity >= 0, "capacity must be >= 0");
|
||||||
|
Preconditions.checkArgument(maxLoadFactor > 0 && maxLoadFactor < 1, "maxLoadFactor must be > 0 and < 1");
|
||||||
|
this.maxLoadFactor = maxLoadFactor;
|
||||||
|
long buckets = 1L + (long) (capacity / maxLoadFactor);
|
||||||
|
buckets = Math.max(1, Long.highestOneBit(buckets - 1) << 1); // next power of two
|
||||||
|
assert buckets == Long.highestOneBit(buckets);
|
||||||
|
maxSize = (long) (buckets * maxLoadFactor);
|
||||||
|
assert maxSize >= capacity;
|
||||||
|
size = 0;
|
||||||
|
keys = BigArrays.newLongArray(buckets);
|
||||||
|
ids = BigArrays.newLongArray(buckets);
|
||||||
|
mask = buckets - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the number of allocated slots to store this hash table.
|
||||||
|
*/
|
||||||
|
public long capacity() {
|
||||||
|
return keys.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the number of longs in this hash table.
|
||||||
|
*/
|
||||||
|
public long size() {
|
||||||
|
return size;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long hash(long value) {
|
||||||
|
// Don't use the value directly. Under some cases eg dates, it could be that the low bits don't carry much value and we would like
|
||||||
|
// all bits of the hash to carry as much value
|
||||||
|
return MurmurHash3.hash(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long slot(long hash, long mask) {
|
||||||
|
return hash & mask;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long nextSlot(long curSlot, long mask) {
|
||||||
|
return (curSlot + 1) & mask; // linear probing
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the id associated with key at <code>0 <e; index <e; capacity()</code> or -1 if this slot is unused.
|
||||||
|
*/
|
||||||
|
public long id(long index) {
|
||||||
|
return ids.get(index) - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the key at <code>0 <e; index <e; capacity()</code>. The result is undefined if the slot is unused.
|
||||||
|
*/
|
||||||
|
public long key(long index) {
|
||||||
|
return keys.get(index);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the id associated with <code>key</code>
|
||||||
|
*/
|
||||||
|
public long get(long key) {
|
||||||
|
final long slot = slot(hash(key), mask);
|
||||||
|
for (long index = slot; ; index = nextSlot(index, mask)) {
|
||||||
|
final long id = ids.get(index);
|
||||||
|
if (id == 0L || keys.get(index) == key) {
|
||||||
|
return id - 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private long set(long key, long id) {
|
||||||
|
assert size < maxSize;
|
||||||
|
final long slot = slot(hash(key), mask);
|
||||||
|
for (long index = slot; ; index = nextSlot(index, mask)) {
|
||||||
|
final long curId = ids.get(index);
|
||||||
|
if (curId == 0) { // means unset
|
||||||
|
ids.set(index, id + 1);
|
||||||
|
keys.set(index, key);
|
||||||
|
++size;
|
||||||
|
return id;
|
||||||
|
} else if (keys.get(index) == key) {
|
||||||
|
return - curId;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to add <code>key</code>. Return its newly allocated id if it wasn't in the hash table yet, or </code>-1-id</code>
|
||||||
|
* if it was already present in the hash table.
|
||||||
|
*/
|
||||||
|
public long add(long key) {
|
||||||
|
if (size >= maxSize) {
|
||||||
|
assert size == maxSize;
|
||||||
|
grow();
|
||||||
|
}
|
||||||
|
assert size < maxSize;
|
||||||
|
return set(key, size);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void grow() {
|
||||||
|
// The difference of this implementation of grow() compared to standard hash tables is that we are growing in-place, which makes
|
||||||
|
// the re-mapping of keys to slots a bit more tricky.
|
||||||
|
assert size == maxSize;
|
||||||
|
final long prevSize = size;
|
||||||
|
final long buckets = keys.size();
|
||||||
|
// Resize arrays
|
||||||
|
final long newBuckets = buckets << 1;
|
||||||
|
assert newBuckets == Long.highestOneBit(newBuckets) : newBuckets; // power of 2
|
||||||
|
keys = BigArrays.resize(keys, newBuckets);
|
||||||
|
ids = BigArrays.resize(ids, newBuckets);
|
||||||
|
mask = newBuckets - 1;
|
||||||
|
size = 0;
|
||||||
|
// First let's remap in-place: most data will be put in its final position directly
|
||||||
|
for (long i = 0; i < buckets; ++i) {
|
||||||
|
final long id = ids.set(i, 0);
|
||||||
|
if (id > 0) {
|
||||||
|
final long key = keys.set(i, 0);
|
||||||
|
final long newId = set(key, id - 1);
|
||||||
|
assert newId == id - 1 : newId + " " + (id - 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// The only entries which have not been put in their final position in the previous loop are those that were stored in a slot that
|
||||||
|
// is < slot(key, mask). This only happens when slot(key, mask) returned a slot that was close to the end of the array and colision
|
||||||
|
// resolution has put it back in the first slots. This time, collision resolution will have put them at the beginning of the newly
|
||||||
|
// allocated slots. Let's re-add them to make sure they are in the right slot. This 2nd loop will typically exit very early.
|
||||||
|
for (long i = buckets; i < newBuckets; ++i) {
|
||||||
|
final long id = ids.set(i, 0);
|
||||||
|
if (id > 0) {
|
||||||
|
--size; // we just removed an entry
|
||||||
|
final long key = keys.set(i, 0);
|
||||||
|
final long newId = set(key, id - 1); // add it back
|
||||||
|
assert newId == id - 1 : newId + " " + (id - 1);
|
||||||
|
assert newId == get(key);
|
||||||
|
} else {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assert size == prevSize;
|
||||||
|
maxSize = (long) (newBuckets * maxLoadFactor);
|
||||||
|
assert size < maxSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A base class for all the single bucket aggregations.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
public abstract class SingleBucketAggregation<B extends SingleBucketAggregation<B>> extends InternalAggregation {
|
||||||
|
|
||||||
|
protected long docCount;
|
||||||
|
protected InternalAggregations aggregations;
|
||||||
|
|
||||||
|
protected SingleBucketAggregation() {} // for serialization
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a single bucket aggregation.
|
||||||
|
*
|
||||||
|
* @param name The aggregation name.
|
||||||
|
* @param docCount The document count in the single bucket.
|
||||||
|
* @param aggregations The already built sub-aggregations that are associated with the bucket.
|
||||||
|
*/
|
||||||
|
protected SingleBucketAggregation(String name, long docCount, InternalAggregations aggregations) {
|
||||||
|
super(name);
|
||||||
|
this.docCount = docCount;
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getDocCount() {
|
||||||
|
return docCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
public InternalAggregations getAggregations() {
|
||||||
|
return aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation reduce(ReduceContext reduceContext) {
|
||||||
|
List<InternalAggregation> aggregations = reduceContext.aggregations();
|
||||||
|
if (aggregations.size() == 1) {
|
||||||
|
return aggregations.get(0);
|
||||||
|
}
|
||||||
|
B reduced = null;
|
||||||
|
List<InternalAggregations> subAggregationsList = new ArrayList<InternalAggregations>(aggregations.size());
|
||||||
|
for (InternalAggregation aggregation : aggregations) {
|
||||||
|
if (reduced == null) {
|
||||||
|
reduced = (B) aggregation;
|
||||||
|
} else {
|
||||||
|
this.docCount += ((B) aggregation).docCount;
|
||||||
|
}
|
||||||
|
subAggregationsList.add(((B) aggregation).aggregations);
|
||||||
|
}
|
||||||
|
reduced.aggregations = InternalAggregations.reduce(subAggregationsList, reduceContext.cacheRecycler());
|
||||||
|
return reduced;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
name = in.readString();
|
||||||
|
docCount = in.readVLong();
|
||||||
|
aggregations = InternalAggregations.readAggregations(in);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeString(name);
|
||||||
|
out.writeVLong(docCount);
|
||||||
|
aggregations.writeTo(out);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
builder.startObject(name);
|
||||||
|
builder.field(CommonFields.DOC_COUNT, docCount);
|
||||||
|
aggregations.toXContentInternal(builder, params);
|
||||||
|
return builder.endObject();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,41 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A bucket aggregator that doesn't create new buckets.
|
||||||
|
*/
|
||||||
|
public abstract class SingleBucketAggregator extends BucketsAggregator {
|
||||||
|
|
||||||
|
protected SingleBucketAggregator(String name, AggregatorFactories factories,
|
||||||
|
AggregationContext aggregationContext, Aggregator parent) {
|
||||||
|
super(name, BucketAggregationMode.MULTI_BUCKETS, factories, parent == null ? 1 : parent.estimatedBucketCount(), aggregationContext, parent);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldCollect() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,34 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.filter;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregations;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface Filter extends Aggregation {
|
||||||
|
|
||||||
|
long getDocCount();
|
||||||
|
|
||||||
|
Aggregations getAggregations();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,34 @@
|
||||||
|
package org.elasticsearch.search.aggregations.bucket.filter;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.index.query.FilterBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
import org.elasticsearch.search.builder.SearchSourceBuilderException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class FilterAggregationBuilder extends AggregationBuilder<FilterAggregationBuilder> {
|
||||||
|
|
||||||
|
private FilterBuilder filter;
|
||||||
|
|
||||||
|
public FilterAggregationBuilder(String name) {
|
||||||
|
super(name, InternalFilter.TYPE.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
public FilterAggregationBuilder filter(FilterBuilder filter) {
|
||||||
|
this.filter = filter;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
if (filter == null) {
|
||||||
|
throw new SearchSourceBuilderException("filter must be set on filter aggregation [" + name + "]");
|
||||||
|
}
|
||||||
|
filter.toXContent(builder, params);
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,100 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.filter;
|
||||||
|
|
||||||
|
import org.apache.lucene.index.AtomicReaderContext;
|
||||||
|
import org.apache.lucene.search.Filter;
|
||||||
|
import org.apache.lucene.util.Bits;
|
||||||
|
import org.elasticsearch.common.lucene.ReaderContextAware;
|
||||||
|
import org.elasticsearch.common.lucene.docset.DocIdSets;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Aggregate all docs that match a filter.
|
||||||
|
*/
|
||||||
|
public class FilterAggregator extends SingleBucketAggregator implements ReaderContextAware {
|
||||||
|
|
||||||
|
private final Filter filter;
|
||||||
|
|
||||||
|
private Bits bits;
|
||||||
|
|
||||||
|
public FilterAggregator(String name,
|
||||||
|
org.apache.lucene.search.Filter filter,
|
||||||
|
AggregatorFactories factories,
|
||||||
|
AggregationContext aggregationContext,
|
||||||
|
Aggregator parent) {
|
||||||
|
super(name, factories, aggregationContext, parent);
|
||||||
|
this.filter = filter;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setNextReader(AtomicReaderContext reader) {
|
||||||
|
try {
|
||||||
|
bits = DocIdSets.toSafeBits(reader.reader(), filter.getDocIdSet(reader, reader.reader().getLiveDocs()));
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
throw new AggregationExecutionException("Failed to aggregate filter aggregator [" + name + "]", ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, long owningBucketOrdinal) throws IOException {
|
||||||
|
if (bits.get(doc)) {
|
||||||
|
collectBucket(doc, owningBucketOrdinal);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||||
|
return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildEmptyAggregation() {
|
||||||
|
return new InternalFilter(name, 0, buildEmptySubAggregations());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Factory extends AggregatorFactory {
|
||||||
|
|
||||||
|
private org.apache.lucene.search.Filter filter;
|
||||||
|
|
||||||
|
public Factory(String name, Filter filter) {
|
||||||
|
super(name, InternalFilter.TYPE.name());
|
||||||
|
this.filter = filter;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
|
||||||
|
FilterAggregator aggregator = new FilterAggregator(name, filter, factories, context, parent);
|
||||||
|
context.registerReaderContextAware(aggregator);
|
||||||
|
return aggregator;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,46 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.filter;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.query.ParsedFilter;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class FilterParser implements Aggregator.Parser {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String type() {
|
||||||
|
return InternalFilter.TYPE.name();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||||
|
ParsedFilter filter = context.queryParserService().parseInnerFilter(parser);
|
||||||
|
return new FilterAggregator.Factory(aggregationName, filter.filter());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,60 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.filter;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationStreams;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class InternalFilter extends SingleBucketAggregation<InternalFilter> implements Filter {
|
||||||
|
|
||||||
|
public final static Type TYPE = new Type("filter");
|
||||||
|
|
||||||
|
public final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
|
||||||
|
@Override
|
||||||
|
public InternalFilter readResult(StreamInput in) throws IOException {
|
||||||
|
InternalFilter result = new InternalFilter();
|
||||||
|
result.readFrom(in);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public static void registerStreams() {
|
||||||
|
AggregationStreams.registerStream(STREAM, TYPE.stream());
|
||||||
|
}
|
||||||
|
|
||||||
|
InternalFilter() {} // for serialization
|
||||||
|
|
||||||
|
InternalFilter(String name, long docCount, InternalAggregations subAggregations) {
|
||||||
|
super(name, docCount, subAggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type type() {
|
||||||
|
return TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,34 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.global;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregations;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface Global extends Aggregation {
|
||||||
|
|
||||||
|
long getDocCount();
|
||||||
|
|
||||||
|
Aggregations getAggregations();
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,74 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.global;
|
||||||
|
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationExecutionException;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class GlobalAggregator extends SingleBucketAggregator {
|
||||||
|
|
||||||
|
public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext) {
|
||||||
|
super(name, subFactories, aggregationContext, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, long owningBucketOrdinal) throws IOException {
|
||||||
|
assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator";
|
||||||
|
collectBucket(doc, owningBucketOrdinal);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||||
|
assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator";
|
||||||
|
return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildEmptyAggregation() {
|
||||||
|
throw new UnsupportedOperationException("global aggregations cannot serve as sub-aggregations, hence should never be called on #buildEmptyAggregations");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Factory extends AggregatorFactory {
|
||||||
|
|
||||||
|
public Factory(String name) {
|
||||||
|
super(name, InternalGlobal.TYPE.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator create(AggregationContext context, Aggregator parent, long expectedBucketsCount) {
|
||||||
|
if (parent != null) {
|
||||||
|
throw new AggregationExecutionException("Aggregation [" + parent.name() + "] cannot have a global " +
|
||||||
|
"sub-aggregation [" + name + "]. Global aggregations can only be defined as top level aggregations");
|
||||||
|
}
|
||||||
|
return new GlobalAggregator(name, factories, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,21 @@
|
||||||
|
package org.elasticsearch.search.aggregations.bucket.global;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class GlobalBuilder extends AggregationBuilder<GlobalBuilder> {
|
||||||
|
|
||||||
|
public GlobalBuilder(String name) {
|
||||||
|
super(name, InternalGlobal.TYPE.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
return builder.startObject().endObject();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,45 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.global;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class GlobalParser implements Aggregator.Parser {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String type() {
|
||||||
|
return InternalGlobal.TYPE.name();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||||
|
parser.nextToken();
|
||||||
|
return new GlobalAggregator.Factory(aggregationName);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,60 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.global;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationStreams;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregation;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A global scope get (the document set on which we aggregate is all documents in the search context (ie. index + type)
|
||||||
|
* regardless the query.
|
||||||
|
*/
|
||||||
|
public class InternalGlobal extends SingleBucketAggregation<InternalGlobal> implements Global {
|
||||||
|
|
||||||
|
public final static Type TYPE = new Type("global");
|
||||||
|
|
||||||
|
public final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
|
||||||
|
@Override
|
||||||
|
public InternalGlobal readResult(StreamInput in) throws IOException {
|
||||||
|
InternalGlobal result = new InternalGlobal();
|
||||||
|
result.readFrom(in);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public static void registerStreams() {
|
||||||
|
AggregationStreams.registerStream(STREAM, TYPE.stream());
|
||||||
|
}
|
||||||
|
|
||||||
|
public InternalGlobal() {} // for serialization
|
||||||
|
|
||||||
|
public InternalGlobal(String name, long docCount, InternalAggregations aggregations) {
|
||||||
|
super(name, docCount, aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type type() {
|
||||||
|
return TYPE;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,343 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import com.carrotsearch.hppc.LongObjectOpenHashMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.lucene.util.CollectionUtil;
|
||||||
|
import org.elasticsearch.cache.recycler.CacheRecycler;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
|
import org.elasticsearch.common.io.stream.Streamable;
|
||||||
|
import org.elasticsearch.common.recycler.Recycler;
|
||||||
|
import org.elasticsearch.common.rounding.Rounding;
|
||||||
|
import org.elasticsearch.common.text.StringText;
|
||||||
|
import org.elasticsearch.common.text.Text;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatterStreams;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.ListIterator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An internal implementation of {@link HistogramBase}
|
||||||
|
*/
|
||||||
|
abstract class AbstractHistogramBase<B extends HistogramBase.Bucket> extends InternalAggregation implements HistogramBase<B>, ToXContent, Streamable {
|
||||||
|
|
||||||
|
public static class Bucket implements HistogramBase.Bucket {
|
||||||
|
|
||||||
|
private long key;
|
||||||
|
private long docCount;
|
||||||
|
private InternalAggregations aggregations;
|
||||||
|
|
||||||
|
public Bucket(long key, long docCount, InternalAggregations aggregations) {
|
||||||
|
this.key = key;
|
||||||
|
this.docCount = docCount;
|
||||||
|
this.aggregations = aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getKey() {
|
||||||
|
return key;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getDocCount() {
|
||||||
|
return docCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregations getAggregations() {
|
||||||
|
return aggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
Bucket reduce(List<Bucket> buckets, CacheRecycler cacheRecycler) {
|
||||||
|
if (buckets.size() == 1) {
|
||||||
|
return buckets.get(0);
|
||||||
|
}
|
||||||
|
List<InternalAggregations> aggregations = new ArrayList<InternalAggregations>(buckets.size());
|
||||||
|
Bucket reduced = null;
|
||||||
|
for (Bucket bucket : buckets) {
|
||||||
|
if (reduced == null) {
|
||||||
|
reduced = bucket;
|
||||||
|
} else {
|
||||||
|
reduced.docCount += bucket.docCount;
|
||||||
|
}
|
||||||
|
aggregations.add((InternalAggregations) bucket.getAggregations());
|
||||||
|
}
|
||||||
|
reduced.aggregations = InternalAggregations.reduce(aggregations, cacheRecycler);
|
||||||
|
return reduced;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class EmptyBucketInfo {
|
||||||
|
final Rounding rounding;
|
||||||
|
final InternalAggregations subAggregations;
|
||||||
|
|
||||||
|
EmptyBucketInfo(Rounding rounding, InternalAggregations subAggregations) {
|
||||||
|
this.rounding = rounding;
|
||||||
|
this.subAggregations = subAggregations;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static EmptyBucketInfo readFrom(StreamInput in) throws IOException {
|
||||||
|
return new EmptyBucketInfo(Rounding.Streams.read(in), InternalAggregations.readAggregations(in));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void writeTo(EmptyBucketInfo info, StreamOutput out) throws IOException {
|
||||||
|
Rounding.Streams.write(info.rounding, out);
|
||||||
|
info.subAggregations.writeTo(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static interface Factory<B extends HistogramBase.Bucket> {
|
||||||
|
|
||||||
|
String type();
|
||||||
|
|
||||||
|
AbstractHistogramBase create(String name, List<B> buckets, InternalOrder order, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed);
|
||||||
|
|
||||||
|
Bucket createBucket(long key, long docCount, InternalAggregations aggregations);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private List<B> buckets;
|
||||||
|
private LongObjectOpenHashMap<HistogramBase.Bucket> bucketsMap;
|
||||||
|
private InternalOrder order;
|
||||||
|
private ValueFormatter formatter;
|
||||||
|
private boolean keyed;
|
||||||
|
private EmptyBucketInfo emptyBucketInfo;
|
||||||
|
|
||||||
|
protected AbstractHistogramBase() {} // for serialization
|
||||||
|
|
||||||
|
protected AbstractHistogramBase(String name, List<B> buckets, InternalOrder order, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
|
||||||
|
super(name);
|
||||||
|
this.buckets = buckets;
|
||||||
|
this.order = order;
|
||||||
|
this.emptyBucketInfo = emptyBucketInfo;
|
||||||
|
this.formatter = formatter;
|
||||||
|
this.keyed = keyed;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<B> iterator() {
|
||||||
|
return buckets.iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<B> buckets() {
|
||||||
|
return buckets;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public B getByKey(long key) {
|
||||||
|
if (bucketsMap == null) {
|
||||||
|
bucketsMap = new LongObjectOpenHashMap<HistogramBase.Bucket>(buckets.size());
|
||||||
|
for (HistogramBase.Bucket bucket : buckets) {
|
||||||
|
bucketsMap.put(bucket.getKey(), bucket);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return (B) bucketsMap.get(key);
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO extract the reduce logic to a strategy class and have it configurable at request time (two possible strategies - total & delta)
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation reduce(ReduceContext reduceContext) {
|
||||||
|
List<InternalAggregation> aggregations = reduceContext.aggregations();
|
||||||
|
if (aggregations.size() == 1) {
|
||||||
|
|
||||||
|
if (emptyBucketInfo == null) {
|
||||||
|
return aggregations.get(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
// we need to fill the gaps with empty buckets
|
||||||
|
AbstractHistogramBase histo = (AbstractHistogramBase) aggregations.get(0);
|
||||||
|
CollectionUtil.introSort(histo.buckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
|
||||||
|
List<HistogramBase.Bucket> list = order.asc ? histo.buckets : Lists.reverse(histo.buckets);
|
||||||
|
HistogramBase.Bucket prevBucket = null;
|
||||||
|
ListIterator<HistogramBase.Bucket> iter = list.listIterator();
|
||||||
|
while (iter.hasNext()) {
|
||||||
|
// look ahead on the next bucket without advancing the iter
|
||||||
|
// so we'll be able to insert elements at the right position
|
||||||
|
HistogramBase.Bucket nextBucket = list.get(iter.nextIndex());
|
||||||
|
if (prevBucket != null) {
|
||||||
|
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.getKey());
|
||||||
|
while (key != nextBucket.getKey()) {
|
||||||
|
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations));
|
||||||
|
key = emptyBucketInfo.rounding.nextRoundingValue(key);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
prevBucket = iter.next();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
|
||||||
|
CollectionUtil.introSort(histo.buckets, order.comparator());
|
||||||
|
}
|
||||||
|
|
||||||
|
return histo;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
AbstractHistogramBase reduced = (AbstractHistogramBase) aggregations.get(0);
|
||||||
|
|
||||||
|
Recycler.V<LongObjectOpenHashMap<List<Bucket>>> bucketsByKey = reduceContext.cacheRecycler().longObjectMap(-1);
|
||||||
|
for (InternalAggregation aggregation : aggregations) {
|
||||||
|
AbstractHistogramBase<B> histogram = (AbstractHistogramBase) aggregation;
|
||||||
|
for (B bucket : histogram.buckets) {
|
||||||
|
List<Bucket> bucketList = bucketsByKey.v().get(((Bucket) bucket).key);
|
||||||
|
if (bucketList == null) {
|
||||||
|
bucketList = new ArrayList<Bucket>(aggregations.size());
|
||||||
|
bucketsByKey.v().put(((Bucket) bucket).key, bucketList);
|
||||||
|
}
|
||||||
|
bucketList.add((Bucket) bucket);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<HistogramBase.Bucket> reducedBuckets = new ArrayList<HistogramBase.Bucket>(bucketsByKey.v().size());
|
||||||
|
Object[] buckets = bucketsByKey.v().values;
|
||||||
|
boolean[] allocated = bucketsByKey.v().allocated;
|
||||||
|
for (int i = 0; i < allocated.length; i++) {
|
||||||
|
if (allocated[i]) {
|
||||||
|
Bucket bucket = ((List<Bucket>) buckets[i]).get(0).reduce(((List<Bucket>) buckets[i]), reduceContext.cacheRecycler());
|
||||||
|
reducedBuckets.add(bucket);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
bucketsByKey.release();
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
// adding empty buckets in needed
|
||||||
|
if (emptyBucketInfo != null) {
|
||||||
|
CollectionUtil.introSort(reducedBuckets, order.asc ? InternalOrder.KEY_ASC.comparator() : InternalOrder.KEY_DESC.comparator());
|
||||||
|
List<HistogramBase.Bucket> list = order.asc ? reducedBuckets : Lists.reverse(reducedBuckets);
|
||||||
|
HistogramBase.Bucket prevBucket = null;
|
||||||
|
ListIterator<HistogramBase.Bucket> iter = list.listIterator();
|
||||||
|
while (iter.hasNext()) {
|
||||||
|
HistogramBase.Bucket nextBucket = list.get(iter.nextIndex());
|
||||||
|
if (prevBucket != null) {
|
||||||
|
long key = emptyBucketInfo.rounding.nextRoundingValue(prevBucket.getKey());
|
||||||
|
while (key != nextBucket.getKey()) {
|
||||||
|
iter.add(createBucket(key, 0, emptyBucketInfo.subAggregations));
|
||||||
|
key = emptyBucketInfo.rounding.nextRoundingValue(key);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
prevBucket = iter.next();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (order != InternalOrder.KEY_ASC && order != InternalOrder.KEY_DESC) {
|
||||||
|
CollectionUtil.introSort(reducedBuckets, order.comparator());
|
||||||
|
}
|
||||||
|
|
||||||
|
} else {
|
||||||
|
CollectionUtil.introSort(reducedBuckets, order.comparator());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
reduced.buckets = reducedBuckets;
|
||||||
|
return reduced;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected B createBucket(long key, long docCount, InternalAggregations aggregations) {
|
||||||
|
return (B) new Bucket(key, docCount, aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void readFrom(StreamInput in) throws IOException {
|
||||||
|
name = in.readString();
|
||||||
|
order = InternalOrder.Streams.readOrder(in);
|
||||||
|
if (in.readBoolean()) {
|
||||||
|
emptyBucketInfo = EmptyBucketInfo.readFrom(in);
|
||||||
|
}
|
||||||
|
formatter = ValueFormatterStreams.readOptional(in);
|
||||||
|
keyed = in.readBoolean();
|
||||||
|
int size = in.readVInt();
|
||||||
|
List<B> buckets = new ArrayList<B>(size);
|
||||||
|
for (int i = 0; i < size; i++) {
|
||||||
|
buckets.add(createBucket(in.readLong(), in.readVLong(), InternalAggregations.readAggregations(in)));
|
||||||
|
}
|
||||||
|
this.buckets = buckets;
|
||||||
|
this.bucketsMap = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeTo(StreamOutput out) throws IOException {
|
||||||
|
out.writeString(name);
|
||||||
|
InternalOrder.Streams.writeOrder(order, out);
|
||||||
|
if (emptyBucketInfo == null) {
|
||||||
|
out.writeBoolean(false);
|
||||||
|
} else {
|
||||||
|
out.writeBoolean(true);
|
||||||
|
EmptyBucketInfo.writeTo(emptyBucketInfo, out);
|
||||||
|
}
|
||||||
|
ValueFormatterStreams.writeOptional(formatter, out);
|
||||||
|
out.writeBoolean(keyed);
|
||||||
|
out.writeVInt(buckets.size());
|
||||||
|
for (HistogramBase.Bucket bucket : buckets) {
|
||||||
|
out.writeLong(((Bucket) bucket).key);
|
||||||
|
out.writeVLong(((Bucket) bucket).docCount);
|
||||||
|
((Bucket) bucket).aggregations.writeTo(out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
if (keyed) {
|
||||||
|
builder.startObject(name);
|
||||||
|
} else {
|
||||||
|
builder.startArray(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (HistogramBase.Bucket bucket : buckets) {
|
||||||
|
if (formatter != null) {
|
||||||
|
Text keyTxt = new StringText(formatter.format(bucket.getKey()));
|
||||||
|
if (keyed) {
|
||||||
|
builder.startObject(keyTxt.string());
|
||||||
|
} else {
|
||||||
|
builder.startObject();
|
||||||
|
}
|
||||||
|
builder.field(CommonFields.KEY_AS_STRING, keyTxt);
|
||||||
|
} else {
|
||||||
|
if (keyed) {
|
||||||
|
builder.startObject(String.valueOf(bucket.getKey()));
|
||||||
|
} else {
|
||||||
|
builder.startObject();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
builder.field(CommonFields.KEY, ((Bucket) bucket).key);
|
||||||
|
builder.field(CommonFields.DOC_COUNT, ((Bucket) bucket).docCount);
|
||||||
|
((Bucket) bucket).aggregations.toXContentInternal(builder, params);
|
||||||
|
builder.endObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (keyed) {
|
||||||
|
builder.endObject();
|
||||||
|
} else {
|
||||||
|
builder.endArray();
|
||||||
|
}
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,77 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public interface DateHistogram extends HistogramBase<DateHistogram.Bucket> {
|
||||||
|
|
||||||
|
static interface Bucket extends HistogramBase.Bucket {
|
||||||
|
|
||||||
|
DateTime getKeyAsDate();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
static class Interval {
|
||||||
|
|
||||||
|
public static final Interval SECOND = new Interval("1s");
|
||||||
|
public static final Interval MINUTE = new Interval("1m");
|
||||||
|
public static final Interval HOUR = new Interval("1h");
|
||||||
|
public static final Interval DAY = new Interval("1d");
|
||||||
|
public static final Interval WEEK = new Interval("1w");
|
||||||
|
public static final Interval MONTH = new Interval("1M");
|
||||||
|
public static final Interval QUARTER = new Interval("1q");
|
||||||
|
public static final Interval YEAR = new Interval("1y");
|
||||||
|
|
||||||
|
public static Interval seconds(int sec) {
|
||||||
|
return new Interval(sec + "s");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Interval minutes(int min) {
|
||||||
|
return new Interval(min + "m");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Interval hours(int hours) {
|
||||||
|
return new Interval(hours + "h");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Interval days(int days) {
|
||||||
|
return new Interval(days + "d");
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Interval week(int weeks) {
|
||||||
|
return new Interval(weeks + "w");
|
||||||
|
}
|
||||||
|
|
||||||
|
private final String expression;
|
||||||
|
|
||||||
|
public Interval(String expression) {
|
||||||
|
this.expression = expression;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return expression;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,115 @@
|
||||||
|
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.builder.SearchSourceBuilderException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DateHistogramBuilder extends ValuesSourceAggregationBuilder<DateHistogramBuilder> {
|
||||||
|
|
||||||
|
private Object interval;
|
||||||
|
private HistogramBase.Order order;
|
||||||
|
private String preZone;
|
||||||
|
private String postZone;
|
||||||
|
private boolean preZoneAdjustLargeInterval;
|
||||||
|
long preOffset = 0;
|
||||||
|
long postOffset = 0;
|
||||||
|
float factor = 1.0f;
|
||||||
|
|
||||||
|
public DateHistogramBuilder(String name) {
|
||||||
|
super(name, InternalDateHistogram.TYPE.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder interval(long interval) {
|
||||||
|
this.interval = interval;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder interval(DateHistogram.Interval interval) {
|
||||||
|
this.interval = interval;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder order(DateHistogram.Order order) {
|
||||||
|
this.order = order;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder preZone(String preZone) {
|
||||||
|
this.preZone = preZone;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder postZone(String postZone) {
|
||||||
|
this.postZone = postZone;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder preZoneAdjustLargeInterval(boolean preZoneAdjustLargeInterval) {
|
||||||
|
this.preZoneAdjustLargeInterval = preZoneAdjustLargeInterval;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder preOffset(long preOffset) {
|
||||||
|
this.preOffset = preOffset;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder postOffset(long postOffset) {
|
||||||
|
this.postOffset = postOffset;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public DateHistogramBuilder factor(float factor) {
|
||||||
|
this.factor = factor;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
if (interval == null) {
|
||||||
|
throw new SearchSourceBuilderException("[interval] must be defined for histogram aggregation [" + name + "]");
|
||||||
|
}
|
||||||
|
if (interval instanceof Number) {
|
||||||
|
interval = TimeValue.timeValueMillis(((Number) interval).longValue()).toString();
|
||||||
|
}
|
||||||
|
builder.field("interval", interval);
|
||||||
|
|
||||||
|
if (order != null) {
|
||||||
|
builder.field("order");
|
||||||
|
order.toXContent(builder, params);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (preZone != null) {
|
||||||
|
builder.field("pre_zone", preZone);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (postZone != null) {
|
||||||
|
builder.field("post_zone", postZone);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (preZoneAdjustLargeInterval) {
|
||||||
|
builder.field("pre_zone_adjust_large_interval", true);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (preOffset != 0) {
|
||||||
|
builder.field("pre_offset", preOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (postOffset != 0) {
|
||||||
|
builder.field("post_offset", postOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (factor != 1.0f) {
|
||||||
|
builder.field("factor", factor);
|
||||||
|
}
|
||||||
|
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,259 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.elasticsearch.common.collect.MapBuilder;
|
||||||
|
import org.elasticsearch.common.joda.DateMathParser;
|
||||||
|
import org.elasticsearch.common.rounding.DateTimeUnit;
|
||||||
|
import org.elasticsearch.common.rounding.TimeZoneRounding;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||||
|
import org.elasticsearch.index.mapper.FieldMapper;
|
||||||
|
import org.elasticsearch.index.mapper.core.DateFieldMapper;
|
||||||
|
import org.elasticsearch.script.SearchScript;
|
||||||
|
import org.elasticsearch.search.SearchParseException;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueParser;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
import org.joda.time.DateTimeZone;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class DateHistogramParser implements Aggregator.Parser {
|
||||||
|
|
||||||
|
private final ImmutableMap<String, DateTimeUnit> dateFieldUnits;
|
||||||
|
|
||||||
|
public DateHistogramParser() {
|
||||||
|
dateFieldUnits = MapBuilder.<String, DateTimeUnit>newMapBuilder()
|
||||||
|
.put("year", DateTimeUnit.YEAR_OF_CENTURY)
|
||||||
|
.put("1y", DateTimeUnit.YEAR_OF_CENTURY)
|
||||||
|
.put("quarter", DateTimeUnit.QUARTER)
|
||||||
|
.put("1q", DateTimeUnit.QUARTER)
|
||||||
|
.put("month", DateTimeUnit.MONTH_OF_YEAR)
|
||||||
|
.put("1M", DateTimeUnit.MONTH_OF_YEAR)
|
||||||
|
.put("week", DateTimeUnit.WEEK_OF_WEEKYEAR)
|
||||||
|
.put("1w", DateTimeUnit.WEEK_OF_WEEKYEAR)
|
||||||
|
.put("day", DateTimeUnit.DAY_OF_MONTH)
|
||||||
|
.put("1d", DateTimeUnit.DAY_OF_MONTH)
|
||||||
|
.put("hour", DateTimeUnit.HOUR_OF_DAY)
|
||||||
|
.put("1h", DateTimeUnit.HOUR_OF_DAY)
|
||||||
|
.put("minute", DateTimeUnit.MINUTES_OF_HOUR)
|
||||||
|
.put("1m", DateTimeUnit.MINUTES_OF_HOUR)
|
||||||
|
.put("second", DateTimeUnit.SECOND_OF_MINUTE)
|
||||||
|
.put("1s", DateTimeUnit.SECOND_OF_MINUTE)
|
||||||
|
.immutableMap();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String type() {
|
||||||
|
return InternalDateHistogram.TYPE.name();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||||
|
|
||||||
|
ValuesSourceConfig<NumericValuesSource> config = new ValuesSourceConfig<NumericValuesSource>(NumericValuesSource.class);
|
||||||
|
|
||||||
|
String field = null;
|
||||||
|
String script = null;
|
||||||
|
String scriptLang = null;
|
||||||
|
Map<String, Object> scriptParams = null;
|
||||||
|
boolean keyed = false;
|
||||||
|
boolean computeEmptyBuckets = false;
|
||||||
|
InternalOrder order = (InternalOrder) Histogram.Order.KEY_ASC;
|
||||||
|
String interval = null;
|
||||||
|
boolean preZoneAdjustLargeInterval = false;
|
||||||
|
DateTimeZone preZone = DateTimeZone.UTC;
|
||||||
|
DateTimeZone postZone = DateTimeZone.UTC;
|
||||||
|
String format = null;
|
||||||
|
long preOffset = 0;
|
||||||
|
long postOffset = 0;
|
||||||
|
boolean assumeSorted = false;
|
||||||
|
|
||||||
|
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 (token == XContentParser.Token.VALUE_STRING) {
|
||||||
|
if ("field".equals(currentFieldName)) {
|
||||||
|
field = parser.text();
|
||||||
|
} else if ("script".equals(currentFieldName)) {
|
||||||
|
script = parser.text();
|
||||||
|
} else if ("script_lang".equals(currentFieldName) || "scriptLang".equals(currentFieldName)) {
|
||||||
|
scriptLang = parser.text();
|
||||||
|
} else if ("time_zone".equals(currentFieldName) || "timeZone".equals(currentFieldName)) {
|
||||||
|
preZone = parseZone(parser, token);
|
||||||
|
} else if ("pre_zone".equals(currentFieldName) || "preZone".equals(currentFieldName)) {
|
||||||
|
preZone = parseZone(parser, token);
|
||||||
|
} else if ("pre_zone_adjust_large_interval".equals(currentFieldName) || "preZoneAdjustLargeInterval".equals(currentFieldName)) {
|
||||||
|
preZoneAdjustLargeInterval = parser.booleanValue();
|
||||||
|
} else if ("post_zone".equals(currentFieldName) || "postZone".equals(currentFieldName)) {
|
||||||
|
postZone = parseZone(parser, token);
|
||||||
|
} else if ("pre_offset".equals(currentFieldName) || "preOffset".equals(currentFieldName)) {
|
||||||
|
preOffset = parseOffset(parser.text());
|
||||||
|
} else if ("post_offset".equals(currentFieldName) || "postOffset".equals(currentFieldName)) {
|
||||||
|
postOffset = parseOffset(parser.text());
|
||||||
|
} else if ("interval".equals(currentFieldName)) {
|
||||||
|
interval = parser.text();
|
||||||
|
} else if ("format".equals(currentFieldName)) {
|
||||||
|
format = parser.text();
|
||||||
|
}
|
||||||
|
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
|
||||||
|
if ("keyed".equals(currentFieldName)) {
|
||||||
|
keyed = parser.booleanValue();
|
||||||
|
} else if ("compute_empty_buckets".equals(currentFieldName) || "computeEmptyBuckets".equals(currentFieldName)) {
|
||||||
|
computeEmptyBuckets = parser.booleanValue();
|
||||||
|
} else if ("script_values_sorted".equals(currentFieldName)) {
|
||||||
|
assumeSorted = parser.booleanValue();
|
||||||
|
}
|
||||||
|
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||||
|
if ("params".equals(currentFieldName)) {
|
||||||
|
scriptParams = parser.map();
|
||||||
|
} else if ("order".equals(currentFieldName)) {
|
||||||
|
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||||
|
if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
|
currentFieldName = parser.currentName();
|
||||||
|
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||||
|
String dir = parser.text();
|
||||||
|
boolean asc = "asc".equals(dir);
|
||||||
|
order = resolveOrder(currentFieldName, asc);
|
||||||
|
//TODO should we throw an error if the value is not "asc" or "desc"???
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (interval == null) {
|
||||||
|
throw new SearchParseException(context, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
|
||||||
|
}
|
||||||
|
|
||||||
|
SearchScript searchScript = null;
|
||||||
|
if (script != null) {
|
||||||
|
searchScript = context.scriptService().search(context.lookup(), scriptLang, script, scriptParams);
|
||||||
|
config.script(searchScript);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!assumeSorted) {
|
||||||
|
// we need values to be sorted and unique for efficiency
|
||||||
|
config.ensureSorted(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeZoneRounding.Builder tzRoundingBuilder;
|
||||||
|
DateTimeUnit dateTimeUnit = dateFieldUnits.get(interval);
|
||||||
|
if (dateTimeUnit != null) {
|
||||||
|
tzRoundingBuilder = TimeZoneRounding.builder(dateTimeUnit);
|
||||||
|
} else {
|
||||||
|
// the interval is a time value?
|
||||||
|
tzRoundingBuilder = TimeZoneRounding.builder(TimeValue.parseTimeValue(interval, null));
|
||||||
|
}
|
||||||
|
|
||||||
|
TimeZoneRounding rounding = tzRoundingBuilder
|
||||||
|
.preZone(preZone).postZone(postZone)
|
||||||
|
.preZoneAdjustLargeInterval(preZoneAdjustLargeInterval)
|
||||||
|
.preOffset(preOffset).postOffset(postOffset)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
if (format != null) {
|
||||||
|
config.formatter(new ValueFormatter.DateTime(format));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (field == null) {
|
||||||
|
|
||||||
|
if (searchScript != null) {
|
||||||
|
ValueParser valueParser = new ValueParser.DateMath(new DateMathParser(DateFieldMapper.Defaults.DATE_TIME_FORMATTER, DateFieldMapper.Defaults.TIME_UNIT));
|
||||||
|
config.parser(valueParser);
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, computeEmptyBuckets, InternalDateHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
// falling back on the get field data context
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, computeEmptyBuckets, InternalDateHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
FieldMapper<?> mapper = context.smartNameFieldMapper(field);
|
||||||
|
if (mapper == null) {
|
||||||
|
config.unmapped(true);
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, computeEmptyBuckets, InternalDateHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!(mapper instanceof DateFieldMapper)) {
|
||||||
|
throw new SearchParseException(context, "date histogram can only be aggregated on date fields but [" + field + "] is not a date field");
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexFieldData<?> indexFieldData = context.fieldData().getForField(mapper);
|
||||||
|
config.fieldContext(new FieldContext(field, indexFieldData));
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, computeEmptyBuckets, InternalDateHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static InternalOrder resolveOrder(String key, boolean asc) {
|
||||||
|
if ("_key".equals(key) || "_time".equals(key)) {
|
||||||
|
return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
|
||||||
|
}
|
||||||
|
if ("_count".equals(key)) {
|
||||||
|
return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
|
||||||
|
}
|
||||||
|
int i = key.indexOf('.');
|
||||||
|
if (i < 0) {
|
||||||
|
return HistogramBase.Order.aggregation(key, asc);
|
||||||
|
}
|
||||||
|
return HistogramBase.Order.aggregation(key.substring(0, i), key.substring(i + 1), asc);
|
||||||
|
}
|
||||||
|
|
||||||
|
private long parseOffset(String offset) throws IOException {
|
||||||
|
if (offset.charAt(0) == '-') {
|
||||||
|
return -TimeValue.parseTimeValue(offset.substring(1), null).millis();
|
||||||
|
}
|
||||||
|
int beginIndex = offset.charAt(0) == '+' ? 1 : 0;
|
||||||
|
return TimeValue.parseTimeValue(offset.substring(beginIndex), null).millis();
|
||||||
|
}
|
||||||
|
|
||||||
|
private DateTimeZone parseZone(XContentParser parser, XContentParser.Token token) throws IOException {
|
||||||
|
if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||||
|
return DateTimeZone.forOffsetHours(parser.intValue());
|
||||||
|
} else {
|
||||||
|
String text = parser.text();
|
||||||
|
int index = text.indexOf(':');
|
||||||
|
if (index != -1) {
|
||||||
|
int beginIndex = text.charAt(0) == '+' ? 1 : 0;
|
||||||
|
// format like -02:30
|
||||||
|
return DateTimeZone.forOffsetHoursMinutes(
|
||||||
|
Integer.parseInt(text.substring(beginIndex, index)),
|
||||||
|
Integer.parseInt(text.substring(index + 1))
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
// id, listed here: http://joda-time.sourceforge.net/timezones.html
|
||||||
|
return DateTimeZone.forID(text);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,30 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* TODO should be renamed to NumericHistogram? This would also clean up the code and make it less confusing
|
||||||
|
*/
|
||||||
|
public interface Histogram extends HistogramBase<Histogram.Bucket> {
|
||||||
|
|
||||||
|
static interface Bucket extends HistogramBase.Bucket {
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,164 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import org.apache.lucene.util.CollectionUtil;
|
||||||
|
import org.elasticsearch.common.inject.internal.Nullable;
|
||||||
|
import org.elasticsearch.common.rounding.Rounding;
|
||||||
|
import org.elasticsearch.index.fielddata.LongValues;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.BucketsAggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.bucket.LongHash;
|
||||||
|
import org.elasticsearch.search.aggregations.support.AggregationContext;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValueSourceAggregatorFactory;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public class HistogramAggregator extends BucketsAggregator {
|
||||||
|
|
||||||
|
private final static int INITIAL_CAPACITY = 50; // TODO sizing
|
||||||
|
|
||||||
|
private final NumericValuesSource valuesSource;
|
||||||
|
private final Rounding rounding;
|
||||||
|
private final InternalOrder order;
|
||||||
|
private final boolean keyed;
|
||||||
|
private final boolean computeEmptyBuckets;
|
||||||
|
private final AbstractHistogramBase.Factory histogramFactory;
|
||||||
|
|
||||||
|
private final LongHash bucketOrds;
|
||||||
|
|
||||||
|
public HistogramAggregator(String name,
|
||||||
|
AggregatorFactories factories,
|
||||||
|
Rounding rounding,
|
||||||
|
InternalOrder order,
|
||||||
|
boolean keyed,
|
||||||
|
boolean computeEmptyBuckets,
|
||||||
|
@Nullable NumericValuesSource valuesSource,
|
||||||
|
AbstractHistogramBase.Factory<?> histogramFactory,
|
||||||
|
AggregationContext aggregationContext,
|
||||||
|
Aggregator parent) {
|
||||||
|
|
||||||
|
super(name, BucketAggregationMode.PER_BUCKET, factories, 50, aggregationContext, parent);
|
||||||
|
this.valuesSource = valuesSource;
|
||||||
|
this.rounding = rounding;
|
||||||
|
this.order = order;
|
||||||
|
this.keyed = keyed;
|
||||||
|
this.computeEmptyBuckets = computeEmptyBuckets;
|
||||||
|
this.histogramFactory = histogramFactory;
|
||||||
|
|
||||||
|
bucketOrds = new LongHash(INITIAL_CAPACITY);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean shouldCollect() {
|
||||||
|
return valuesSource != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc, long owningBucketOrdinal) throws IOException {
|
||||||
|
assert owningBucketOrdinal == 0;
|
||||||
|
final LongValues values = valuesSource.longValues();
|
||||||
|
final int valuesCount = values.setDocument(doc);
|
||||||
|
|
||||||
|
long previousKey = Long.MIN_VALUE;
|
||||||
|
for (int i = 0; i < valuesCount; ++i) {
|
||||||
|
long value = values.nextValue();
|
||||||
|
long key = rounding.round(value);
|
||||||
|
assert key >= previousKey;
|
||||||
|
if (key == previousKey) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
long bucketOrd = bucketOrds.add(key);
|
||||||
|
if (bucketOrd < 0) { // already seen
|
||||||
|
bucketOrd = -1 - bucketOrd;
|
||||||
|
}
|
||||||
|
collectBucket(doc, bucketOrd);
|
||||||
|
previousKey = key;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildAggregation(long owningBucketOrdinal) {
|
||||||
|
assert owningBucketOrdinal == 0;
|
||||||
|
List<HistogramBase.Bucket> buckets = new ArrayList<HistogramBase.Bucket>((int) bucketOrds.size());
|
||||||
|
for (long i = 0; i < bucketOrds.capacity(); ++i) {
|
||||||
|
final long ord = bucketOrds.id(i);
|
||||||
|
if (ord < 0) {
|
||||||
|
continue; // slot is not allocated
|
||||||
|
}
|
||||||
|
buckets.add(histogramFactory.createBucket(bucketOrds.key(i), bucketDocCount(ord), bucketAggregations(ord)));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
CollectionUtil.introSort(buckets, order.comparator());
|
||||||
|
|
||||||
|
// value source will be null for unmapped fields
|
||||||
|
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
|
||||||
|
AbstractHistogramBase.EmptyBucketInfo emptyBucketInfo = computeEmptyBuckets ? new AbstractHistogramBase.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
|
||||||
|
return histogramFactory.create(name, buckets, order, emptyBucketInfo, formatter, keyed);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public InternalAggregation buildEmptyAggregation() {
|
||||||
|
ValueFormatter formatter = valuesSource != null ? valuesSource.formatter() : null;
|
||||||
|
AbstractHistogramBase.EmptyBucketInfo emptyBucketInfo = computeEmptyBuckets ? new AbstractHistogramBase.EmptyBucketInfo(rounding, buildEmptySubAggregations()) : null;
|
||||||
|
return histogramFactory.create(name, (List<HistogramBase.Bucket>) Collections.EMPTY_LIST, order, emptyBucketInfo, formatter, keyed);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
public static class Factory extends ValueSourceAggregatorFactory<NumericValuesSource> {
|
||||||
|
|
||||||
|
private final Rounding rounding;
|
||||||
|
private final InternalOrder order;
|
||||||
|
private final boolean keyed;
|
||||||
|
private final boolean computeEmptyBuckets;
|
||||||
|
private final AbstractHistogramBase.Factory<?> histogramFactory;
|
||||||
|
|
||||||
|
public Factory(String name, ValuesSourceConfig<NumericValuesSource> valueSourceConfig,
|
||||||
|
Rounding rounding, InternalOrder order, boolean keyed, boolean computeEmptyBuckets, AbstractHistogramBase.Factory<?> histogramFactory) {
|
||||||
|
super(name, histogramFactory.type(), valueSourceConfig);
|
||||||
|
this.rounding = rounding;
|
||||||
|
this.order = order;
|
||||||
|
this.keyed = keyed;
|
||||||
|
this.computeEmptyBuckets = computeEmptyBuckets;
|
||||||
|
this.histogramFactory = histogramFactory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent) {
|
||||||
|
return new HistogramAggregator(name, factories, rounding, order, keyed, computeEmptyBuckets, null, histogramFactory, aggregationContext, parent);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Aggregator create(NumericValuesSource valuesSource, long expectedBucketsCount, AggregationContext aggregationContext, Aggregator parent) {
|
||||||
|
return new HistogramAggregator(name, factories, rounding, order, keyed, computeEmptyBuckets, valuesSource, histogramFactory, aggregationContext, parent);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,129 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregation;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A histogram get result
|
||||||
|
*/
|
||||||
|
interface HistogramBase<B extends HistogramBase.Bucket> extends Aggregation, Iterable<B> {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A bucket in the histogram where documents fall in
|
||||||
|
*/
|
||||||
|
static interface Bucket extends org.elasticsearch.search.aggregations.bucket.Bucket {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The key associated with the bucket (all documents that fall in this bucket were rounded to this key)
|
||||||
|
*/
|
||||||
|
long getKey();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
List<B> buckets();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a bucket by the key associated with it.
|
||||||
|
*
|
||||||
|
* @param key The key of the bucket.
|
||||||
|
* @return The bucket that is associated with the given key.
|
||||||
|
*/
|
||||||
|
B getByKey(long key);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A strategy defining the order in which the buckets in this histogram are ordered.
|
||||||
|
*/
|
||||||
|
static abstract class Order implements ToXContent {
|
||||||
|
|
||||||
|
public static final Order KEY_ASC = new InternalOrder((byte) 1, "_key", true, new Comparator<HistogramBase.Bucket>() {
|
||||||
|
@Override
|
||||||
|
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
|
||||||
|
if (b1.getKey() > b2.getKey()) {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
if (b1.getKey() < b2.getKey()) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
public static final Order KEY_DESC = new InternalOrder((byte) 2, "_key", false, new Comparator<HistogramBase.Bucket>() {
|
||||||
|
@Override
|
||||||
|
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
|
||||||
|
return -KEY_ASC.comparator().compare(b1, b2);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
public static final Order COUNT_ASC = new InternalOrder((byte) 3, "_count", true, new Comparator<HistogramBase.Bucket>() {
|
||||||
|
@Override
|
||||||
|
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
|
||||||
|
if (b1.getDocCount() > b2.getDocCount()) {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
if (b1.getDocCount() < b2.getDocCount()) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
|
||||||
|
public static final Order COUNT_DESC = new InternalOrder((byte) 4, "_count", false, new Comparator<HistogramBase.Bucket>() {
|
||||||
|
@Override
|
||||||
|
public int compare(HistogramBase.Bucket b1, HistogramBase.Bucket b2) {
|
||||||
|
return -COUNT_ASC.comparator().compare(b1, b2);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a bucket ordering strategy which sorts buckets based on a single-valued calc get
|
||||||
|
*
|
||||||
|
* @param aggregationName the name of the get
|
||||||
|
* @param asc The direction of the order (ascending or descending)
|
||||||
|
*/
|
||||||
|
public static InternalOrder aggregation(String aggregationName, boolean asc) {
|
||||||
|
return new InternalOrder.Aggregation(aggregationName, null, asc);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a bucket ordering strategy which sorts buckets based on a multi-valued calc get
|
||||||
|
*
|
||||||
|
* @param aggregationName the name of the get
|
||||||
|
* @param valueName The name of the value of the multi-value get by which the sorting will be applied
|
||||||
|
* @param asc The direction of the order (ascending or descending)
|
||||||
|
*/
|
||||||
|
public static InternalOrder aggregation(String aggregationName, String valueName, boolean asc) {
|
||||||
|
return new InternalOrder.Aggregation(aggregationName, valueName, asc);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The bucket comparator by which the order will be applied.
|
||||||
|
*/
|
||||||
|
abstract Comparator<Bucket> comparator();
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,56 @@
|
||||||
|
package org.elasticsearch.search.aggregations.bucket.histogram;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.search.aggregations.ValuesSourceAggregationBuilder;
|
||||||
|
import org.elasticsearch.search.builder.SearchSourceBuilderException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class HistogramBuilder extends ValuesSourceAggregationBuilder<HistogramBuilder> {
|
||||||
|
|
||||||
|
private Long interval;
|
||||||
|
private HistogramBase.Order order;
|
||||||
|
private Boolean computeEmptyBuckets;
|
||||||
|
|
||||||
|
public HistogramBuilder(String name) {
|
||||||
|
super(name, InternalHistogram.TYPE.name());
|
||||||
|
}
|
||||||
|
|
||||||
|
public HistogramBuilder interval(long interval) {
|
||||||
|
this.interval = interval;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HistogramBuilder order(Histogram.Order order) {
|
||||||
|
this.order = order;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HistogramBuilder emptyBuckets(boolean computeEmptyBuckets) {
|
||||||
|
this.computeEmptyBuckets = computeEmptyBuckets;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected XContentBuilder doInternalXContent(XContentBuilder builder, Params params) throws IOException {
|
||||||
|
if (interval == null) {
|
||||||
|
throw new SearchSourceBuilderException("[interval] must be defined for histogram aggregation [" + name + "]");
|
||||||
|
}
|
||||||
|
builder.field("interval", interval);
|
||||||
|
|
||||||
|
if (order != null) {
|
||||||
|
builder.field("order");
|
||||||
|
order.toXContent(builder, params);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (computeEmptyBuckets != null) {
|
||||||
|
builder.field("empty_buckets", computeEmptyBuckets);
|
||||||
|
}
|
||||||
|
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,157 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.rounding.Rounding;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentParser;
|
||||||
|
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||||
|
import org.elasticsearch.index.mapper.FieldMapper;
|
||||||
|
import org.elasticsearch.search.SearchParseException;
|
||||||
|
import org.elasticsearch.search.aggregations.Aggregator;
|
||||||
|
import org.elasticsearch.search.aggregations.support.FieldContext;
|
||||||
|
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.NumericValuesSource;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregatorFactory;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
|
||||||
|
import org.elasticsearch.search.internal.SearchContext;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Parses the histogram request
|
||||||
|
*/
|
||||||
|
public class HistogramParser implements Aggregator.Parser {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String type() {
|
||||||
|
return InternalHistogram.TYPE.name();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
|
||||||
|
|
||||||
|
ValuesSourceConfig<NumericValuesSource> config = new ValuesSourceConfig<NumericValuesSource>(NumericValuesSource.class);
|
||||||
|
|
||||||
|
String field = null;
|
||||||
|
String script = null;
|
||||||
|
String scriptLang = null;
|
||||||
|
Map<String, Object> scriptParams = null;
|
||||||
|
boolean keyed = false;
|
||||||
|
boolean emptyBuckets = false;
|
||||||
|
InternalOrder order = (InternalOrder) InternalOrder.KEY_ASC;
|
||||||
|
long interval = -1;
|
||||||
|
boolean assumeSorted = false;
|
||||||
|
String format = null;
|
||||||
|
|
||||||
|
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 (token == XContentParser.Token.VALUE_STRING) {
|
||||||
|
if ("field".equals(currentFieldName)) {
|
||||||
|
field = parser.text();
|
||||||
|
} else if ("script".equals(currentFieldName)) {
|
||||||
|
script = parser.text();
|
||||||
|
} else if ("script_lang".equals(currentFieldName) || "scriptLang".equals(currentFieldName)) {
|
||||||
|
scriptLang = parser.text();
|
||||||
|
} else if ("format".equals(currentFieldName)) {
|
||||||
|
format = parser.text();
|
||||||
|
}
|
||||||
|
} else if (token == XContentParser.Token.VALUE_NUMBER) {
|
||||||
|
if ("interval".equals(currentFieldName)) {
|
||||||
|
interval = parser.longValue();
|
||||||
|
}
|
||||||
|
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
|
||||||
|
if ("keyed".equals(currentFieldName)) {
|
||||||
|
keyed = parser.booleanValue();
|
||||||
|
} else if ("empty_buckets".equals(currentFieldName) || "emptyBuckets".equals(currentFieldName)) {
|
||||||
|
emptyBuckets = parser.booleanValue();
|
||||||
|
} else if ("script_values_sorted".equals(currentFieldName)) {
|
||||||
|
assumeSorted = parser.booleanValue();
|
||||||
|
}
|
||||||
|
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||||
|
if ("params".equals(currentFieldName)) {
|
||||||
|
scriptParams = parser.map();
|
||||||
|
} else if ("order".equals(currentFieldName)) {
|
||||||
|
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||||
|
if (token == XContentParser.Token.FIELD_NAME) {
|
||||||
|
currentFieldName = parser.currentName();
|
||||||
|
} else if (token == XContentParser.Token.VALUE_STRING) {
|
||||||
|
String dir = parser.text();
|
||||||
|
boolean asc = "asc".equals(dir);
|
||||||
|
order = resolveOrder(currentFieldName, asc);
|
||||||
|
//TODO should we throw an error if the value is not "asc" or "desc"???
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (interval < 0) {
|
||||||
|
throw new SearchParseException(context, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]");
|
||||||
|
}
|
||||||
|
Rounding rounding = new Rounding.Interval(interval);
|
||||||
|
|
||||||
|
if (script != null) {
|
||||||
|
config.script(context.scriptService().search(context.lookup(), scriptLang, script, scriptParams));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!assumeSorted) {
|
||||||
|
// we need values to be sorted and unique for efficiency
|
||||||
|
config.ensureSorted(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (field == null) {
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, emptyBuckets, InternalHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
FieldMapper<?> mapper = context.smartNameFieldMapper(field);
|
||||||
|
if (mapper == null) {
|
||||||
|
config.unmapped(true);
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, emptyBuckets, InternalHistogram.FACTORY);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexFieldData<?> indexFieldData = context.fieldData().getForField(mapper);
|
||||||
|
config.fieldContext(new FieldContext(field, indexFieldData));
|
||||||
|
|
||||||
|
if (format != null) {
|
||||||
|
config.formatter(new ValueFormatter.Number.Pattern(format));
|
||||||
|
}
|
||||||
|
|
||||||
|
return new HistogramAggregator.Factory(aggregationName, config, rounding, order, keyed, emptyBuckets, InternalHistogram.FACTORY);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
static InternalOrder resolveOrder(String key, boolean asc) {
|
||||||
|
if ("_key".equals(key)) {
|
||||||
|
return (InternalOrder) (asc ? InternalOrder.KEY_ASC : InternalOrder.KEY_DESC);
|
||||||
|
}
|
||||||
|
if ("_count".equals(key)) {
|
||||||
|
return (InternalOrder) (asc ? InternalOrder.COUNT_ASC : InternalOrder.COUNT_DESC);
|
||||||
|
}
|
||||||
|
int i = key.indexOf('.');
|
||||||
|
if (i < 0) {
|
||||||
|
return HistogramBase.Order.aggregation(key, asc);
|
||||||
|
}
|
||||||
|
return HistogramBase.Order.aggregation(key.substring(0, i), key.substring(i + 1), asc);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* Licensed to ElasticSearch and Shay Banon 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.bucket.histogram;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationStreams;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
||||||
|
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||||
|
import org.elasticsearch.search.aggregations.support.numeric.ValueFormatter;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
public class InternalDateHistogram extends AbstractHistogramBase<DateHistogram.Bucket> implements DateHistogram {
|
||||||
|
|
||||||
|
public final static Type TYPE = new Type("date_histogram", "dhisto");
|
||||||
|
public final static Factory FACTORY = new Factory();
|
||||||
|
|
||||||
|
private final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
|
||||||
|
@Override
|
||||||
|
public InternalDateHistogram readResult(StreamInput in) throws IOException {
|
||||||
|
InternalDateHistogram histogram = new InternalDateHistogram();
|
||||||
|
histogram.readFrom(in);
|
||||||
|
return histogram;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public static void registerStream() {
|
||||||
|
AggregationStreams.registerStream(STREAM, TYPE.stream());
|
||||||
|
}
|
||||||
|
|
||||||
|
static class Bucket extends AbstractHistogramBase.Bucket implements DateHistogram.Bucket {
|
||||||
|
|
||||||
|
Bucket(long key, long docCount, InternalAggregations aggregations) {
|
||||||
|
super(key, docCount, aggregations);
|
||||||
|
}
|
||||||
|
|
||||||
|
Bucket(long key, long docCount, List<InternalAggregation> aggregations) {
|
||||||
|
super(key, docCount, new InternalAggregations(aggregations));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DateTime getKeyAsDate() {
|
||||||
|
return new DateTime(getKey());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class Factory implements AbstractHistogramBase.Factory<DateHistogram.Bucket> {
|
||||||
|
|
||||||
|
private Factory() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String type() {
|
||||||
|
return TYPE.name();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AbstractHistogramBase<?> create(String name, List<DateHistogram.Bucket> buckets, InternalOrder order, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
|
||||||
|
return new InternalDateHistogram(name, buckets, order, emptyBucketInfo, formatter, keyed);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AbstractHistogramBase.Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
|
||||||
|
return new Bucket(key, docCount, aggregations);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
InternalDateHistogram() {} // for serialization
|
||||||
|
|
||||||
|
InternalDateHistogram(String name, List<DateHistogram.Bucket> buckets, InternalOrder order, EmptyBucketInfo emptyBucketInfo, ValueFormatter formatter, boolean keyed) {
|
||||||
|
super(name, buckets, order, emptyBucketInfo, formatter, keyed);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Type type() {
|
||||||
|
return TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected DateHistogram.Bucket createBucket(long key, long docCount, InternalAggregations aggregations) {
|
||||||
|
return new Bucket(key, docCount, aggregations);
|
||||||
|
}
|
||||||
|
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue