diff --git a/docs/java-api/bulk.asciidoc b/docs/java-api/bulk.asciidoc index 9ac61f47f30..96b0b2eb6dc 100644 --- a/docs/java-api/bulk.asciidoc +++ b/docs/java-api/bulk.asciidoc @@ -99,3 +99,22 @@ By default, `BulkProcessor`: * does not set flushInterval * sets concurrentRequests to 1 +When all documents are loaded to the `BulkProcessor` it can be closed by using `awaitClose` or `close` methods: + +[source,java] +-------------------------------------------------- +bulkProcessor.awaitClose(10, TimeUnit.MINUTES); +-------------------------------------------------- + +or + +[source,java] +-------------------------------------------------- +bulkProcessor.close(); +-------------------------------------------------- + +Both methods flush any remaining documents and disable all other scheduled flushes if they were scheduled by setting +`flushInterval`. If concurrent requests were enabled the `awaitClose` method waits for up to the specified timeout for +all bulk requests to complete then returns `true`, if the specified waiting time elapses before all bulk requests complete, +`false` is returned. The `close` method doesn't wait for any remaining bulk requests to complete and exists immediately. + diff --git a/docs/reference/images/reducers_movavg/double_0.2beta.png b/docs/reference/images/reducers_movavg/double_0.2beta.png new file mode 100644 index 00000000000..64499b98342 Binary files /dev/null and b/docs/reference/images/reducers_movavg/double_0.2beta.png differ diff --git a/docs/reference/images/reducers_movavg/double_0.7beta.png b/docs/reference/images/reducers_movavg/double_0.7beta.png new file mode 100644 index 00000000000..b9f530227d9 Binary files /dev/null and b/docs/reference/images/reducers_movavg/double_0.7beta.png differ diff --git a/docs/reference/images/reducers_movavg/double_prediction_global.png b/docs/reference/images/reducers_movavg/double_prediction_global.png new file mode 100644 index 00000000000..faee6d22bc2 Binary files /dev/null and b/docs/reference/images/reducers_movavg/double_prediction_global.png differ diff --git a/docs/reference/images/reducers_movavg/double_prediction_local.png b/docs/reference/images/reducers_movavg/double_prediction_local.png new file mode 100644 index 00000000000..930a5cfde9b Binary files /dev/null and b/docs/reference/images/reducers_movavg/double_prediction_local.png differ diff --git a/docs/reference/images/reducers_movavg/linear_100window.png b/docs/reference/images/reducers_movavg/linear_100window.png new file mode 100644 index 00000000000..3a4d51ae956 Binary files /dev/null and b/docs/reference/images/reducers_movavg/linear_100window.png differ diff --git a/docs/reference/images/reducers_movavg/linear_10window.png b/docs/reference/images/reducers_movavg/linear_10window.png new file mode 100644 index 00000000000..1407ded8791 Binary files /dev/null and b/docs/reference/images/reducers_movavg/linear_10window.png differ diff --git a/docs/reference/images/reducers_movavg/movavg_100window.png b/docs/reference/images/reducers_movavg/movavg_100window.png new file mode 100644 index 00000000000..45094ec2681 Binary files /dev/null and b/docs/reference/images/reducers_movavg/movavg_100window.png differ diff --git a/docs/reference/images/reducers_movavg/movavg_10window.png b/docs/reference/images/reducers_movavg/movavg_10window.png new file mode 100644 index 00000000000..1e9f543385f Binary files /dev/null and b/docs/reference/images/reducers_movavg/movavg_10window.png differ diff --git a/docs/reference/images/reducers_movavg/simple_prediction.png b/docs/reference/images/reducers_movavg/simple_prediction.png new file mode 100644 index 00000000000..d74724e1546 Binary files /dev/null and b/docs/reference/images/reducers_movavg/simple_prediction.png differ diff --git a/docs/reference/images/reducers_movavg/single_0.2alpha.png b/docs/reference/images/reducers_movavg/single_0.2alpha.png new file mode 100644 index 00000000000..d96cf771743 Binary files /dev/null and b/docs/reference/images/reducers_movavg/single_0.2alpha.png differ diff --git a/docs/reference/images/reducers_movavg/single_0.7alpha.png b/docs/reference/images/reducers_movavg/single_0.7alpha.png new file mode 100644 index 00000000000..bf7bdd1752e Binary files /dev/null and b/docs/reference/images/reducers_movavg/single_0.7alpha.png differ diff --git a/docs/reference/mapping/types/core-types.asciidoc b/docs/reference/mapping/types/core-types.asciidoc index e650ded89ca..1f6dcc01cb5 100644 --- a/docs/reference/mapping/types/core-types.asciidoc +++ b/docs/reference/mapping/types/core-types.asciidoc @@ -426,6 +426,9 @@ and it can be retrieved from it). in `_source`, have `include_in_all` enabled, or `store` be set to `true` for this to be useful. +|`doc_values` |Set to `true` to store field values in a column-stride fashion. +Automatically set to `true` when the fielddata format is `doc_values`. + |`boost` |The boost value. Defaults to `1.0`. |`null_value` |When there is a (JSON) null value for the field, use the diff --git a/docs/reference/search/aggregations.asciidoc b/docs/reference/search/aggregations.asciidoc index 6733177b3a6..cf4b4348eda 100644 --- a/docs/reference/search/aggregations.asciidoc +++ b/docs/reference/search/aggregations.asciidoc @@ -118,6 +118,38 @@ aggregated for 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. +[float] +=== Reducer Aggregations + +coming[2.0.0] + +experimental[] + +Reducer aggregations work on the outputs produced from other aggregations rather than from document sets, adding +information to the output tree. There are many different types of reducer, each computing different information from +other aggregations, but these types can broken down into two families: + +_Parent_:: + A family of reducer aggregations that is provided with the output of its parent aggregation and is able + to compute new buckets or new aggregations to add to existing buckets. + +_Sibling_:: + Reducer aggregations that are provided with the output of a sibling aggregation and are able to compute a + new aggregation which will be at the same level as the sibling aggregation. + +Reducer aggregations can reference the aggregations they need to perform their computation by using the `buckets_paths` +parameter to indicate the paths to the required metrics. The syntax for defining these paths can be found in the +<> section. + +?????? SHOULD THE SECTION ABOUT DEFINING AGGREGATION PATHS +BE IN THIS PAGE AND REFERENCED FROM THE TERMS AGGREGATION DOCUMENTATION ??????? + +Reducer aggregations cannot have sub-aggregations but depending on the type it can reference another reducer in the `buckets_path` +allowing reducers to be chained. + +NOTE: Because reducer aggregations only add to the output, when chaining reducer aggregations the output of each reducer will be +included in the final output. + [float] === Caching heavy aggregations @@ -197,3 +229,6 @@ Then that piece of metadata will be returned in place for our `titles` terms agg include::aggregations/metrics.asciidoc[] include::aggregations/bucket.asciidoc[] + +include::aggregations/reducer.asciidoc[] + diff --git a/docs/reference/search/aggregations/reducer.asciidoc b/docs/reference/search/aggregations/reducer.asciidoc new file mode 100644 index 00000000000..11b0826e9eb --- /dev/null +++ b/docs/reference/search/aggregations/reducer.asciidoc @@ -0,0 +1,5 @@ +[[search-aggregations-reducer]] + +include::reducer/derivative-aggregation.asciidoc[] +include::reducer/max-bucket-aggregation.asciidoc[] +include::reducer/movavg-aggregation.asciidoc[] diff --git a/docs/reference/search/aggregations/reducer/derivative-aggregation.asciidoc b/docs/reference/search/aggregations/reducer/derivative-aggregation.asciidoc new file mode 100644 index 00000000000..8369d0c1ba0 --- /dev/null +++ b/docs/reference/search/aggregations/reducer/derivative-aggregation.asciidoc @@ -0,0 +1,194 @@ +[[search-aggregations-reducer-derivative-aggregation]] +=== Derivative Aggregation + +A parent reducer aggregation which calculates the derivative of a specified metric in a parent histogram (or date_histogram) +aggregation. The specified metric must be numeric and the enclosing histogram must have `min_doc_count` set to `0`. + +The following snippet calculates the derivative of the total monthly `sales`: + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "sales_per_month" : { + "date_histogram" : { + "field" : "date", + "interval" : "month", + "min_doc_count" : 0 + }, + "aggs": { + "sales": { + "sum": { + "field": "price" + } + }, + "sales_deriv": { + "derivative": { + "buckets_paths": "sales" <1> + } + } + } + } + } +} +-------------------------------------------------- + +<1> `bucket_paths` instructs this derivative aggregation to use the output of the `sales` aggregation for the derivative + +And the following may be the response: + +[source,js] +-------------------------------------------------- +{ + "aggregations": { + "sales_per_month": { + "buckets": [ + { + "key_as_string": "2015/01/01 00:00:00", + "key": 1420070400000, + "doc_count": 3, + "sales": { + "value": 550 + } <1> + }, + { + "key_as_string": "2015/02/01 00:00:00", + "key": 1422748800000, + "doc_count": 2, + "sales": { + "value": 60 + }, + "sales_deriv": { + "value": -490 <2> + } + }, + { + "key_as_string": "2015/03/01 00:00:00", + "key": 1425168000000, + "doc_count": 2, <3> + "sales": { + "value": 375 + }, + "sales_deriv": { + "value": 315 + } + } + ] + } + } +} +-------------------------------------------------- + +<1> No derivative for the first bucket since we need at least 2 data points to calculate the derivative +<2> Derivative value units are implicitly defined by the `sales` aggregation and the parent histogram so in this case the units +would be $/month assuming the `price` field has units of $. +<3> The number of documents in the bucket are represented by the `doc_count` value + +==== Second Order Derivative + +A second order derivative can be calculated by chaining the derivative reducer aggregation onto the result of another derivative +reducer aggregation as in the following example which will calculate both the first and the second order derivative of the total +monthly sales: + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "sales_per_month" : { + "date_histogram" : { + "field" : "date", + "interval" : "month" + }, + "aggs": { + "sales": { + "sum": { + "field": "price" + } + }, + "sales_deriv": { + "derivative": { + "buckets_paths": "sales" + } + }, + "sales_2nd_deriv": { + "derivative": { + "buckets_paths": "sales_deriv" <1> + } + } + } + } + } +} +-------------------------------------------------- + +<1> `bucket_paths` for the second derivative points to the name of the first derivative + +And the following may be the response: + +[source,js] +-------------------------------------------------- +{ + "aggregations": { + "sales_per_month": { + "buckets": [ + { + "key_as_string": "2015/01/01 00:00:00", + "key": 1420070400000, + "doc_count": 3, + "sales": { + "value": 550 + } <1> + }, + { + "key_as_string": "2015/02/01 00:00:00", + "key": 1422748800000, + "doc_count": 2, + "sales": { + "value": 60 + }, + "sales_deriv": { + "value": -490 + } <1> + }, + { + "key_as_string": "2015/03/01 00:00:00", + "key": 1425168000000, + "doc_count": 2, + "sales": { + "value": 375 + }, + "sales_deriv": { + "value": 315 + }, + "sales_2nd_deriv": { + "value": 805 + } + } + ] + } + } +} +-------------------------------------------------- +<1> No second derivative for the first two buckets since we need at least 2 data points from the first derivative to calculate the +second derivative + +==== Dealing with gaps in the data + +There are a couple of reasons why the data output by the enclosing histogram may have gaps: + +* There are no documents matching the query for some buckets +* The data for a metric is missing in all of the documents falling into a bucket (this is most likely with either a small interval +on the enclosing histogram or with a query matching only a small number of documents) + +Where there is no data available in a bucket for a given metric it presents a problem for calculating the derivative value for both +the current bucket and the next bucket. In the derivative reducer aggregation has a `gap_policy` parameter to define what the behavior +should be when a gap in the data is found. There are currently two options for controlling the gap policy: + +_ignore_:: + This option will not produce a derivative value for any buckets where the value in the current or previous bucket is + missing + +_insert_zeros_:: + This option will assume the missing value is `0` and calculate the derivative with the value `0`. + + diff --git a/docs/reference/search/aggregations/reducer/max-bucket-aggregation.asciidoc b/docs/reference/search/aggregations/reducer/max-bucket-aggregation.asciidoc new file mode 100644 index 00000000000..a93c7ed8036 --- /dev/null +++ b/docs/reference/search/aggregations/reducer/max-bucket-aggregation.asciidoc @@ -0,0 +1,82 @@ +[[search-aggregations-reducer-max-bucket-aggregation]] +=== Max Bucket Aggregation + +A sibling reducer aggregation which identifies the bucket(s) with the maximum value of a specified metric in a sibing aggregation +and outputs both the value and the key(s) of the bucket(s). The specified metric must be numeric and the sibling aggregation must +be a multi-bucket aggregation. + +The following snippet calculates the maximum of the total monthly `sales`: + +[source,js] +-------------------------------------------------- +{ + "aggs" : { + "sales_per_month" : { + "date_histogram" : { + "field" : "date", + "interval" : "month" + }, + "aggs": { + "sales": { + "sum": { + "field": "price" + } + } + } + }, + "max_monthly_sales": { + "max_bucket": { + "buckets_paths": "sales_per_month>sales" <1> + } + } + } +} +-------------------------------------------------- + +<1> `bucket_paths` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the +`sales_per_month` date histogram. + +And the following may be the response: + +[source,js] +-------------------------------------------------- +{ + "aggregations": { + "sales_per_month": { + "buckets": [ + { + "key_as_string": "2015/01/01 00:00:00", + "key": 1420070400000, + "doc_count": 3, + "sales": { + "value": 550 + } + }, + { + "key_as_string": "2015/02/01 00:00:00", + "key": 1422748800000, + "doc_count": 2, + "sales": { + "value": 60 + } + }, + { + "key_as_string": "2015/03/01 00:00:00", + "key": 1425168000000, + "doc_count": 2, + "sales": { + "value": 375 + } + } + ] + }, + "max_monthly_sales": { + "keys": ["2015/01/01 00:00:00"], <1> + "value": 550 + } + } +} +-------------------------------------------------- + +<1> `keys` is an array of strings since the maximum value may be present in multiple buckets + diff --git a/docs/reference/search/aggregations/reducer/movavg-aggregation.asciidoc b/docs/reference/search/aggregations/reducer/movavg-aggregation.asciidoc new file mode 100644 index 00000000000..9b2f89ca43e --- /dev/null +++ b/docs/reference/search/aggregations/reducer/movavg-aggregation.asciidoc @@ -0,0 +1,297 @@ +[[search-aggregations-reducers-movavg-reducer]] +=== Moving Average Aggregation + +Given an ordered series of data, the Moving Average aggregation will slide a window across the data and emit the average +value of that window. For example, given the data `[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]`, we can calculate a simple moving +average with windows size of `5` as follows: + +- (1 + 2 + 3 + 4 + 5) / 5 = 3 +- (2 + 3 + 4 + 5 + 6) / 5 = 4 +- (3 + 4 + 5 + 6 + 7) / 5 = 5 +- etc + +Moving averages are a simple method to smooth sequential data. Moving averages are typically applied to time-based data, +such as stock prices or server metrics. The smoothing can be used to eliminate high frequency fluctuations or random noise, +which allows the lower frequency trends to be more easily visualized, such as seasonality. + +==== Syntax + +A `moving_avg` aggregation looks like this in isolation: + +[source,js] +-------------------------------------------------- +{ + "movavg": { + "buckets_path": "the_sum", + "model": "double_exp", + "window": 5, + "gap_policy": "insert_zero", + "settings": { + "alpha": 0.8 + } + } +} +-------------------------------------------------- + +.`moving_avg` Parameters +|=== +|Parameter Name |Description |Required |Default + +|`buckets_path` |The path to the metric that we wish to calculate a moving average for |Required | +|`model` |The moving average weighting model that we wish to use |Optional |`simple` +|`gap_policy` |Determines what should happen when a gap in the data is encountered. |Optional |`insert_zero` +|`window` |The size of window to "slide" across the histogram. |Optional |`5` +|`settings` |Model-specific settings, contents which differ depending on the model specified. |Optional | +|=== + + +`moving_avg` aggregations must be embedded inside of a `histogram` or `date_histogram` aggregation. They can be +embedded like any other metric aggregation: + +[source,js] +-------------------------------------------------- +{ + "my_date_histo":{ <1> + "date_histogram":{ + "field":"timestamp", + "interval":"day", + "min_doc_count": 0 <2> + }, + "aggs":{ + "the_sum":{ + "sum":{ "field": "lemmings" } <3> + }, + "the_movavg":{ + "moving_avg":{ "buckets_path": "the_sum" } <4> + } + } + } +} +-------------------------------------------------- +<1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals +<2> We must specify "min_doc_count: 0" in our date histogram that all buckets are returned, even if they are empty. +<3> A `sum` metric is used to calculate the sum of a field. This could be any metric (sum, min, max, etc) +<4> Finally, we specify a `moving_avg` aggregation which uses "the_sum" metric as its input. + +Moving averages are built by first specifying a `histogram` or `date_histogram` over a field. You can then optionally +add normal metrics, such as a `sum`, inside of that histogram. Finally, the `moving_avg` is embedded inside the histogram. +The `buckets_path` parameter is then used to "point" at one of the sibling metrics inside of the histogram. + +A moving average can also be calculated on the document count of each bucket, instead of a metric: + +[source,js] +-------------------------------------------------- +{ + "my_date_histo":{ + "date_histogram":{ + "field":"timestamp", + "interval":"day", + "min_doc_count": 0 + }, + "aggs":{ + "the_movavg":{ + "moving_avg":{ "buckets_path": "_count" } <1> + } + } + } +} +-------------------------------------------------- +<1> By using `_count` instead of a metric name, we can calculate the moving average of document counts in the histogram + +==== Models + +The `moving_avg` aggregation includes four different moving average "models". The main difference is how the values in the +window are weighted. As data-points become "older" in the window, they may be weighted differently. This will +affect the final average for that window. + +Models are specified using the `model` parameter. Some models may have optional configurations which are specified inside +the `settings` parameter. + +===== Simple + +The `simple` model calculates the sum of all values in the window, then divides by the size of the window. It is effectively +a simple arithmetic mean of the window. The simple model does not perform any time-dependent weighting, which means +the values from a `simple` moving average tend to "lag" behind the real data. + +[source,js] +-------------------------------------------------- +{ + "the_movavg":{ + "moving_avg":{ + "buckets_path": "the_sum", + "model" : "simple" + } + } +} +-------------------------------------------------- + +A `simple` model has no special settings to configure + +The window size can change the behavior of the moving average. For example, a small window (`"window": 10`) will closely +track the data and only smooth out small scale fluctuations: + +[[movavg_10window]] +.Moving average with window of size 10 +image::images/reducers_movavg/movavg_10window.png[] + +In contrast, a `simple` moving average with larger window (`"window": 100`) will smooth out all higher-frequency fluctuations, +leaving only low-frequency, long term trends. It also tends to "lag" behind the actual data by a substantial amount: + +[[movavg_100window]] +.Moving average with window of size 100 +image::images/reducers_movavg/movavg_100window.png[] + + +==== Linear + +The `linear` model assigns a linear weighting to points in the series, such that "older" datapoints (e.g. those at +the beginning of the window) contribute a linearly less amount to the total average. The linear weighting helps reduce +the "lag" behind the data's mean, since older points have less influence. + +[source,js] +-------------------------------------------------- +{ + "the_movavg":{ + "moving_avg":{ + "buckets_path": "the_sum", + "model" : "linear" + } +} +-------------------------------------------------- + +A `linear` model has no special settings to configure + +Like the `simple` model, window size can change the behavior of the moving average. For example, a small window (`"window": 10`) +will closely track the data and only smooth out small scale fluctuations: + +[[linear_10window]] +.Linear moving average with window of size 10 +image::images/reducers_movavg/linear_10window.png[] + +In contrast, a `linear` moving average with larger window (`"window": 100`) will smooth out all higher-frequency fluctuations, +leaving only low-frequency, long term trends. It also tends to "lag" behind the actual data by a substantial amount, +although typically less than the `simple` model: + +[[linear_100window]] +.Linear moving average with window of size 100 +image::images/reducers_movavg/linear_100window.png[] + +==== Single Exponential + +The `single_exp` model is similar to the `linear` model, except older data-points become exponentially less important, +rather than linearly less important. The speed at which the importance decays can be controlled with an `alpha` +setting. Small values make the weight decay slowly, which provides greater smoothing and takes into account a larger +portion of the window. Larger valuers make the weight decay quickly, which reduces the impact of older values on the +moving average. This tends to make the moving average track the data more closely but with less smoothing. + +The default value of `alpha` is `0.5`, and the setting accepts any float from 0-1 inclusive. + +[source,js] +-------------------------------------------------- +{ + "the_movavg":{ + "moving_avg":{ + "buckets_path": "the_sum", + "model" : "single_exp", + "settings" : { + "alpha" : 0.5 + } + } +} +-------------------------------------------------- + + + +[[single_0.2alpha]] +.Single Exponential moving average with window of size 10, alpha = 0.2 +image::images/reducers_movavg/single_0.2alpha.png[] + +[[single_0.7alpha]] +.Single Exponential moving average with window of size 10, alpha = 0.7 +image::images/reducers_movavg/single_0.7alpha.png[] + +==== Double Exponential + +The `double_exp` model, sometimes called "Holt's Linear Trend" model, incorporates a second exponential term which +tracks the data's trend. Single exponential does not perform well when the data has an underlying linear trend. The +double exponential model calculates two values internally: a "level" and a "trend". + +The level calculation is similar to `single_exp`, and is an exponentially weighted view of the data. The difference is +that the previously smoothed value is used instead of the raw value, which allows it to stay close to the original series. +The trend calculation looks at the difference between the current and last value (e.g. the slope, or trend, of the +smoothed data). The trend value is also exponentially weighted. + +Values are produced by multiplying the level and trend components. + +The default value of `alpha` and `beta` is `0.5`, and the settings accept any float from 0-1 inclusive. + +[source,js] +-------------------------------------------------- +{ + "the_movavg":{ + "moving_avg":{ + "buckets_path": "the_sum", + "model" : "double_exp", + "settings" : { + "alpha" : 0.5, + "beta" : 0.5 + } + } +} +-------------------------------------------------- + +In practice, the `alpha` value behaves very similarly in `double_exp` as `single_exp`: small values produce more smoothing +and more lag, while larger values produce closer tracking and less lag. The value of `beta` is often difficult +to see. Small values emphasize long-term trends (such as a constant linear trend in the whole series), while larger +values emphasize short-term trends. This will become more apparently when you are predicting values. + +[[double_0.2beta]] +.Double Exponential moving average with window of size 100, alpha = 0.5, beta = 0.2 +image::images/reducers_movavg/double_0.2beta.png[] + +[[double_0.7beta]] +.Double Exponential moving average with window of size 100, alpha = 0.5, beta = 0.7 +image::images/reducers_movavg/double_0.7beta.png[] + +=== Prediction + +All the moving average model support a "prediction" mode, which will attempt to extrapolate into the future given the +current smoothed, moving average. Depending on the model and parameter, these predictions may or may not be accurate. + +Predictions are enabled by adding a `predict` parameter to any moving average aggregation, specifying the nubmer of +predictions you would like appended to the end of the series. These predictions will be spaced out at the same interval +as your buckets: + +[source,js] +-------------------------------------------------- +{ + "the_movavg":{ + "moving_avg":{ + "buckets_path": "the_sum", + "model" : "simple", + "predict" 10 + } +} +-------------------------------------------------- + +The `simple`, `linear` and `single_exp` models all produce "flat" predictions: they essentially converge on the mean +of the last value in the series, producing a flat: + +[[simple_prediction]] +.Simple moving average with window of size 10, predict = 50 +image::images/reducers_movavg/simple_prediction.png[] + +In contrast, the `double_exp` model can extrapolate based on local or global constant trends. If we set a high `beta` +value, we can extrapolate based on local constant trends (in this case the predictions head down, because the data at the end +of the series was heading in a downward direction): + +[[double_prediction_local]] +.Double Exponential moving average with window of size 100, predict = 20, alpha = 0.5, beta = 0.8 +image::images/reducers_movavg/double_prediction_local.png[] + +In contrast, if we choose a small `beta`, the predictions are based on the global constant trend. In this series, the +global trend is slightly positive, so the prediction makes a sharp u-turn and begins a positive slope: + +[[double_prediction_global]] +.Double Exponential moving average with window of size 100, predict = 20, alpha = 0.5, beta = 0.1 +image::images/reducers_movavg/double_prediction_global.png[] \ No newline at end of file diff --git a/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java b/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java index c87f9144709..d55374cd5b9 100644 --- a/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java +++ b/src/main/java/org/apache/lucene/queryparser/classic/MapperQueryParser.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.core.DateFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.support.QueryParsers; @@ -131,9 +132,6 @@ public class MapperQueryParser extends QueryParser { setFuzzyMinSim(settings.fuzzyMinSim()); setFuzzyPrefixLength(settings.fuzzyPrefixLength()); setLocale(settings.locale()); - if (settings.timeZone() != null) { - setTimeZone(settings.timeZone().toTimeZone()); - } this.analyzeWildcard = settings.analyzeWildcard(); } @@ -377,7 +375,14 @@ public class MapperQueryParser extends QueryParser { } try { - return currentMapper.rangeQuery(part1, part2, startInclusive, endInclusive, parseContext); + Query rangeQuery; + if (currentMapper instanceof DateFieldMapper && settings.timeZone() != null) { + DateFieldMapper dateFieldMapper = (DateFieldMapper) this.currentMapper; + rangeQuery = dateFieldMapper.rangeQuery(part1, part2, startInclusive, endInclusive, settings.timeZone(), null, parseContext); + } else { + rangeQuery = currentMapper.rangeQuery(part1, part2, startInclusive, endInclusive, parseContext); + } + return rangeQuery; } catch (RuntimeException e) { if (settings.lenient()) { return null; diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java b/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java index 79b31f620d5..28f9cb1db90 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponse.java @@ -68,7 +68,7 @@ public class ClusterRerouteResponse extends AcknowledgedResponse { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - ClusterState.Builder.writeTo(state, out); + state.writeTo(out); writeAcknowledged(out); RoutingExplanations.writeTo(explanations, out); } diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java b/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java index 861a84a9e71..e9aa9b723fa 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/state/ClusterStateResponse.java @@ -62,6 +62,6 @@ public class ClusterStateResponse extends ActionResponse { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); clusterName.writeTo(out); - ClusterState.Builder.writeTo(clusterState, out); + clusterState.writeTo(out); } } diff --git a/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java b/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java index 7b114c92d43..5c8905fd97b 100644 --- a/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/src/main/java/org/elasticsearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.admin.cluster.state; -import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; @@ -29,7 +28,6 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; -import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData.Custom; @@ -39,11 +37,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.util.List; - -import static com.google.common.collect.Lists.newArrayList; -import static org.elasticsearch.cluster.metadata.MetaData.lookupFactorySafe; - /** * */ @@ -84,6 +77,7 @@ public class TransportClusterStateAction extends TransportMasterNodeReadOperatio logger.trace("Serving cluster state request using version {}", currentState.version()); ClusterState.Builder builder = ClusterState.builder(currentState.getClusterName()); builder.version(currentState.version()); + builder.uuid(currentState.uuid()); if (request.nodes()) { builder.nodes(currentState.nodes()); } @@ -122,10 +116,9 @@ public class TransportClusterStateAction extends TransportMasterNodeReadOperatio } // Filter our metadata that shouldn't be returned by API - for(ObjectCursor type : currentState.metaData().customs().keys()) { - Custom.Factory factory = lookupFactorySafe(type.value); - if(!factory.context().contains(MetaData.XContentContext.API)) { - mdBuilder.removeCustom(type.value); + for(ObjectObjectCursor custom : currentState.metaData().customs()) { + if(!custom.value.context().contains(MetaData.XContentContext.API)) { + mdBuilder.removeCustom(custom.key); } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java index 765a9395afc..106e864a367 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/alias/get/GetAliasesResponse.java @@ -74,7 +74,7 @@ public class GetAliasesResponse extends ActionResponse { out.writeString(entry.key); out.writeVInt(entry.value.size()); for (AliasMetaData aliasMetaData : entry.value) { - AliasMetaData.Builder.writeTo(aliasMetaData, out); + aliasMetaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java index d79c2128611..60a265de785 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java @@ -396,11 +396,11 @@ public class CreateIndexRequest extends AcknowledgedRequest aliases((Map) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); - if (factory != null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); + if (proto != null) { found = true; try { - customs.put(name, factory.fromMap((Map) entry.getValue())); + customs.put(name, proto.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -448,7 +448,7 @@ public class CreateIndexRequest extends AcknowledgedRequest int customSize = in.readVInt(); for (int i = 0; i < customSize; i++) { String type = in.readString(); - IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupFactorySafe(type).readFrom(in); + IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupPrototypeSafe(type).readFrom(in); customs.put(type, customIndexMetaData); } int aliasesSize = in.readVInt(); @@ -472,7 +472,7 @@ public class CreateIndexRequest extends AcknowledgedRequest out.writeVInt(customs.size()); for (Map.Entry entry : customs.entrySet()) { out.writeString(entry.getKey()); - IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), out); + entry.getValue().writeTo(out); } out.writeVInt(aliases.size()); for (Alias alias : aliases) { diff --git a/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java index 35e6cfa4804..7080a694a11 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java @@ -134,7 +134,7 @@ public class GetIndexResponse extends ActionResponse { int valueSize = in.readVInt(); ImmutableOpenMap.Builder mappingEntryBuilder = ImmutableOpenMap.builder(); for (int j = 0; j < valueSize; j++) { - mappingEntryBuilder.put(in.readString(), MappingMetaData.readFrom(in)); + mappingEntryBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in)); } mappingsMapBuilder.put(key, mappingEntryBuilder.build()); } @@ -181,7 +181,7 @@ public class GetIndexResponse extends ActionResponse { out.writeVInt(indexEntry.value.size()); for (ObjectObjectCursor mappingEntry : indexEntry.value) { out.writeString(mappingEntry.key); - MappingMetaData.writeTo(mappingEntry.value, out); + mappingEntry.value.writeTo(out); } } out.writeVInt(aliases.size()); @@ -189,7 +189,7 @@ public class GetIndexResponse extends ActionResponse { out.writeString(indexEntry.key); out.writeVInt(indexEntry.value.size()); for (AliasMetaData aliasEntry : indexEntry.value) { - AliasMetaData.Builder.writeTo(aliasEntry, out); + aliasEntry.writeTo(out); } } out.writeVInt(settings.size()); diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java index b27577f8da3..30e9e24c493 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java @@ -59,7 +59,7 @@ public class GetMappingsResponse extends ActionResponse { int valueSize = in.readVInt(); ImmutableOpenMap.Builder typeMapBuilder = ImmutableOpenMap.builder(); for (int j = 0; j < valueSize; j++) { - typeMapBuilder.put(in.readString(), MappingMetaData.readFrom(in)); + typeMapBuilder.put(in.readString(), MappingMetaData.PROTO.readFrom(in)); } indexMapBuilder.put(key, typeMapBuilder.build()); } @@ -75,7 +75,7 @@ public class GetMappingsResponse extends ActionResponse { out.writeVInt(indexEntry.value.size()); for (ObjectObjectCursor typeEntry : indexEntry.value) { out.writeString(typeEntry.key); - MappingMetaData.writeTo(typeEntry.value, out); + typeEntry.value.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java b/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java index 56de19872f2..2ce6d8d2c1a 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java @@ -60,7 +60,7 @@ public class GetIndexTemplatesResponse extends ActionResponse { super.writeTo(out); out.writeVInt(indexTemplates.size()); for (IndexTemplateMetaData indexTemplate : indexTemplates) { - IndexTemplateMetaData.Builder.writeTo(indexTemplate, out); + indexTemplate.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 41dd9ec2b45..1b752855c20 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -292,10 +292,10 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest) entry.getValue()); } else { // maybe custom? - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(name); - if (factory != null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(name); + if (proto != null) { try { - customs.put(name, factory.fromMap((Map) entry.getValue())); + customs.put(name, proto.fromMap((Map) entry.getValue())); } catch (IOException e) { throw new ElasticsearchParseException("failed to parse custom metadata for [" + name + "]"); } @@ -440,7 +440,7 @@ public class PutIndexTemplateRequest extends MasterNodeOperationRequest entry : customs.entrySet()) { out.writeString(entry.getKey()); - IndexMetaData.lookupFactorySafe(entry.getKey()).writeTo(entry.getValue(), out); + entry.getValue().writeTo(out); } out.writeVInt(aliases.size()); for (Alias alias : aliases) { diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java b/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java index e1309a5c095..732e08ac36b 100644 --- a/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java @@ -28,7 +28,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.FilterBuilder; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; import org.elasticsearch.search.highlight.HighlightBuilder; import org.elasticsearch.search.sort.SortBuilder; @@ -162,9 +164,9 @@ public class PercolateRequestBuilder extends BroadcastOperationRequestBuilder reducers; PercolateShardResponse() { hls = new ArrayList<>(); @@ -69,6 +75,7 @@ public class PercolateShardResponse extends BroadcastShardOperationResponse { if (result.aggregations() != null) { this.aggregations = (InternalAggregations) result.aggregations(); } + this.reducers = result.reducers(); } } @@ -112,6 +119,10 @@ public class PercolateShardResponse extends BroadcastShardOperationResponse { return aggregations; } + public List reducers() { + return reducers; + } + public byte percolatorTypeId() { return percolatorTypeId; } @@ -144,6 +155,16 @@ public class PercolateShardResponse extends BroadcastShardOperationResponse { hls.add(fields); } aggregations = InternalAggregations.readOptionalAggregations(in); + if (in.readBoolean()) { + int reducersSize = in.readVInt(); + List reducers = new ArrayList<>(reducersSize); + for (int i = 0; i < reducersSize; i++) { + BytesReference type = in.readBytesReference(); + Reducer reducer = ReducerStreams.stream(type).readResult(in); + reducers.add((SiblingReducer) reducer); + } + this.reducers = reducers; + } } @Override @@ -169,5 +190,15 @@ public class PercolateShardResponse extends BroadcastShardOperationResponse { } } out.writeOptionalStreamable(aggregations); + if (reducers == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVInt(reducers.size()); + for (Reducer reducer : reducers) { + out.writeBytesReference(reducer.type().stream()); + reducer.writeTo(out); + } + } } } diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateSourceBuilder.java b/src/main/java/org/elasticsearch/action/percolate/PercolateSourceBuilder.java index f09e630f459..68fc57b2a17 100644 --- a/src/main/java/org/elasticsearch/action/percolate/PercolateSourceBuilder.java +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateSourceBuilder.java @@ -29,6 +29,7 @@ import org.elasticsearch.index.query.FilterBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; import org.elasticsearch.search.highlight.HighlightBuilder; import org.elasticsearch.search.sort.ScoreSortBuilder; import org.elasticsearch.search.sort.SortBuilder; @@ -50,7 +51,7 @@ public class PercolateSourceBuilder implements ToXContent { private List sorts; private Boolean trackScores; private HighlightBuilder highlightBuilder; - private List aggregations; + private List aggregations; /** * Sets the document to run the percolate queries against. @@ -130,7 +131,7 @@ public class PercolateSourceBuilder implements ToXContent { /** * Add an aggregation definition. */ - public PercolateSourceBuilder addAggregation(AggregationBuilder aggregationBuilder) { + public PercolateSourceBuilder addAggregation(AbstractAggregationBuilder aggregationBuilder) { if (aggregations == null) { aggregations = Lists.newArrayList(); } diff --git a/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index 0a96133201d..349da4103f0 100644 --- a/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -34,6 +34,7 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.Scroll; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder; import org.elasticsearch.search.highlight.HighlightBuilder; diff --git a/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java b/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java new file mode 100644 index 00000000000..4e6da2bd569 --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/AbstractDiffable.java @@ -0,0 +1,108 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamableReader; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Abstract diffable object with simple diffs implementation that sends the entire object if object has changed or + * nothing is object remained the same. + */ +public abstract class AbstractDiffable> implements Diffable { + + @Override + public Diff diff(T previousState) { + if (this.get().equals(previousState)) { + return new CompleteDiff<>(); + } else { + return new CompleteDiff<>(get()); + } + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new CompleteDiff<>(this, in); + } + + public static > Diff readDiffFrom(StreamableReader reader, StreamInput in) throws IOException { + return new CompleteDiff(reader, in); + } + + private static class CompleteDiff> implements Diff { + + @Nullable + private final T part; + + /** + * Creates simple diff with changes + */ + public CompleteDiff(T part) { + this.part = part; + } + + /** + * Creates simple diff without changes + */ + public CompleteDiff() { + this.part = null; + } + + /** + * Read simple diff from the stream + */ + public CompleteDiff(StreamableReader reader, StreamInput in) throws IOException { + if (in.readBoolean()) { + this.part = reader.readFrom(in); + } else { + this.part = null; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + if (part != null) { + out.writeBoolean(true); + part.writeTo(out); + } else { + out.writeBoolean(false); + } + } + + @Override + public T apply(T part) { + if (this.part != null) { + return this.part; + } else { + return part; + } + } + } + + @SuppressWarnings("unchecked") + public T get() { + return (T) this; + } +} + diff --git a/src/main/java/org/elasticsearch/cluster/ClusterState.java b/src/main/java/org/elasticsearch/cluster/ClusterState.java index b90bc0bb2ac..4f63d9e00e3 100644 --- a/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.ImmutableSet; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -55,7 +56,9 @@ import java.util.Map; /** * */ -public class ClusterState implements ToXContent { +public class ClusterState implements ToXContent, Diffable { + + public static final ClusterState PROTO = builder(ClusterName.DEFAULT).build(); public static enum ClusterStateStatus { UNKNOWN((byte) 0), @@ -74,47 +77,43 @@ public class ClusterState implements ToXContent { } } - public interface Custom { + public interface Custom extends Diffable, ToXContent { - interface Factory { - - String type(); - - T readFrom(StreamInput in) throws IOException; - - void writeTo(T customState, StreamOutput out) throws IOException; - - void toXContent(T customState, XContentBuilder builder, ToXContent.Params params); - } + String type(); } - private final static Map customFactories = new HashMap<>(); + private final static Map customPrototypes = new HashMap<>(); /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom state factory registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + @SuppressWarnings("unchecked") + T proto = (T)customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom state prototype registered for type [" + type + "]"); } - return factory; + return proto; } + public static final String UNKNOWN_UUID = "_na_"; public static final long UNKNOWN_VERSION = -1; private final long version; + private final String uuid; + private final RoutingTable routingTable; private final DiscoveryNodes nodes; @@ -127,17 +126,20 @@ public class ClusterState implements ToXContent { private final ClusterName clusterName; + private final boolean wasReadFromDiff; + // built on demand private volatile RoutingNodes routingNodes; private volatile ClusterStateStatus status; - public ClusterState(long version, ClusterState state) { - this(state.clusterName, version, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), state.customs()); + public ClusterState(long version, String uuid, ClusterState state) { + this(state.clusterName, version, uuid, state.metaData(), state.routingTable(), state.nodes(), state.blocks(), state.customs(), false); } - public ClusterState(ClusterName clusterName, long version, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs) { + public ClusterState(ClusterName clusterName, long version, String uuid, MetaData metaData, RoutingTable routingTable, DiscoveryNodes nodes, ClusterBlocks blocks, ImmutableOpenMap customs, boolean wasReadFromDiff) { this.version = version; + this.uuid = uuid; this.clusterName = clusterName; this.metaData = metaData; this.routingTable = routingTable; @@ -145,6 +147,7 @@ public class ClusterState implements ToXContent { this.blocks = blocks; this.customs = customs; this.status = ClusterStateStatus.UNKNOWN; + this.wasReadFromDiff = wasReadFromDiff; } public ClusterStateStatus status() { @@ -164,6 +167,14 @@ public class ClusterState implements ToXContent { return version(); } + /** + * This uuid is automatically generated for for each version of cluster state. It is used to make sure that + * we are applying diffs to the right previous state. + */ + public String uuid() { + return this.uuid; + } + public DiscoveryNodes nodes() { return this.nodes; } @@ -216,6 +227,11 @@ public class ClusterState implements ToXContent { return this.clusterName; } + // Used for testing and logging to determine how this cluster state was send over the wire + boolean wasReadFromDiff() { + return wasReadFromDiff; + } + /** * Returns a built (on demand) routing nodes view of the routing table. NOTE, the routing nodes * are mutable, use them just for read operations @@ -231,6 +247,8 @@ public class ClusterState implements ToXContent { public String prettyPrint() { StringBuilder sb = new StringBuilder(); sb.append("version: ").append(version).append("\n"); + sb.append("uuid: ").append(uuid).append("\n"); + sb.append("from_diff: ").append(wasReadFromDiff).append("\n"); sb.append("meta data version: ").append(metaData.version()).append("\n"); sb.append(nodes().prettyPrint()); sb.append(routingTable().prettyPrint()); @@ -302,14 +320,13 @@ public class ClusterState implements ToXContent { } } - - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { EnumSet metrics = Metric.parseString(params.param("metric", "_all"), true); if (metrics.contains(Metric.VERSION)) { builder.field("version", version); + builder.field("uuid", uuid); } if (metrics.contains(Metric.MASTER_NODE)) { @@ -434,7 +451,7 @@ public class ClusterState implements ToXContent { for (ObjectObjectCursor cursor : metaData.customs()) { builder.startObject(cursor.key); - MetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -473,7 +490,7 @@ public class ClusterState implements ToXContent { builder.startObject("nodes"); for (RoutingNode routingNode : readOnlyRoutingNodes()) { - builder.startArray(routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); + builder.startArray(routingNode.nodeId() == null ? "null" : routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE); for (ShardRouting shardRouting : routingNode) { shardRouting.toXContent(builder, params); } @@ -486,7 +503,7 @@ public class ClusterState implements ToXContent { if (metrics.contains(Metric.CUSTOMS)) { for (ObjectObjectCursor cursor : customs) { builder.startObject(cursor.key); - lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } } @@ -506,21 +523,25 @@ public class ClusterState implements ToXContent { private final ClusterName clusterName; private long version = 0; + private String uuid = UNKNOWN_UUID; private MetaData metaData = MetaData.EMPTY_META_DATA; private RoutingTable routingTable = RoutingTable.EMPTY_ROUTING_TABLE; private DiscoveryNodes nodes = DiscoveryNodes.EMPTY_NODES; private ClusterBlocks blocks = ClusterBlocks.EMPTY_CLUSTER_BLOCK; private final ImmutableOpenMap.Builder customs; + private boolean fromDiff; public Builder(ClusterState state) { this.clusterName = state.clusterName; this.version = state.version(); + this.uuid = state.uuid(); this.nodes = state.nodes(); this.routingTable = state.routingTable(); this.metaData = state.metaData(); this.blocks = state.blocks(); this.customs = ImmutableOpenMap.builder(state.customs()); + this.fromDiff = false; } public Builder(ClusterName clusterName) { @@ -574,6 +595,17 @@ public class ClusterState implements ToXContent { return this; } + public Builder incrementVersion() { + this.version = version + 1; + this.uuid = UNKNOWN_UUID; + return this; + } + + public Builder uuid(String uuid) { + this.uuid = uuid; + return this; + } + public Custom getCustom(String type) { return customs.get(type); } @@ -588,13 +620,26 @@ public class ClusterState implements ToXContent { return this; } + public Builder customs(ImmutableOpenMap customs) { + this.customs.putAll(customs); + return this; + } + + public Builder fromDiff(boolean fromDiff) { + this.fromDiff = fromDiff; + return this; + } + public ClusterState build() { - return new ClusterState(clusterName, version, metaData, routingTable, nodes, blocks, customs.build()); + if (UNKNOWN_UUID.equals(uuid)) { + uuid = Strings.randomBase64UUID(); + } + return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), fromDiff); } public static byte[] toBytes(ClusterState state) throws IOException { BytesStreamOutput os = new BytesStreamOutput(); - writeTo(state, os); + state.writeTo(os); return os.bytes().toBytes(); } @@ -606,39 +651,152 @@ public class ClusterState implements ToXContent { return readFrom(new BytesStreamInput(data), localNode); } - public static void writeTo(ClusterState state, StreamOutput out) throws IOException { - state.clusterName.writeTo(out); - out.writeLong(state.version()); - MetaData.Builder.writeTo(state.metaData(), out); - RoutingTable.Builder.writeTo(state.routingTable(), out); - DiscoveryNodes.Builder.writeTo(state.nodes(), out); - ClusterBlocks.Builder.writeClusterBlocks(state.blocks(), out); - out.writeVInt(state.customs().size()); - for (ObjectObjectCursor cursor : state.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } - } - /** * @param in input stream * @param localNode used to set the local node in the cluster state. can be null. */ public static ClusterState readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { - ClusterName clusterName = ClusterName.readClusterName(in); + return PROTO.readFrom(in, localNode); + } + + } + + @Override + public Diff diff(ClusterState previousState) { + return new ClusterStateDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new ClusterStateDiff(in, this); + } + + public ClusterState readFrom(StreamInput in, DiscoveryNode localNode) throws IOException { + ClusterName clusterName = ClusterName.readClusterName(in); + Builder builder = new Builder(clusterName); + builder.version = in.readLong(); + builder.uuid = in.readString(); + builder.metaData = MetaData.Builder.readFrom(in); + builder.routingTable = RoutingTable.Builder.readFrom(in); + builder.nodes = DiscoveryNodes.Builder.readFrom(in, localNode); + builder.blocks = ClusterBlocks.Builder.readClusterBlocks(in); + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public ClusterState readFrom(StreamInput in) throws IOException { + return readFrom(in, nodes.localNode()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterName.writeTo(out); + out.writeLong(version); + out.writeString(uuid); + metaData.writeTo(out); + routingTable.writeTo(out); + nodes.writeTo(out); + blocks.writeTo(out); + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + + private static class ClusterStateDiff implements Diff { + + private final long toVersion; + + private final String fromUuid; + + private final String toUuid; + + private final ClusterName clusterName; + + private final Diff routingTable; + + private final Diff nodes; + + private final Diff metaData; + + private final Diff blocks; + + private final Diff> customs; + + public ClusterStateDiff(ClusterState before, ClusterState after) { + fromUuid = before.uuid; + toUuid = after.uuid; + toVersion = after.version; + clusterName = after.clusterName; + routingTable = after.routingTable.diff(before.routingTable); + nodes = after.nodes.diff(before.nodes); + metaData = after.metaData.diff(before.metaData); + blocks = after.blocks.diff(before.blocks); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public ClusterStateDiff(StreamInput in, ClusterState proto) throws IOException { + clusterName = ClusterName.readClusterName(in); + fromUuid = in.readString(); + toUuid = in.readString(); + toVersion = in.readLong(); + routingTable = proto.routingTable.readDiffFrom(in); + nodes = proto.nodes.readDiffFrom(in); + metaData = proto.metaData.readDiffFrom(in); + blocks = proto.blocks.readDiffFrom(in); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + clusterName.writeTo(out); + out.writeString(fromUuid); + out.writeString(toUuid); + out.writeLong(toVersion); + routingTable.writeTo(out); + nodes.writeTo(out); + metaData.writeTo(out); + blocks.writeTo(out); + customs.writeTo(out); + } + + @Override + public ClusterState apply(ClusterState state) { Builder builder = new Builder(clusterName); - builder.version = in.readLong(); - builder.metaData = MetaData.Builder.readFrom(in); - builder.routingTable = RoutingTable.Builder.readFrom(in); - builder.nodes = DiscoveryNodes.Builder.readFrom(in, localNode); - builder.blocks = ClusterBlocks.Builder.readClusterBlocks(in); - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); + if (toUuid.equals(state.uuid)) { + // no need to read the rest - cluster state didn't change + return state; } + if (fromUuid.equals(state.uuid) == false) { + throw new IncompatibleClusterStateVersionException(state.version, state.uuid, toVersion, fromUuid); + } + builder.uuid(toUuid); + builder.version(toVersion); + builder.routingTable(routingTable.apply(state.routingTable)); + builder.nodes(nodes.apply(state.nodes)); + builder.metaData(metaData.apply(state.metaData)); + builder.blocks(blocks.apply(state.blocks)); + builder.customs(customs.apply(state.customs)); + builder.fromDiff(true); return builder.build(); } } + } diff --git a/src/main/java/org/elasticsearch/cluster/Diff.java b/src/main/java/org/elasticsearch/cluster/Diff.java new file mode 100644 index 00000000000..2e571f43bca --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/Diff.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Represents difference between states of cluster state parts + */ +public interface Diff { + + /** + * Applies difference to the specified part and retunrs the resulted part + */ + T apply(T part); + + /** + * Writes the differences into the output stream + * @param out + * @throws IOException + */ + void writeTo(StreamOutput out) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/cluster/Diffable.java b/src/main/java/org/elasticsearch/cluster/Diffable.java new file mode 100644 index 00000000000..7ce60047a2b --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/Diffable.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * Cluster state part, changes in which can be serialized + */ +public interface Diffable extends Writeable { + + /** + * Returns serializable object representing differences between this and previousState + */ + Diff diff(T previousState); + + /** + * Reads the {@link org.elasticsearch.cluster.Diff} from StreamInput + */ + Diff readDiffFrom(StreamInput in) throws IOException; + +} diff --git a/src/main/java/org/elasticsearch/cluster/DiffableUtils.java b/src/main/java/org/elasticsearch/cluster/DiffableUtils.java new file mode 100644 index 00000000000..4e912a34f97 --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/DiffableUtils.java @@ -0,0 +1,283 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Maps.newHashMap; + +public final class DiffableUtils { + private DiffableUtils() { + } + + /** + * Calculates diff between two ImmutableOpenMaps of Diffable objects + */ + public static > Diff> diff(ImmutableOpenMap before, ImmutableOpenMap after) { + assert after != null && before != null; + return new ImmutableOpenMapDiff<>(before, after); + } + + /** + * Calculates diff between two ImmutableMaps of Diffable objects + */ + public static > Diff> diff(ImmutableMap before, ImmutableMap after) { + assert after != null && before != null; + return new ImmutableMapDiff<>(before, after); + } + + /** + * Loads an object that represents difference between two ImmutableOpenMaps + */ + public static > Diff> readImmutableOpenMapDiff(StreamInput in, KeyedReader keyedReader) throws IOException { + return new ImmutableOpenMapDiff<>(in, keyedReader); + } + + /** + * Loads an object that represents difference between two ImmutableMaps + */ + public static > Diff> readImmutableMapDiff(StreamInput in, KeyedReader keyedReader) throws IOException { + return new ImmutableMapDiff<>(in, keyedReader); + } + + /** + * Loads an object that represents difference between two ImmutableOpenMaps + */ + public static > Diff> readImmutableOpenMapDiff(StreamInput in, T proto) throws IOException { + return new ImmutableOpenMapDiff<>(in, new PrototypeReader<>(proto)); + } + + /** + * Loads an object that represents difference between two ImmutableMaps + */ + public static > Diff> readImmutableMapDiff(StreamInput in, T proto) throws IOException { + return new ImmutableMapDiff<>(in, new PrototypeReader<>(proto)); + } + + /** + * A reader that can deserialize an object. The reader can select the deserialization type based on the key. It's + * used in custom metadata deserialization. + */ + public interface KeyedReader { + + /** + * reads an object of the type T from the stream input + */ + T readFrom(StreamInput in, String key) throws IOException; + + /** + * reads an object that respresents differences between two objects with the type T from the stream input + */ + Diff readDiffFrom(StreamInput in, String key) throws IOException; + } + + /** + * Implementation of the KeyedReader that is using a prototype object for reading operations + * + * Note: this implementation is ignoring the key. + */ + public static class PrototypeReader> implements KeyedReader { + private T proto; + + public PrototypeReader(T proto) { + this.proto = proto; + } + + @Override + public T readFrom(StreamInput in, String key) throws IOException { + return proto.readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return proto.readDiffFrom(in); + } + } + + /** + * Represents differences between two ImmutableMaps of diffable objects + * + * @param the diffable object + */ + private static class ImmutableMapDiff> extends MapDiff> { + + protected ImmutableMapDiff(StreamInput in, KeyedReader reader) throws IOException { + super(in, reader); + } + + public ImmutableMapDiff(ImmutableMap before, ImmutableMap after) { + assert after != null && before != null; + for (String key : before.keySet()) { + if (!after.containsKey(key)) { + deletes.add(key); + } + } + for (ImmutableMap.Entry partIter : after.entrySet()) { + T beforePart = before.get(partIter.getKey()); + if (beforePart == null) { + adds.put(partIter.getKey(), partIter.getValue()); + } else if (partIter.getValue().equals(beforePart) == false) { + diffs.put(partIter.getKey(), partIter.getValue().diff(beforePart)); + } + } + } + + @Override + public ImmutableMap apply(ImmutableMap map) { + HashMap builder = newHashMap(); + builder.putAll(map); + + for (String part : deletes) { + builder.remove(part); + } + + for (Map.Entry> diff : diffs.entrySet()) { + builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey()))); + } + + for (Map.Entry additon : adds.entrySet()) { + builder.put(additon.getKey(), additon.getValue()); + } + return ImmutableMap.copyOf(builder); + } + } + + /** + * Represents differences between two ImmutableOpenMap of diffable objects + * + * @param the diffable object + */ + private static class ImmutableOpenMapDiff> extends MapDiff> { + + protected ImmutableOpenMapDiff(StreamInput in, KeyedReader reader) throws IOException { + super(in, reader); + } + + public ImmutableOpenMapDiff(ImmutableOpenMap before, ImmutableOpenMap after) { + assert after != null && before != null; + for (ObjectCursor key : before.keys()) { + if (!after.containsKey(key.value)) { + deletes.add(key.value); + } + } + for (ObjectObjectCursor partIter : after) { + T beforePart = before.get(partIter.key); + if (beforePart == null) { + adds.put(partIter.key, partIter.value); + } else if (partIter.value.equals(beforePart) == false) { + diffs.put(partIter.key, partIter.value.diff(beforePart)); + } + } + } + + @Override + public ImmutableOpenMap apply(ImmutableOpenMap map) { + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + builder.putAll(map); + + for (String part : deletes) { + builder.remove(part); + } + + for (Map.Entry> diff : diffs.entrySet()) { + builder.put(diff.getKey(), diff.getValue().apply(builder.get(diff.getKey()))); + } + + for (Map.Entry additon : adds.entrySet()) { + builder.put(additon.getKey(), additon.getValue()); + } + return builder.build(); + } + } + + /** + * Represents differences between two maps of diffable objects + * + * This class is used as base class for different map implementations + * + * @param the diffable object + */ + private static abstract class MapDiff, M> implements Diff { + + protected final List deletes; + protected final Map> diffs; + protected final Map adds; + + protected MapDiff() { + deletes = newArrayList(); + diffs = newHashMap(); + adds = newHashMap(); + } + + protected MapDiff(StreamInput in, KeyedReader reader) throws IOException { + deletes = newArrayList(); + diffs = newHashMap(); + adds = newHashMap(); + int deletesCount = in.readVInt(); + for (int i = 0; i < deletesCount; i++) { + deletes.add(in.readString()); + } + + int diffsCount = in.readVInt(); + for (int i = 0; i < diffsCount; i++) { + String key = in.readString(); + Diff diff = reader.readDiffFrom(in, key); + diffs.put(key, diff); + } + + int addsCount = in.readVInt(); + for (int i = 0; i < addsCount; i++) { + String key = in.readString(); + T part = reader.readFrom(in, key); + adds.put(key, part); + } + } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(deletes.size()); + for (String delete : deletes) { + out.writeString(delete); + } + + out.writeVInt(diffs.size()); + for (Map.Entry> entry : diffs.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + + out.writeVInt(adds.size()); + for (Map.Entry entry : adds.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + } + } +} diff --git a/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java b/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java new file mode 100644 index 00000000000..92f5897bf2e --- /dev/null +++ b/src/main/java/org/elasticsearch/cluster/IncompatibleClusterStateVersionException.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import org.elasticsearch.ElasticsearchException; + +/** + * Thrown by {@link Diffable#readDiffAndApply(org.elasticsearch.common.io.stream.StreamInput)} method + */ +public class IncompatibleClusterStateVersionException extends ElasticsearchException { + public IncompatibleClusterStateVersionException(String msg) { + super(msg); + } + + public IncompatibleClusterStateVersionException(long expectedVersion, String expectedUuid, long receivedVersion, String receivedUuid) { + super("Expected diff for version " + expectedVersion + " with uuid " + expectedUuid + " got version " + receivedVersion + " and uuid " + receivedUuid); + } +} diff --git a/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index bb7d332de4f..95c0ba7127e 100644 --- a/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.common.io.stream.StreamInput; @@ -36,10 +37,12 @@ import java.util.Set; /** * Represents current cluster level blocks to block dirty operations done against the cluster. */ -public class ClusterBlocks { +public class ClusterBlocks extends AbstractDiffable { public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(ImmutableSet.of(), ImmutableMap.>of()); + public static final ClusterBlocks PROTO = EMPTY_CLUSTER_BLOCK; + private final ImmutableSet global; private final ImmutableMap> indicesBlocks; @@ -203,6 +206,43 @@ public class ClusterBlocks { return new ClusterBlockException(builder.build()); } + @Override + public void writeTo(StreamOutput out) throws IOException { + writeBlockSet(global, out); + out.writeVInt(indicesBlocks.size()); + for (Map.Entry> entry : indicesBlocks.entrySet()) { + out.writeString(entry.getKey()); + writeBlockSet(entry.getValue(), out); + } + } + + private static void writeBlockSet(ImmutableSet blocks, StreamOutput out) throws IOException { + out.writeVInt(blocks.size()); + for (ClusterBlock block : blocks) { + block.writeTo(out); + } + } + + @Override + public ClusterBlocks readFrom(StreamInput in) throws IOException { + ImmutableSet global = readBlockSet(in); + ImmutableMap.Builder> indicesBuilder = ImmutableMap.builder(); + int size = in.readVInt(); + for (int j = 0; j < size; j++) { + indicesBuilder.put(in.readString().intern(), readBlockSet(in)); + } + return new ClusterBlocks(global, indicesBuilder.build()); + } + + private static ImmutableSet readBlockSet(StreamInput in) throws IOException { + ImmutableSet.Builder builder = ImmutableSet.builder(); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.add(ClusterBlock.readClusterBlock(in)); + } + return builder.build(); + } + static class ImmutableLevelHolder { static final ImmutableLevelHolder EMPTY = new ImmutableLevelHolder(ImmutableSet.of(), ImmutableMap.>of()); @@ -313,38 +353,7 @@ public class ClusterBlocks { } public static ClusterBlocks readClusterBlocks(StreamInput in) throws IOException { - ImmutableSet global = readBlockSet(in); - ImmutableMap.Builder> indicesBuilder = ImmutableMap.builder(); - int size = in.readVInt(); - for (int j = 0; j < size; j++) { - indicesBuilder.put(in.readString().intern(), readBlockSet(in)); - } - return new ClusterBlocks(global, indicesBuilder.build()); - } - - public static void writeClusterBlocks(ClusterBlocks blocks, StreamOutput out) throws IOException { - writeBlockSet(blocks.global(), out); - out.writeVInt(blocks.indices().size()); - for (Map.Entry> entry : blocks.indices().entrySet()) { - out.writeString(entry.getKey()); - writeBlockSet(entry.getValue(), out); - } - } - - private static void writeBlockSet(ImmutableSet blocks, StreamOutput out) throws IOException { - out.writeVInt(blocks.size()); - for (ClusterBlock block : blocks) { - block.writeTo(out); - } - } - - private static ImmutableSet readBlockSet(StreamInput in) throws IOException { - ImmutableSet.Builder builder = ImmutableSet.builder(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.add(ClusterBlock.readClusterBlock(in)); - } - return builder.build(); + return PROTO.readFrom(in); } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java index 008935ec026..0f7e55c8087 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/AliasMetaData.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableSet; import org.elasticsearch.ElasticsearchGenerationException; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.io.stream.StreamInput; @@ -38,7 +39,9 @@ import java.util.Set; /** * */ -public class AliasMetaData { +public class AliasMetaData extends AbstractDiffable { + + public static final AliasMetaData PROTO = new AliasMetaData("", null, null, null); private final String alias; @@ -146,6 +149,48 @@ public class AliasMetaData { return result; } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(alias()); + if (filter() != null) { + out.writeBoolean(true); + filter.writeTo(out); + } else { + out.writeBoolean(false); + } + if (indexRouting() != null) { + out.writeBoolean(true); + out.writeString(indexRouting()); + } else { + out.writeBoolean(false); + } + if (searchRouting() != null) { + out.writeBoolean(true); + out.writeString(searchRouting()); + } else { + out.writeBoolean(false); + } + + } + + @Override + public AliasMetaData readFrom(StreamInput in) throws IOException { + String alias = in.readString(); + CompressedString filter = null; + if (in.readBoolean()) { + filter = CompressedString.readCompressedString(in); + } + String indexRouting = null; + if (in.readBoolean()) { + indexRouting = in.readString(); + } + String searchRouting = null; + if (in.readBoolean()) { + searchRouting = in.readString(); + } + return new AliasMetaData(alias, filter, indexRouting, searchRouting); + } + public static class Builder { private final String alias; @@ -294,44 +339,12 @@ public class AliasMetaData { return builder.build(); } - public static void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException { - out.writeString(aliasMetaData.alias()); - if (aliasMetaData.filter() != null) { - out.writeBoolean(true); - aliasMetaData.filter.writeTo(out); - } else { - out.writeBoolean(false); - } - if (aliasMetaData.indexRouting() != null) { - out.writeBoolean(true); - out.writeString(aliasMetaData.indexRouting()); - } else { - out.writeBoolean(false); - } - if (aliasMetaData.searchRouting() != null) { - out.writeBoolean(true); - out.writeString(aliasMetaData.searchRouting()); - } else { - out.writeBoolean(false); - } - + public void writeTo(AliasMetaData aliasMetaData, StreamOutput out) throws IOException { + aliasMetaData.writeTo(out); } public static AliasMetaData readFrom(StreamInput in) throws IOException { - String alias = in.readString(); - CompressedString filter = null; - if (in.readBoolean()) { - filter = CompressedString.readCompressedString(in); - } - String indexRouting = null; - if (in.readBoolean()) { - indexRouting = in.readString(); - } - String searchRouting = null; - if (in.readBoolean()) { - searchRouting = in.readString(); - } - return new AliasMetaData(alias, filter, indexRouting, searchRouting); + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index d6bcacf1615..fe76d0f3f2b 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -24,6 +24,9 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import org.elasticsearch.Version; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.Diffable; +import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.node.DiscoveryNodeFilters; @@ -59,60 +62,54 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class IndexMetaData { +public class IndexMetaData implements Diffable { + public static final IndexMetaData PROTO = IndexMetaData.builder("") + .settings(ImmutableSettings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); - public interface Custom { + public interface Custom extends Diffable, ToXContent { String type(); - interface Factory { + Custom fromMap(Map map) throws IOException; - String type(); + Custom fromXContent(XContentParser parser) throws IOException; - T readFrom(StreamInput in) throws IOException; - - void writeTo(T customIndexMetaData, StreamOutput out) throws IOException; - - T fromMap(Map map) throws IOException; - - T fromXContent(XContentParser parser) throws IOException; - - void toXContent(T customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException; - - /** - * Merges from first to second, with first being more important, i.e., if something exists in first and second, - * first will prevail. - */ - T merge(T first, T second); - } + /** + * Merges from this to another, with this being more important, i.e., if something exists in this and another, + * this will prevail. + */ + Custom mergeWith(Custom another); } - public static Map customFactories = new HashMap<>(); + public static Map customPrototypes = new HashMap<>(); static { // register non plugin custom metadata - registerFactory(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.FACTORY); + registerPrototype(IndexWarmersMetaData.TYPE, IndexWarmersMetaData.PROTO); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom index metadata factoy registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + //noinspection unchecked + T proto = (T) customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); } - return factory; + return proto; } public static final ClusterBlock INDEX_READ_ONLY_BLOCK = new ClusterBlock(5, "index read-only (api)", false, false, RestStatus.FORBIDDEN, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE)); @@ -451,7 +448,9 @@ public class IndexMetaData { if (state != that.state) { return false; } - + if (!customs.equals(that.customs)) { + return false; + } return true; } @@ -465,6 +464,126 @@ public class IndexMetaData { return result; } + @Override + public Diff diff(IndexMetaData previousState) { + return new IndexMetaDataDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new IndexMetaDataDiff(in); + } + + private static class IndexMetaDataDiff implements Diff { + + private final String index; + private final long version; + private final State state; + private final Settings settings; + private final Diff> mappings; + private final Diff> aliases; + private Diff> customs; + + public IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) { + index = after.index; + version = after.version; + state = after.state; + settings = after.settings; + mappings = DiffableUtils.diff(before.mappings, after.mappings); + aliases = DiffableUtils.diff(before.aliases, after.aliases); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public IndexMetaDataDiff(StreamInput in) throws IOException { + index = in.readString(); + version = in.readLong(); + state = State.fromId(in.readByte()); + settings = ImmutableSettings.readSettingsFromStream(in); + mappings = DiffableUtils.readImmutableOpenMapDiff(in, MappingMetaData.PROTO); + aliases = DiffableUtils.readImmutableOpenMapDiff(in, AliasMetaData.PROTO); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new DiffableUtils.KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeLong(version); + out.writeByte(state.id); + ImmutableSettings.writeSettingsToStream(settings, out); + mappings.writeTo(out); + aliases.writeTo(out); + customs.writeTo(out); + } + + @Override + public IndexMetaData apply(IndexMetaData part) { + Builder builder = builder(index); + builder.version(version); + builder.state(state); + builder.settings(settings); + builder.mappings.putAll(mappings.apply(part.mappings)); + builder.aliases.putAll(aliases.apply(part.aliases)); + builder.customs.putAll(customs.apply(part.customs)); + return builder.build(); + } + } + + @Override + public IndexMetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(in.readString()); + builder.version(in.readLong()); + builder.state(State.fromId(in.readByte())); + builder.settings(readSettingsFromStream(in)); + int mappingsSize = in.readVInt(); + for (int i = 0; i < mappingsSize; i++) { + MappingMetaData mappingMd = MappingMetaData.PROTO.readFrom(in); + builder.putMapping(mappingMd); + } + int aliasesSize = in.readVInt(); + for (int i = 0; i < aliasesSize; i++) { + AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); + builder.putAlias(aliasMd); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeLong(version); + out.writeByte(state.id()); + writeSettingsToStream(settings, out); + out.writeVInt(mappings.size()); + for (ObjectCursor cursor : mappings.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(aliases.size()); + for (ObjectCursor cursor : aliases.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static Builder builder(String index) { return new Builder(index); } @@ -660,7 +779,7 @@ public class IndexMetaData { for (ObjectObjectCursor cursor : indexMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -707,12 +826,13 @@ public class IndexMetaData { } } else { // check if its a custom index metadata - Custom.Factory factory = lookupFactory(currentFieldName); - if (factory == null) { + Custom proto = lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -741,47 +861,7 @@ public class IndexMetaData { } public static IndexMetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(in.readString()); - builder.version(in.readLong()); - builder.state(State.fromId(in.readByte())); - builder.settings(readSettingsFromStream(in)); - int mappingsSize = in.readVInt(); - for (int i = 0; i < mappingsSize; i++) { - MappingMetaData mappingMd = MappingMetaData.readFrom(in); - builder.putMapping(mappingMd); - } - int aliasesSize = in.readVInt(); - for (int i = 0; i < aliasesSize; i++) { - AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); - builder.putAlias(aliasMd); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws IOException { - out.writeString(indexMetaData.index()); - out.writeLong(indexMetaData.version()); - out.writeByte(indexMetaData.state().id()); - writeSettingsToStream(indexMetaData.settings(), out); - out.writeVInt(indexMetaData.mappings().size()); - for (ObjectCursor cursor : indexMetaData.mappings().values()) { - MappingMetaData.writeTo(cursor.value, out); - } - out.writeVInt(indexMetaData.aliases().size()); - for (ObjectCursor cursor : indexMetaData.aliases().values()) { - AliasMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(indexMetaData.customs().size()); - for (ObjectObjectCursor cursor : indexMetaData.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index 582e008550d..54150ee6a1e 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -21,7 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.Sets; -import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.compress.CompressedString; @@ -42,7 +42,9 @@ import java.util.Set; /** * */ -public class IndexTemplateMetaData { +public class IndexTemplateMetaData extends AbstractDiffable { + + public static final IndexTemplateMetaData PROTO = IndexTemplateMetaData.builder("").build(); private final String name; @@ -161,11 +163,57 @@ public class IndexTemplateMetaData { return result; } + @Override + public IndexTemplateMetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(in.readString()); + builder.order(in.readInt()); + builder.template(in.readString()); + builder.settings(ImmutableSettings.readSettingsFromStream(in)); + int mappingsSize = in.readVInt(); + for (int i = 0; i < mappingsSize; i++) { + builder.putMapping(in.readString(), CompressedString.readCompressedString(in)); + } + int aliasesSize = in.readVInt(); + for (int i = 0; i < aliasesSize; i++) { + AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); + builder.putAlias(aliasMd); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeInt(order); + out.writeString(template); + ImmutableSettings.writeSettingsToStream(settings, out); + out.writeVInt(mappings.size()); + for (ObjectObjectCursor cursor : mappings) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + out.writeVInt(aliases.size()); + for (ObjectCursor cursor : aliases.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static class Builder { private static final Set VALID_FIELDS = Sets.newHashSet("template", "order", "mappings", "settings"); static { - VALID_FIELDS.addAll(IndexMetaData.customFactories.keySet()); + VALID_FIELDS.addAll(IndexMetaData.customPrototypes.keySet()); } private String name; @@ -305,7 +353,7 @@ public class IndexTemplateMetaData { for (ObjectObjectCursor cursor : indexTemplateMetaData.customs()) { builder.startObject(cursor.key, XContentBuilder.FieldCaseConversion.NONE); - IndexMetaData.lookupFactorySafe(cursor.key).toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } @@ -347,12 +395,13 @@ public class IndexTemplateMetaData { } } else { // check if its a custom index metadata - IndexMetaData.Custom.Factory factory = IndexMetaData.lookupFactory(currentFieldName); - if (factory == null) { + IndexMetaData.Custom proto = IndexMetaData.lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + IndexMetaData.Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token == XContentParser.Token.START_ARRAY) { @@ -401,47 +450,7 @@ public class IndexTemplateMetaData { } public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(in.readString()); - builder.order(in.readInt()); - builder.template(in.readString()); - builder.settings(ImmutableSettings.readSettingsFromStream(in)); - int mappingsSize = in.readVInt(); - for (int i = 0; i < mappingsSize; i++) { - builder.putMapping(in.readString(), CompressedString.readCompressedString(in)); - } - int aliasesSize = in.readVInt(); - for (int i = 0; i < aliasesSize; i++) { - AliasMetaData aliasMd = AliasMetaData.Builder.readFrom(in); - builder.putAlias(aliasMd); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - IndexMetaData.Custom customIndexMetaData = IndexMetaData.lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(IndexTemplateMetaData indexTemplateMetaData, StreamOutput out) throws IOException { - out.writeString(indexTemplateMetaData.name()); - out.writeInt(indexTemplateMetaData.order()); - out.writeString(indexTemplateMetaData.template()); - ImmutableSettings.writeSettingsToStream(indexTemplateMetaData.settings(), out); - out.writeVInt(indexTemplateMetaData.mappings().size()); - for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { - out.writeString(cursor.key); - cursor.value.writeTo(out); - } - out.writeVInt(indexTemplateMetaData.aliases().size()); - for (ObjectCursor cursor : indexTemplateMetaData.aliases().values()) { - AliasMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(indexTemplateMetaData.customs().size()); - for (ObjectObjectCursor cursor : indexTemplateMetaData.customs()) { - out.writeString(cursor.key); - IndexMetaData.lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java index f80c6072bfc..7225a43d5ef 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java @@ -19,8 +19,10 @@ package org.elasticsearch.cluster.metadata; +import com.google.common.collect.Maps; import org.elasticsearch.Version; import org.elasticsearch.action.TimestampParsingException; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedString; @@ -38,14 +40,18 @@ import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; import java.util.Map; +import static com.google.common.collect.Maps.newHashMap; import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeBooleanValue; /** * Mapping configuration for a type. */ -public class MappingMetaData { +public class MappingMetaData extends AbstractDiffable { + + public static final MappingMetaData PROTO = new MappingMetaData(); public static class Id { @@ -317,6 +323,15 @@ public class MappingMetaData { initMappers(withoutType); } + private MappingMetaData() { + this.type = ""; + try { + this.source = new CompressedString(""); + } catch (IOException ex) { + throw new IllegalStateException("Cannot create MappingMetaData prototype", ex); + } + } + private void initMappers(Map withoutType) { if (withoutType.containsKey("_id")) { String path = null; @@ -532,34 +547,35 @@ public class MappingMetaData { } } - public static void writeTo(MappingMetaData mappingMd, StreamOutput out) throws IOException { - out.writeString(mappingMd.type()); - mappingMd.source().writeTo(out); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(type()); + source().writeTo(out); // id - if (mappingMd.id().hasPath()) { + if (id().hasPath()) { out.writeBoolean(true); - out.writeString(mappingMd.id().path()); + out.writeString(id().path()); } else { out.writeBoolean(false); } // routing - out.writeBoolean(mappingMd.routing().required()); - if (mappingMd.routing().hasPath()) { + out.writeBoolean(routing().required()); + if (routing().hasPath()) { out.writeBoolean(true); - out.writeString(mappingMd.routing().path()); + out.writeString(routing().path()); } else { out.writeBoolean(false); } // timestamp - out.writeBoolean(mappingMd.timestamp().enabled()); - out.writeOptionalString(mappingMd.timestamp().path()); - out.writeString(mappingMd.timestamp().format()); - out.writeOptionalString(mappingMd.timestamp().defaultTimestamp()); + out.writeBoolean(timestamp().enabled()); + out.writeOptionalString(timestamp().path()); + out.writeString(timestamp().format()); + out.writeOptionalString(timestamp().defaultTimestamp()); // TODO Remove the test in elasticsearch 2.0.0 if (out.getVersion().onOrAfter(Version.V_1_5_0)) { - out.writeOptionalBoolean(mappingMd.timestamp().ignoreMissing()); + out.writeOptionalBoolean(timestamp().ignoreMissing()); } - out.writeBoolean(mappingMd.hasParentField()); + out.writeBoolean(hasParentField()); } @Override @@ -588,7 +604,7 @@ public class MappingMetaData { return result; } - public static MappingMetaData readFrom(StreamInput in) throws IOException { + public MappingMetaData readFrom(StreamInput in) throws IOException { String type = in.readString(); CompressedString source = CompressedString.readCompressedString(in); // id diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index ea25a6d5256..97a1367d8e8 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -25,7 +25,9 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.base.Predicate; import com.google.common.collect.*; +import org.elasticsearch.cluster.*; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.common.Nullable; @@ -55,7 +57,9 @@ import static org.elasticsearch.common.settings.ImmutableSettings.*; /** * */ -public class MetaData implements Iterable { +public class MetaData implements Iterable, Diffable { + + public static final MetaData PROTO = builder().build(); public static final String ALL = "_all"; @@ -67,60 +71,51 @@ public class MetaData implements Iterable { GATEWAY, /* Custom metadata should be stored as part of a snapshot */ - SNAPSHOT; + SNAPSHOT } public static EnumSet API_ONLY = EnumSet.of(XContentContext.API); public static EnumSet API_AND_GATEWAY = EnumSet.of(XContentContext.API, XContentContext.GATEWAY); public static EnumSet API_AND_SNAPSHOT = EnumSet.of(XContentContext.API, XContentContext.SNAPSHOT); - public interface Custom { + public interface Custom extends Diffable, ToXContent { - abstract class Factory { + String type(); - public abstract String type(); + Custom fromXContent(XContentParser parser) throws IOException; - public abstract T readFrom(StreamInput in) throws IOException; - - public abstract void writeTo(T customIndexMetaData, StreamOutput out) throws IOException; - - public abstract T fromXContent(XContentParser parser) throws IOException; - - public abstract void toXContent(T customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException; - - public EnumSet context() { - return API_ONLY; - } - } + EnumSet context(); } - public static Map customFactories = new HashMap<>(); + public static Map customPrototypes = new HashMap<>(); static { // register non plugin custom metadata - registerFactory(RepositoriesMetaData.TYPE, RepositoriesMetaData.FACTORY); - registerFactory(SnapshotMetaData.TYPE, SnapshotMetaData.FACTORY); - registerFactory(RestoreMetaData.TYPE, RestoreMetaData.FACTORY); + registerPrototype(RepositoriesMetaData.TYPE, RepositoriesMetaData.PROTO); + registerPrototype(SnapshotMetaData.TYPE, SnapshotMetaData.PROTO); + registerPrototype(RestoreMetaData.TYPE, RestoreMetaData.PROTO); } /** * Register a custom index meta data factory. Make sure to call it from a static block. */ - public static void registerFactory(String type, Custom.Factory factory) { - customFactories.put(type, factory); + public static void registerPrototype(String type, Custom proto) { + customPrototypes.put(type, proto); } @Nullable - public static Custom.Factory lookupFactory(String type) { - return customFactories.get(type); + public static T lookupPrototype(String type) { + //noinspection unchecked + return (T) customPrototypes.get(type); } - public static Custom.Factory lookupFactorySafe(String type) { - Custom.Factory factory = customFactories.get(type); - if (factory == null) { - throw new IllegalArgumentException("No custom index metadata factory registered for type [" + type + "]"); + public static T lookupPrototypeSafe(String type) { + //noinspection unchecked + T proto = (T) customPrototypes.get(type); + if (proto == null) { + throw new IllegalArgumentException("No custom metadata prototype registered for type [" + type + "]"); } - return factory; + return proto; } @@ -644,14 +639,22 @@ public class MetaData implements Iterable { /** * Translates the provided indices or aliases, eventually containing wildcard expressions, into actual indices. * - * @param indicesOptions how the aliases or indices need to be resolved to concrete indices + * @param indicesOptions how the aliases or indices need to be resolved to concrete indices * @param aliasesOrIndices the aliases or indices to be resolved to concrete indices * @return the obtained concrete indices +<<<<<<< HEAD * @throws IndexMissingException if one of the aliases or indices is missing and the provided indices options * don't allow such a case, or if the final result of the indices resolution is no indices and the indices options * don't allow such a case. * @throws IllegalArgumentException if one of the aliases resolve to multiple indices and the provided * indices options don't allow such a case. +======= + * @throws IndexMissingException if one of the aliases or indices is missing and the provided indices options + * don't allow such a case, or if the final result of the indices resolution is no indices and the indices options + * don't allow such a case. + * @throws ElasticsearchIllegalArgumentException if one of the aliases resolve to multiple indices and the provided + * indices options don't allow such a case. +>>>>>>> Add support for cluster state diffs */ public String[] concreteIndices(IndicesOptions indicesOptions, String... aliasesOrIndices) throws IndexMissingException, IllegalArgumentException { if (indicesOptions.expandWildcardsOpen() || indicesOptions.expandWildcardsClosed()) { @@ -1139,14 +1142,14 @@ public class MetaData implements Iterable { // Check if any persistent metadata needs to be saved int customCount1 = 0; for (ObjectObjectCursor cursor : metaData1.customs) { - if (customFactories.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { if (!cursor.value.equals(metaData2.custom(cursor.key))) return false; customCount1++; } } int customCount2 = 0; for (ObjectObjectCursor cursor : metaData2.customs) { - if (customFactories.get(cursor.key).context().contains(XContentContext.GATEWAY)) { + if (customPrototypes.get(cursor.key).context().contains(XContentContext.GATEWAY)) { customCount2++; } } @@ -1154,6 +1157,129 @@ public class MetaData implements Iterable { return true; } + @Override + public Diff diff(MetaData previousState) { + return new MetaDataDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new MetaDataDiff(in); + } + + private static class MetaDataDiff implements Diff { + + private long version; + + private String uuid; + + private Settings transientSettings; + private Settings persistentSettings; + private Diff> indices; + private Diff> templates; + private Diff> customs; + + + public MetaDataDiff(MetaData before, MetaData after) { + uuid = after.uuid; + version = after.version; + transientSettings = after.transientSettings; + persistentSettings = after.persistentSettings; + indices = DiffableUtils.diff(before.indices, after.indices); + templates = DiffableUtils.diff(before.templates, after.templates); + customs = DiffableUtils.diff(before.customs, after.customs); + } + + public MetaDataDiff(StreamInput in) throws IOException { + uuid = in.readString(); + version = in.readLong(); + transientSettings = ImmutableSettings.readSettingsFromStream(in); + persistentSettings = ImmutableSettings.readSettingsFromStream(in); + indices = DiffableUtils.readImmutableOpenMapDiff(in, IndexMetaData.PROTO); + templates = DiffableUtils.readImmutableOpenMapDiff(in, IndexTemplateMetaData.PROTO); + customs = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public Custom readFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(uuid); + out.writeLong(version); + ImmutableSettings.writeSettingsToStream(transientSettings, out); + ImmutableSettings.writeSettingsToStream(persistentSettings, out); + indices.writeTo(out); + templates.writeTo(out); + customs.writeTo(out); + } + + @Override + public MetaData apply(MetaData part) { + Builder builder = builder(); + builder.uuid(uuid); + builder.version(version); + builder.transientSettings(transientSettings); + builder.persistentSettings(persistentSettings); + builder.indices(indices.apply(part.indices)); + builder.templates(templates.apply(part.templates)); + builder.customs(customs.apply(part.customs)); + return builder.build(); + } + } + + @Override + public MetaData readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(); + builder.version = in.readLong(); + builder.uuid = in.readString(); + builder.transientSettings(readSettingsFromStream(in)); + builder.persistentSettings(readSettingsFromStream(in)); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.put(IndexMetaData.Builder.readFrom(in), false); + } + size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.put(IndexTemplateMetaData.Builder.readFrom(in)); + } + int customSize = in.readVInt(); + for (int i = 0; i < customSize; i++) { + String type = in.readString(); + Custom customIndexMetaData = lookupPrototypeSafe(type).readFrom(in); + builder.putCustom(type, customIndexMetaData); + } + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + out.writeString(uuid); + writeSettingsToStream(transientSettings, out); + writeSettingsToStream(persistentSettings, out); + out.writeVInt(indices.size()); + for (IndexMetaData indexMetaData : this) { + indexMetaData.writeTo(out); + } + out.writeVInt(templates.size()); + for (ObjectCursor cursor : templates.values()) { + cursor.value.writeTo(out); + } + out.writeVInt(customs.size()); + for (ObjectObjectCursor cursor : customs) { + out.writeString(cursor.key); + cursor.value.writeTo(out); + } + } + public static Builder builder() { return new Builder(); } @@ -1225,6 +1351,11 @@ public class MetaData implements Iterable { return this; } + public Builder indices(ImmutableOpenMap indices) { + this.indices.putAll(indices); + return this; + } + public Builder put(IndexTemplateMetaData.Builder template) { return put(template.build()); } @@ -1239,6 +1370,11 @@ public class MetaData implements Iterable { return this; } + public Builder templates(ImmutableOpenMap templates) { + this.templates.putAll(templates); + return this; + } + public Custom getCustom(String type) { return customs.get(type); } @@ -1253,6 +1389,11 @@ public class MetaData implements Iterable { return this; } + public Builder customs(ImmutableOpenMap customs) { + this.customs.putAll(customs); + return this; + } + public Builder updateSettings(Settings settings, String... indices) { if (indices == null || indices.length == 0) { indices = this.indices.keys().toArray(String.class); @@ -1305,6 +1446,11 @@ public class MetaData implements Iterable { return this; } + public Builder uuid(String uuid) { + this.uuid = uuid; + return this; + } + public Builder generateUuidIfNeeded() { if (uuid.equals("_na_")) { uuid = Strings.randomBase64UUID(); @@ -1363,10 +1509,10 @@ public class MetaData implements Iterable { } for (ObjectObjectCursor cursor : metaData.customs()) { - Custom.Factory factory = lookupFactorySafe(cursor.key); - if (factory.context().contains(context)) { + Custom proto = lookupPrototypeSafe(cursor.key); + if (proto.context().contains(context)) { builder.startObject(cursor.key); - factory.toXContent(cursor.value, builder, params); + cursor.value.toXContent(builder, params); builder.endObject(); } } @@ -1410,12 +1556,13 @@ public class MetaData implements Iterable { } } else { // check if its a custom index metadata - Custom.Factory factory = lookupFactory(currentFieldName); - if (factory == null) { + Custom proto = lookupPrototype(currentFieldName); + if (proto == null) { //TODO warn parser.skipChildren(); } else { - builder.putCustom(factory.type(), factory.fromXContent(parser)); + Custom custom = proto.fromXContent(parser); + builder.putCustom(custom.type(), custom); } } } else if (token.isValue()) { @@ -1430,46 +1577,7 @@ public class MetaData implements Iterable { } public static MetaData readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(); - builder.version = in.readLong(); - builder.uuid = in.readString(); - builder.transientSettings(readSettingsFromStream(in)); - builder.persistentSettings(readSettingsFromStream(in)); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.put(IndexMetaData.Builder.readFrom(in), false); - } - size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.put(IndexTemplateMetaData.Builder.readFrom(in)); - } - int customSize = in.readVInt(); - for (int i = 0; i < customSize; i++) { - String type = in.readString(); - Custom customIndexMetaData = lookupFactorySafe(type).readFrom(in); - builder.putCustom(type, customIndexMetaData); - } - return builder.build(); - } - - public static void writeTo(MetaData metaData, StreamOutput out) throws IOException { - out.writeLong(metaData.version); - out.writeString(metaData.uuid); - writeSettingsToStream(metaData.transientSettings(), out); - writeSettingsToStream(metaData.persistentSettings(), out); - out.writeVInt(metaData.indices.size()); - for (IndexMetaData indexMetaData : metaData) { - IndexMetaData.Builder.writeTo(indexMetaData, out); - } - out.writeVInt(metaData.templates.size()); - for (ObjectCursor cursor : metaData.templates.values()) { - IndexTemplateMetaData.Builder.writeTo(cursor.value, out); - } - out.writeVInt(metaData.customs().size()); - for (ObjectObjectCursor cursor : metaData.customs()) { - out.writeString(cursor.key); - lookupFactorySafe(cursor.key).writeTo(cursor.value, out); - } + return PROTO.readFrom(in); } } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 9fcb5182180..732561f66f1 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -272,7 +272,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { if (existing == null) { customs.put(type, custom); } else { - IndexMetaData.Custom merged = IndexMetaData.lookupFactorySafe(type).merge(existing, custom); + IndexMetaData.Custom merged = existing.mergeWith(custom); customs.put(type, merged); } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java index 81b11fc14b1..51cd5db086b 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoriesMetaData.java @@ -21,6 +21,8 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; @@ -39,11 +41,11 @@ import java.util.Map; /** * Contains metadata about registered snapshot repositories */ -public class RepositoriesMetaData implements MetaData.Custom { +public class RepositoriesMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "repositories"; - public static final Factory FACTORY = new Factory(); + public static final RepositoriesMetaData PROTO = new RepositoriesMetaData(); private final ImmutableList repositories; @@ -80,122 +82,132 @@ public class RepositoriesMetaData implements MetaData.Custom { return null; } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RepositoriesMetaData that = (RepositoriesMetaData) o; + + return repositories.equals(that.repositories); + + } + + @Override + public int hashCode() { + return repositories.hashCode(); + } + /** - * Repository metadata factory + * {@inheritDoc} */ - public static class Factory extends MetaData.Custom.Factory { + @Override + public String type() { + return TYPE; + } - /** - * {@inheritDoc} - */ - @Override - public String type() { - return TYPE; + /** + * {@inheritDoc} + */ + @Override + public Custom readFrom(StreamInput in) throws IOException { + RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()]; + for (int i = 0; i < repository.length; i++) { + repository[i] = RepositoryMetaData.readFrom(in); } + return new RepositoriesMetaData(repository); + } - /** - * {@inheritDoc} - */ - @Override - public RepositoriesMetaData readFrom(StreamInput in) throws IOException { - RepositoryMetaData[] repository = new RepositoryMetaData[in.readVInt()]; - for (int i = 0; i < repository.length; i++) { - repository[i] = RepositoryMetaData.readFrom(in); - } - return new RepositoriesMetaData(repository); - } - - /** - * {@inheritDoc} - */ - @Override - public void writeTo(RepositoriesMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.repositories().size()); - for (RepositoryMetaData repository : repositories.repositories()) { - repository.writeTo(out); - } - } - - /** - * {@inheritDoc} - */ - @Override - public RepositoriesMetaData fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token; - List repository = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String name = parser.currentName(); - if (parser.nextToken() != XContentParser.Token.START_OBJECT) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], expected object"); - } - String type = null; - Settings settings = ImmutableSettings.EMPTY; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String currentFieldName = parser.currentName(); - if ("type".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown type"); - } - type = parser.text(); - } else if ("settings".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.START_OBJECT) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], incompatible params"); - } - settings = ImmutableSettings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered())).build(); - } else { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown field [" + currentFieldName + "]"); - } - } else { - throw new ElasticsearchParseException("failed to parse repository [" + name + "]"); - } - } - if (type == null) { - throw new ElasticsearchParseException("failed to parse repository [" + name + "], missing repository type"); - } - repository.add(new RepositoryMetaData(name, type, settings)); - } else { - throw new ElasticsearchParseException("failed to parse repositories"); - } - } - return new RepositoriesMetaData(repository.toArray(new RepositoryMetaData[repository.size()])); - } - - /** - * {@inheritDoc} - */ - @Override - public void toXContent(RepositoriesMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - for (RepositoryMetaData repository : customIndexMetaData.repositories()) { - toXContent(repository, builder, params); - } - } - - @Override - public EnumSet context() { - return MetaData.API_AND_GATEWAY; - } - - /** - * Serializes information about a single repository - * - * @param repository repository metadata - * @param builder XContent builder - * @param params serialization parameters - * @throws IOException - */ - public void toXContent(RepositoryMetaData repository, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(repository.name(), XContentBuilder.FieldCaseConversion.NONE); - builder.field("type", repository.type()); - builder.startObject("settings"); - for (Map.Entry settingEntry : repository.settings().getAsMap().entrySet()) { - builder.field(settingEntry.getKey(), settingEntry.getValue()); - } - builder.endObject(); - - builder.endObject(); + /** + * {@inheritDoc} + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(repositories.size()); + for (RepositoryMetaData repository : repositories) { + repository.writeTo(out); } } + /** + * {@inheritDoc} + */ + @Override + public RepositoriesMetaData fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token; + List repository = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String name = parser.currentName(); + if (parser.nextToken() != XContentParser.Token.START_OBJECT) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], expected object"); + } + String type = null; + Settings settings = ImmutableSettings.EMPTY; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String currentFieldName = parser.currentName(); + if ("type".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown type"); + } + type = parser.text(); + } else if ("settings".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.START_OBJECT) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], incompatible params"); + } + settings = ImmutableSettings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered())).build(); + } else { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], unknown field [" + currentFieldName + "]"); + } + } else { + throw new ElasticsearchParseException("failed to parse repository [" + name + "]"); + } + } + if (type == null) { + throw new ElasticsearchParseException("failed to parse repository [" + name + "], missing repository type"); + } + repository.add(new RepositoryMetaData(name, type, settings)); + } else { + throw new ElasticsearchParseException("failed to parse repositories"); + } + } + return new RepositoriesMetaData(repository.toArray(new RepositoryMetaData[repository.size()])); + } + + /** + * {@inheritDoc} + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + for (RepositoryMetaData repository : repositories) { + toXContent(repository, builder, params); + } + return builder; + } + + @Override + public EnumSet context() { + return MetaData.API_AND_GATEWAY; + } + + /** + * Serializes information about a single repository + * + * @param repository repository metadata + * @param builder XContent builder + * @param params serialization parameters + * @throws IOException + */ + public static void toXContent(RepositoryMetaData repository, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(repository.name(), XContentBuilder.FieldCaseConversion.NONE); + builder.field("type", repository.type()); + builder.startObject("settings"); + for (Map.Entry settingEntry : repository.settings().getAsMap().entrySet()) { + builder.field(settingEntry.getKey(), settingEntry.getValue()); + } + builder.endObject(); + + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java index ea50b30ba88..a283f1f43c1 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RepositoryMetaData.java @@ -99,4 +99,25 @@ public class RepositoryMetaData { out.writeString(type); ImmutableSettings.writeSettingsToStream(settings, out); } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + RepositoryMetaData that = (RepositoryMetaData) o; + + if (!name.equals(that.name)) return false; + if (!type.equals(that.type)) return false; + return settings.equals(that.settings); + + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + type.hashCode(); + result = 31 * result + settings.hashCode(); + return result; + } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java index 642136d7b7e..51fd5e0514a 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/RestoreMetaData.java @@ -21,6 +21,7 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; @@ -29,16 +30,17 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; +import java.util.EnumSet; import java.util.Map; /** * Meta data about restore processes that are currently executing */ -public class RestoreMetaData implements MetaData.Custom { +public class RestoreMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "restore"; - public static final Factory FACTORY = new Factory(); + public static final RestoreMetaData PROTO = new RestoreMetaData(); private final ImmutableList entries; @@ -394,124 +396,122 @@ public class RestoreMetaData implements MetaData.Custom { } /** - * Restore metadata factory + * {@inheritDoc} */ - public static class Factory extends MetaData.Custom.Factory { + @Override + public String type() { + return TYPE; + } - /** - * {@inheritDoc} - */ - @Override - public String type() { - return TYPE; - } - - /** - * {@inheritDoc} - */ - @Override - public RestoreMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - SnapshotId snapshotId = SnapshotId.readSnapshotId(in); - State state = State.fromValue(in.readByte()); - int indices = in.readVInt(); - ImmutableList.Builder indexBuilder = ImmutableList.builder(); - for (int j = 0; j < indices; j++) { - indexBuilder.add(in.readString()); - } - ImmutableMap.Builder builder = ImmutableMap.builder(); - int shards = in.readVInt(); - for (int j = 0; j < shards; j++) { - ShardId shardId = ShardId.readShardId(in); - ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in); - builder.put(shardId, shardState); - } - entries[i] = new Entry(snapshotId, state, indexBuilder.build(), builder.build()); + /** + * {@inheritDoc} + */ + @Override + public RestoreMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + SnapshotId snapshotId = SnapshotId.readSnapshotId(in); + State state = State.fromValue(in.readByte()); + int indices = in.readVInt(); + ImmutableList.Builder indexBuilder = ImmutableList.builder(); + for (int j = 0; j < indices; j++) { + indexBuilder.add(in.readString()); } - return new RestoreMetaData(entries); - } - - /** - * {@inheritDoc} - */ - @Override - public void writeTo(RestoreMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.entries().size()); - for (Entry entry : repositories.entries()) { - entry.snapshotId().writeTo(out); - out.writeByte(entry.state().value()); - out.writeVInt(entry.indices().size()); - for (String index : entry.indices()) { - out.writeString(index); - } - out.writeVInt(entry.shards().size()); - for (Map.Entry shardEntry : entry.shards().entrySet()) { - shardEntry.getKey().writeTo(out); - shardEntry.getValue().writeTo(out); - } + ImmutableMap.Builder builder = ImmutableMap.builder(); + int shards = in.readVInt(); + for (int j = 0; j < shards; j++) { + ShardId shardId = ShardId.readShardId(in); + ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in); + builder.put(shardId, shardState); } + entries[i] = new Entry(snapshotId, state, indexBuilder.build(), builder.build()); } + return new RestoreMetaData(entries); + } - /** - * {@inheritDoc} - */ - @Override - public RestoreMetaData fromXContent(XContentParser parser) throws IOException { - throw new UnsupportedOperationException(); - } - - /** - * {@inheritDoc} - */ - @Override - public void toXContent(RestoreMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray("snapshots"); - for (Entry entry : customIndexMetaData.entries()) { - toXContent(entry, builder, params); + /** + * {@inheritDoc} + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries.size()); + for (Entry entry : entries) { + entry.snapshotId().writeTo(out); + out.writeByte(entry.state().value()); + out.writeVInt(entry.indices().size()); + for (String index : entry.indices()) { + out.writeString(index); } - builder.endArray(); - } - - /** - * Serializes single restore operation - * - * @param entry restore operation metadata - * @param builder XContent builder - * @param params serialization parameters - * @throws IOException - */ - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(); - builder.field("snapshot", entry.snapshotId().getSnapshot()); - builder.field("repository", entry.snapshotId().getRepository()); - builder.field("state", entry.state()); - builder.startArray("indices"); - { - for (String index : entry.indices()) { - builder.value(index); - } + out.writeVInt(entry.shards().size()); + for (Map.Entry shardEntry : entry.shards().entrySet()) { + shardEntry.getKey().writeTo(out); + shardEntry.getValue().writeTo(out); } - builder.endArray(); - builder.startArray("shards"); - { - for (Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardRestoreStatus status = shardEntry.getValue(); - builder.startObject(); - { - builder.field("index", shardId.getIndex()); - builder.field("shard", shardId.getId()); - builder.field("state", status.state()); - } - builder.endObject(); - } - } - - builder.endArray(); - builder.endObject(); } } + /** + * {@inheritDoc} + */ + @Override + public RestoreMetaData fromXContent(XContentParser parser) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public EnumSet context() { + return MetaData.API_ONLY; + } + + /** + * {@inheritDoc} + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startArray("snapshots"); + for (Entry entry : entries) { + toXContent(entry, builder, params); + } + builder.endArray(); + return builder; + } + + /** + * Serializes single restore operation + * + * @param entry restore operation metadata + * @param builder XContent builder + * @param params serialization parameters + * @throws IOException + */ + public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field("snapshot", entry.snapshotId().getSnapshot()); + builder.field("repository", entry.snapshotId().getRepository()); + builder.field("state", entry.state()); + builder.startArray("indices"); + { + for (String index : entry.indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.startArray("shards"); + { + for (Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardRestoreStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field("index", shardId.getIndex()); + builder.field("shard", shardId.getId()); + builder.field("state", status.state()); + } + builder.endObject(); + } + } + + builder.endArray(); + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java index b1bcc92b8bd..b23c58710a0 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/SnapshotMetaData.java @@ -21,6 +21,8 @@ package org.elasticsearch.cluster.metadata; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; @@ -30,6 +32,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; +import java.util.EnumSet; import java.util.Map; import static com.google.common.collect.Maps.newHashMap; @@ -37,10 +40,10 @@ import static com.google.common.collect.Maps.newHashMap; /** * Meta data about snapshots that are currently executing */ -public class SnapshotMetaData implements MetaData.Custom { +public class SnapshotMetaData extends AbstractDiffable implements MetaData.Custom { public static final String TYPE = "snapshots"; - public static final Factory FACTORY = new Factory(); + public static final SnapshotMetaData PROTO = new SnapshotMetaData(); @Override public boolean equals(Object o) { @@ -329,123 +332,123 @@ public class SnapshotMetaData implements MetaData.Custom { return null; } + @Override + public String type() { + return TYPE; + } - public static class Factory extends MetaData.Custom.Factory { - - @Override - public String type() { - return TYPE; //To change body of implemented methods use File | Settings | File Templates. - } - - @Override - public SnapshotMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - SnapshotId snapshotId = SnapshotId.readSnapshotId(in); - boolean includeGlobalState = in.readBoolean(); - State state = State.fromValue(in.readByte()); - int indices = in.readVInt(); - ImmutableList.Builder indexBuilder = ImmutableList.builder(); - for (int j = 0; j < indices; j++) { - indexBuilder.add(in.readString()); - } - long startTime = in.readLong(); - ImmutableMap.Builder builder = ImmutableMap.builder(); - int shards = in.readVInt(); - for (int j = 0; j < shards; j++) { - ShardId shardId = ShardId.readShardId(in); - String nodeId = in.readOptionalString(); - State shardState = State.fromValue(in.readByte()); - builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState)); - } - entries[i] = new Entry(snapshotId, includeGlobalState, state, indexBuilder.build(), startTime, builder.build()); + @Override + public SnapshotMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + SnapshotId snapshotId = SnapshotId.readSnapshotId(in); + boolean includeGlobalState = in.readBoolean(); + State state = State.fromValue(in.readByte()); + int indices = in.readVInt(); + ImmutableList.Builder indexBuilder = ImmutableList.builder(); + for (int j = 0; j < indices; j++) { + indexBuilder.add(in.readString()); } - return new SnapshotMetaData(entries); - } - - @Override - public void writeTo(SnapshotMetaData repositories, StreamOutput out) throws IOException { - out.writeVInt(repositories.entries().size()); - for (Entry entry : repositories.entries()) { - entry.snapshotId().writeTo(out); - out.writeBoolean(entry.includeGlobalState()); - out.writeByte(entry.state().value()); - out.writeVInt(entry.indices().size()); - for (String index : entry.indices()) { - out.writeString(index); - } - out.writeLong(entry.startTime()); - out.writeVInt(entry.shards().size()); - for (Map.Entry shardEntry : entry.shards().entrySet()) { - shardEntry.getKey().writeTo(out); - out.writeOptionalString(shardEntry.getValue().nodeId()); - out.writeByte(shardEntry.getValue().state().value()); - } + long startTime = in.readLong(); + ImmutableMap.Builder builder = ImmutableMap.builder(); + int shards = in.readVInt(); + for (int j = 0; j < shards; j++) { + ShardId shardId = ShardId.readShardId(in); + String nodeId = in.readOptionalString(); + State shardState = State.fromValue(in.readByte()); + builder.put(shardId, new ShardSnapshotStatus(nodeId, shardState)); } + entries[i] = new Entry(snapshotId, includeGlobalState, state, indexBuilder.build(), startTime, builder.build()); } + return new SnapshotMetaData(entries); + } - @Override - public SnapshotMetaData fromXContent(XContentParser parser) throws IOException { - throw new UnsupportedOperationException(); - } - - static final class Fields { - static final XContentBuilderString REPOSITORY = new XContentBuilderString("repository"); - static final XContentBuilderString SNAPSHOTS = new XContentBuilderString("snapshots"); - static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot"); - static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state"); - static final XContentBuilderString STATE = new XContentBuilderString("state"); - static final XContentBuilderString INDICES = new XContentBuilderString("indices"); - static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis"); - static final XContentBuilderString START_TIME = new XContentBuilderString("start_time"); - static final XContentBuilderString SHARDS = new XContentBuilderString("shards"); - static final XContentBuilderString INDEX = new XContentBuilderString("index"); - static final XContentBuilderString SHARD = new XContentBuilderString("shard"); - static final XContentBuilderString NODE = new XContentBuilderString("node"); - } - - @Override - public void toXContent(SnapshotMetaData customIndexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startArray(Fields.SNAPSHOTS); - for (Entry entry : customIndexMetaData.entries()) { - toXContent(entry, builder, params); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries.size()); + for (Entry entry : entries) { + entry.snapshotId().writeTo(out); + out.writeBoolean(entry.includeGlobalState()); + out.writeByte(entry.state().value()); + out.writeVInt(entry.indices().size()); + for (String index : entry.indices()) { + out.writeString(index); } - builder.endArray(); - } - - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.startObject(); - builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository()); - builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot()); - builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState()); - builder.field(Fields.STATE, entry.state()); - builder.startArray(Fields.INDICES); - { - for (String index : entry.indices()) { - builder.value(index); - } + out.writeLong(entry.startTime()); + out.writeVInt(entry.shards().size()); + for (Map.Entry shardEntry : entry.shards().entrySet()) { + shardEntry.getKey().writeTo(out); + out.writeOptionalString(shardEntry.getValue().nodeId()); + out.writeByte(shardEntry.getValue().state().value()); } - builder.endArray(); - builder.timeValueField(Fields.START_TIME_MILLIS, Fields.START_TIME, entry.startTime()); - builder.startArray(Fields.SHARDS); - { - for (Map.Entry shardEntry : entry.shards.entrySet()) { - ShardId shardId = shardEntry.getKey(); - ShardSnapshotStatus status = shardEntry.getValue(); - builder.startObject(); - { - builder.field(Fields.INDEX, shardId.getIndex()); - builder.field(Fields.SHARD, shardId.getId()); - builder.field(Fields.STATE, status.state()); - builder.field(Fields.NODE, status.nodeId()); - } - builder.endObject(); - } - } - builder.endArray(); - builder.endObject(); } } + @Override + public SnapshotMetaData fromXContent(XContentParser parser) throws IOException { + throw new UnsupportedOperationException(); + } + @Override + public EnumSet context() { + return MetaData.API_ONLY; + } + + static final class Fields { + static final XContentBuilderString REPOSITORY = new XContentBuilderString("repository"); + static final XContentBuilderString SNAPSHOTS = new XContentBuilderString("snapshots"); + static final XContentBuilderString SNAPSHOT = new XContentBuilderString("snapshot"); + static final XContentBuilderString INCLUDE_GLOBAL_STATE = new XContentBuilderString("include_global_state"); + static final XContentBuilderString STATE = new XContentBuilderString("state"); + static final XContentBuilderString INDICES = new XContentBuilderString("indices"); + static final XContentBuilderString START_TIME_MILLIS = new XContentBuilderString("start_time_millis"); + static final XContentBuilderString START_TIME = new XContentBuilderString("start_time"); + static final XContentBuilderString SHARDS = new XContentBuilderString("shards"); + static final XContentBuilderString INDEX = new XContentBuilderString("index"); + static final XContentBuilderString SHARD = new XContentBuilderString("shard"); + static final XContentBuilderString NODE = new XContentBuilderString("node"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startArray(Fields.SNAPSHOTS); + for (Entry entry : entries) { + toXContent(entry, builder, params); + } + builder.endArray(); + return builder; + } + + public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(Fields.REPOSITORY, entry.snapshotId().getRepository()); + builder.field(Fields.SNAPSHOT, entry.snapshotId().getSnapshot()); + builder.field(Fields.INCLUDE_GLOBAL_STATE, entry.includeGlobalState()); + builder.field(Fields.STATE, entry.state()); + builder.startArray(Fields.INDICES); + { + for (String index : entry.indices()) { + builder.value(index); + } + } + builder.endArray(); + builder.timeValueField(Fields.START_TIME_MILLIS, Fields.START_TIME, entry.startTime()); + builder.startArray(Fields.SHARDS); + { + for (Map.Entry shardEntry : entry.shards.entrySet()) { + ShardId shardId = shardEntry.getKey(); + ShardSnapshotStatus status = shardEntry.getValue(); + builder.startObject(); + { + builder.field(Fields.INDEX, shardId.getIndex()); + builder.field(Fields.SHARD, shardId.getId()); + builder.field(Fields.STATE, status.state()); + builder.field(Fields.NODE, status.nodeId()); + } + builder.endObject(); + } + } + builder.endArray(); + builder.endObject(); + } } diff --git a/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 2831af8183d..8692e5fb006 100644 --- a/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.google.common.collect.ImmutableList; import com.google.common.collect.UnmodifiableIterator; import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -44,9 +45,10 @@ import static com.google.common.collect.Lists.newArrayList; * This class holds all {@link DiscoveryNode} in the cluster and provides convenience methods to * access, modify merge / diff discovery nodes. */ -public class DiscoveryNodes implements Iterable { +public class DiscoveryNodes extends AbstractDiffable implements Iterable { public static final DiscoveryNodes EMPTY_NODES = builder().build(); + public static final DiscoveryNodes PROTO = EMPTY_NODES; private final ImmutableOpenMap nodes; private final ImmutableOpenMap dataNodes; @@ -567,6 +569,44 @@ public class DiscoveryNodes implements Iterable { } } + public void writeTo(StreamOutput out) throws IOException { + if (masterNodeId == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeString(masterNodeId); + } + out.writeVInt(nodes.size()); + for (DiscoveryNode node : this) { + node.writeTo(out); + } + } + + public DiscoveryNodes readFrom(StreamInput in, DiscoveryNode localNode) throws IOException { + Builder builder = new Builder(); + if (in.readBoolean()) { + builder.masterNodeId(in.readString()); + } + if (localNode != null) { + builder.localNodeId(localNode.id()); + } + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + DiscoveryNode node = DiscoveryNode.readNode(in); + if (localNode != null && node.id().equals(localNode.id())) { + // reuse the same instance of our address and local node id for faster equality + node = localNode; + } + builder.put(node); + } + return builder.build(); + } + + @Override + public DiscoveryNodes readFrom(StreamInput in) throws IOException { + return readFrom(in, localNode()); + } + public static Builder builder() { return new Builder(); } @@ -631,37 +671,8 @@ public class DiscoveryNodes implements Iterable { return new DiscoveryNodes(nodes.build(), dataNodesBuilder.build(), masterNodesBuilder.build(), masterNodeId, localNodeId, minNodeVersion, minNonClientNodeVersion); } - public static void writeTo(DiscoveryNodes nodes, StreamOutput out) throws IOException { - if (nodes.masterNodeId() == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeString(nodes.masterNodeId); - } - out.writeVInt(nodes.size()); - for (DiscoveryNode node : nodes) { - node.writeTo(out); - } - } - public static DiscoveryNodes readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { - Builder builder = new Builder(); - if (in.readBoolean()) { - builder.masterNodeId(in.readString()); - } - if (localNode != null) { - builder.localNodeId(localNode.id()); - } - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - DiscoveryNode node = DiscoveryNode.readNode(in); - if (localNode != null && node.id().equals(localNode.id())) { - // reuse the same instance of our address and local node id for faster equality - node = localNode; - } - builder.put(node); - } - return builder.build(); + return PROTO.readFrom(in, localNode); } } } diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index de4ed5434e1..6aaa260c4b5 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -25,6 +25,7 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.google.common.collect.UnmodifiableIterator; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -55,7 +56,9 @@ import static com.google.common.collect.Lists.newArrayList; * represented as {@link ShardRouting}. *

*/ -public class IndexRoutingTable implements Iterable { +public class IndexRoutingTable extends AbstractDiffable implements Iterable { + + public static final IndexRoutingTable PROTO = builder("").build(); private final String index; private final ShardShuffler shuffler; @@ -314,9 +317,51 @@ public class IndexRoutingTable implements Iterable { return new GroupShardsIterator(set); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexRoutingTable that = (IndexRoutingTable) o; + + if (!index.equals(that.index)) return false; + if (!shards.equals(that.shards)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = index.hashCode(); + result = 31 * result + shards.hashCode(); + return result; + } + public void validate() throws RoutingValidationException { } + @Override + public IndexRoutingTable readFrom(StreamInput in) throws IOException { + String index = in.readString(); + Builder builder = new Builder(index); + + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + builder.addIndexShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); + } + + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeVInt(shards.size()); + for (IndexShardRoutingTable indexShard : this) { + IndexShardRoutingTable.Builder.writeToThin(indexShard, out); + } + } + public static Builder builder(String index) { return new Builder(index); } @@ -338,30 +383,7 @@ public class IndexRoutingTable implements Iterable { * @throws IOException if something happens during read */ public static IndexRoutingTable readFrom(StreamInput in) throws IOException { - String index = in.readString(); - Builder builder = new Builder(index); - - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - builder.addIndexShard(IndexShardRoutingTable.Builder.readFromThin(in, index)); - } - - return builder.build(); - } - - /** - * Writes an {@link IndexRoutingTable} to a {@link StreamOutput}. - * - * @param index {@link IndexRoutingTable} to write - * @param out {@link StreamOutput} to write to - * @throws IOException if something happens during write - */ - public static void writeTo(IndexRoutingTable index, StreamOutput out) throws IOException { - out.writeString(index.index()); - out.writeVInt(index.shards.size()); - for (IndexShardRoutingTable indexShard : index) { - IndexShardRoutingTable.Builder.writeToThin(indexShard, out); - } + return PROTO.readFrom(in); } /** diff --git a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index 00e50b76129..2371b96f5b0 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -347,6 +347,28 @@ public class IndexShardRoutingTable implements Iterable { return new PlainShardIterator(shardId, ordered); } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexShardRoutingTable that = (IndexShardRoutingTable) o; + + if (primaryAllocatedPostApi != that.primaryAllocatedPostApi) return false; + if (!shardId.equals(that.shardId)) return false; + if (!shards.equals(that.shards)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = shardId.hashCode(); + result = 31 * result + shards.hashCode(); + result = 31 * result + (primaryAllocatedPostApi ? 1 : 0); + return result; + } + /** * Returns true iff all shards in the routing table are started otherwise false */ diff --git a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index 9f1b5db6c6b..25a8bac2f88 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -21,7 +21,7 @@ package org.elasticsearch.cluster.routing; import com.carrotsearch.hppc.IntSet; import com.google.common.collect.*; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.io.stream.StreamInput; @@ -44,7 +44,9 @@ import static com.google.common.collect.Maps.newHashMap; * * @see IndexRoutingTable */ -public class RoutingTable implements Iterable { +public class RoutingTable implements Iterable, Diffable { + + public static RoutingTable PROTO = builder().build(); public static final RoutingTable EMPTY_ROUTING_TABLE = builder().build(); @@ -254,6 +256,66 @@ public class RoutingTable implements Iterable { return new GroupShardsIterator(set); } + @Override + public Diff diff(RoutingTable previousState) { + return new RoutingTableDiff(previousState, this); + } + + @Override + public Diff readDiffFrom(StreamInput in) throws IOException { + return new RoutingTableDiff(in); + } + + @Override + public RoutingTable readFrom(StreamInput in) throws IOException { + Builder builder = new Builder(); + builder.version = in.readLong(); + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + IndexRoutingTable index = IndexRoutingTable.Builder.readFrom(in); + builder.add(index); + } + + return builder.build(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + out.writeVInt(indicesRouting.size()); + for (IndexRoutingTable index : indicesRouting.values()) { + index.writeTo(out); + } + } + + private static class RoutingTableDiff implements Diff { + + private final long version; + + private final Diff> indicesRouting; + + public RoutingTableDiff(RoutingTable before, RoutingTable after) { + version = after.version; + indicesRouting = DiffableUtils.diff(before.indicesRouting, after.indicesRouting); + } + + public RoutingTableDiff(StreamInput in) throws IOException { + version = in.readLong(); + indicesRouting = DiffableUtils.readImmutableMapDiff(in, IndexRoutingTable.PROTO); + } + + @Override + public RoutingTable apply(RoutingTable part) { + return new RoutingTable(version, indicesRouting.apply(part.indicesRouting)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(version); + indicesRouting.writeTo(out); + } + } + public static Builder builder() { return new Builder(); } @@ -403,6 +465,11 @@ public class RoutingTable implements Iterable { return this; } + public Builder indicesRouting(ImmutableMap indicesRouting) { + this.indicesRouting.putAll(indicesRouting); + return this; + } + public Builder remove(String index) { indicesRouting.remove(index); return this; @@ -422,23 +489,7 @@ public class RoutingTable implements Iterable { } public static RoutingTable readFrom(StreamInput in) throws IOException { - Builder builder = new Builder(); - builder.version = in.readLong(); - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - IndexRoutingTable index = IndexRoutingTable.Builder.readFrom(in); - builder.add(index); - } - - return builder.build(); - } - - public static void writeTo(RoutingTable table, StreamOutput out) throws IOException { - out.writeLong(table.version); - out.writeVInt(table.indicesRouting.size()); - for (IndexRoutingTable index : table.indicesRouting.values()) { - IndexRoutingTable.Builder.writeTo(index, out); - } + return PROTO.readFrom(in); } } @@ -450,5 +501,4 @@ public class RoutingTable implements Iterable { return sb.toString(); } - } diff --git a/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index 726a588d1bf..a3969dcc232 100644 --- a/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -142,19 +142,19 @@ public class DiskThresholdDecider extends AllocationDecider { private void warnAboutDiskIfNeeded(DiskUsage usage) { // Check absolute disk values if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdHigh.bytes()) { - logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", + logger.warn("high disk watermark [{} free] exceeded on {}, shards will be relocated away from this node", DiskThresholdDecider.this.freeBytesThresholdHigh, usage); } else if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdLow.bytes()) { - logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", + logger.info("low disk watermark [{} free] exceeded on {}, replicas will not be assigned to this node", DiskThresholdDecider.this.freeBytesThresholdLow, usage); } // Check percentage disk values if (usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdHigh) { - logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", + logger.warn("high disk watermark [{} free] exceeded on {}, shards will be relocated away from this node", Strings.format1Decimals(DiskThresholdDecider.this.freeDiskThresholdHigh, "%"), usage); } else if (usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdLow) { - logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", + logger.info("low disk watermark [{} free] exceeded on {}, replicas will not be assigned to this node", Strings.format1Decimals(DiskThresholdDecider.this.freeDiskThresholdLow, "%"), usage); } } diff --git a/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java b/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java index 17350ba6c04..b1823e5d74e 100644 --- a/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java +++ b/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java @@ -401,7 +401,7 @@ public class InternalClusterService extends AbstractLifecycleComponent { + /** + * Reads a copy of an object with the same type form the stream input + * + * The caller object remains unchanged. + */ + T readFrom(StreamInput in) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/common/io/stream/Writeable.java b/src/main/java/org/elasticsearch/common/io/stream/Writeable.java new file mode 100644 index 00000000000..9025315dc43 --- /dev/null +++ b/src/main/java/org/elasticsearch/common/io/stream/Writeable.java @@ -0,0 +1,30 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.io.stream; + +import java.io.IOException; + +public interface Writeable extends StreamableReader { + + /** + * Writes the current object into the output stream out + */ + void writeTo(StreamOutput out) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java b/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java index 1bc519435de..bfa4233d917 100644 --- a/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java +++ b/src/main/java/org/elasticsearch/common/transport/InetSocketTransportAddress.java @@ -38,6 +38,10 @@ public class InetSocketTransportAddress implements TransportAddress { InetSocketTransportAddress.resolveAddress = resolveAddress; } + public static boolean getResolveAddress() { + return resolveAddress; + } + private InetSocketAddress address; InetSocketTransportAddress() { diff --git a/src/main/java/org/elasticsearch/common/xcontent/XContentLocation.java b/src/main/java/org/elasticsearch/common/xcontent/XContentLocation.java new file mode 100644 index 00000000000..ade2a457797 --- /dev/null +++ b/src/main/java/org/elasticsearch/common/xcontent/XContentLocation.java @@ -0,0 +1,37 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.xcontent; + +/** + * Simple data structure representing the line and column number of a position + * in some XContent e.g. JSON. Locations are typically used to communicate the + * position of a parsing error to end users and consequently have line and + * column numbers starting from 1. + */ +public class XContentLocation { + public final int lineNumber; + public final int columnNumber; + + public XContentLocation(int lineNumber, int columnNumber) { + super(); + this.lineNumber = lineNumber; + this.columnNumber = columnNumber; + } +} diff --git a/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java b/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java index 0aab32c4ba3..738fd9f6e72 100644 --- a/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java +++ b/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java @@ -241,4 +241,12 @@ public interface XContentParser extends Releasable { * */ byte[] binaryValue() throws IOException; + + /** + * Used for error reporting to highlight where syntax errors occur in + * content being parsed. + * + * @return last token's location or null if cannot be determined + */ + XContentLocation getTokenLocation(); } diff --git a/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java b/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java index 08174e30a3e..5d3a3f99f4e 100644 --- a/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java +++ b/src/main/java/org/elasticsearch/common/xcontent/json/JsonXContentParser.java @@ -19,10 +19,13 @@ package org.elasticsearch.common.xcontent.json; +import com.fasterxml.jackson.core.JsonLocation; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; + import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; +import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.AbstractXContentParser; @@ -187,6 +190,15 @@ public class JsonXContentParser extends AbstractXContentParser { return parser.getBinaryValue(); } + @Override + public XContentLocation getTokenLocation() { + JsonLocation loc = parser.getTokenLocation(); + if (loc == null) { + return null; + } + return new XContentLocation(loc.getLineNr(), loc.getColumnNr()); + } + @Override public void close() { IOUtils.closeWhileHandlingException(parser); diff --git a/src/main/java/org/elasticsearch/discovery/Discovery.java b/src/main/java/org/elasticsearch/discovery/Discovery.java index dfd51e6348f..36b8e5da6f5 100644 --- a/src/main/java/org/elasticsearch/discovery/Discovery.java +++ b/src/main/java/org/elasticsearch/discovery/Discovery.java @@ -19,6 +19,7 @@ package org.elasticsearch.discovery; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -59,7 +60,7 @@ public interface Discovery extends LifecycleComponent { * The {@link AckListener} allows to keep track of the ack received from nodes, and verify whether * they updated their own cluster state or not. */ - void publish(ClusterState clusterState, AckListener ackListener); + void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener); public static interface AckListener { void onNodeAck(DiscoveryNode node, @Nullable Throwable t); diff --git a/src/main/java/org/elasticsearch/discovery/DiscoveryService.java b/src/main/java/org/elasticsearch/discovery/DiscoveryService.java index 1f7207abd5b..a95c313447b 100644 --- a/src/main/java/org/elasticsearch/discovery/DiscoveryService.java +++ b/src/main/java/org/elasticsearch/discovery/DiscoveryService.java @@ -21,6 +21,7 @@ package org.elasticsearch.discovery; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -132,9 +133,9 @@ public class DiscoveryService extends AbstractLifecycleComponent implem private static final ConcurrentMap clusterGroups = ConcurrentCollections.newConcurrentMap(); + private volatile ClusterState lastProcessedClusterState; + @Inject public LocalDiscovery(Settings settings, ClusterName clusterName, TransportService transportService, ClusterService clusterService, DiscoveryNodeService discoveryNodeService, Version version, DiscoverySettings discoverySettings) { @@ -273,7 +279,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } @Override - public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { + public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { if (!master) { throw new IllegalStateException("Shouldn't publish state when not master"); } @@ -286,7 +292,7 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem } nodesToPublishTo.add(localDiscovery.localNode); } - publish(members, clusterState, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(members, clusterChangedEvent, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } } @@ -299,17 +305,47 @@ public class LocalDiscovery extends AbstractLifecycleComponent implem return members.toArray(new LocalDiscovery[members.size()]); } - private void publish(LocalDiscovery[] members, ClusterState clusterState, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { + private void publish(LocalDiscovery[] members, ClusterChangedEvent clusterChangedEvent, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { try { // we do the marshaling intentionally, to check it works well... - final byte[] clusterStateBytes = Builder.toBytes(clusterState); + byte[] clusterStateBytes = null; + byte[] clusterStateDiffBytes = null; + ClusterState clusterState = clusterChangedEvent.state(); for (final LocalDiscovery discovery : members) { if (discovery.master) { continue; } - final ClusterState nodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode); + ClusterState newNodeSpecificClusterState = null; + synchronized (this) { + // we do the marshaling intentionally, to check it works well... + // check if we publsihed cluster state at least once and node was in the cluster when we published cluster state the last time + if (discovery.lastProcessedClusterState != null && clusterChangedEvent.previousState().nodes().nodeExists(discovery.localNode.id())) { + // both conditions are true - which means we can try sending cluster state as diffs + if (clusterStateDiffBytes == null) { + Diff diff = clusterState.diff(clusterChangedEvent.previousState()); + BytesStreamOutput os = new BytesStreamOutput(); + diff.writeTo(os); + clusterStateDiffBytes = os.bytes().toBytes(); + } + try { + newNodeSpecificClusterState = discovery.lastProcessedClusterState.readDiffFrom(new BytesStreamInput(clusterStateDiffBytes)).apply(discovery.lastProcessedClusterState); + logger.debug("sending diff cluster state version with size {} to [{}]", clusterStateDiffBytes.length, discovery.localNode.getName()); + } catch (IncompatibleClusterStateVersionException ex) { + logger.warn("incompatible cluster state version - resending complete cluster state", ex); + } + } + if (newNodeSpecificClusterState == null) { + if (clusterStateBytes == null) { + clusterStateBytes = Builder.toBytes(clusterState); + } + newNodeSpecificClusterState = ClusterState.Builder.fromBytes(clusterStateBytes, discovery.localNode); + } + discovery.lastProcessedClusterState = newNodeSpecificClusterState; + } + final ClusterState nodeSpecificClusterState = newNodeSpecificClusterState; + nodeSpecificClusterState.status(ClusterState.ClusterStateStatus.RECEIVED); // ignore cluster state messages that do not include "me", not in the game yet... if (nodeSpecificClusterState.nodes().localNode() != null) { diff --git a/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 0defcb7edd5..5bec60abf04 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -22,7 +22,6 @@ package org.elasticsearch.discovery.zen; import com.google.common.base.Objects; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.*; import org.elasticsearch.cluster.block.ClusterBlocks; @@ -329,12 +328,12 @@ public class ZenDiscovery extends AbstractLifecycleComponent implemen @Override - public void publish(ClusterState clusterState, AckListener ackListener) { - if (!clusterState.getNodes().localNodeMaster()) { + public void publish(ClusterChangedEvent clusterChangedEvent, AckListener ackListener) { + if (!clusterChangedEvent.state().getNodes().localNodeMaster()) { throw new IllegalStateException("Shouldn't publish state when not master"); } - nodesFD.updateNodesAndPing(clusterState); - publishClusterState.publish(clusterState, ackListener); + nodesFD.updateNodesAndPing(clusterChangedEvent.state()); + publishClusterState.publish(clusterChangedEvent, ackListener); } /** diff --git a/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java b/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java index fd1ba85c25c..c4ad8895e79 100644 --- a/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java +++ b/src/main/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateAction.java @@ -21,8 +21,12 @@ package org.elasticsearch.discovery.zen.publish; import com.google.common.collect.Maps; import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.IncompatibleClusterStateVersionException; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.compress.Compressor; @@ -40,10 +44,13 @@ import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.*; +import java.io.IOException; import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; /** * @@ -83,73 +90,43 @@ public class PublishClusterStateAction extends AbstractComponent { transportService.removeHandler(ACTION_NAME); } - public void publish(ClusterState clusterState, final Discovery.AckListener ackListener) { - Set nodesToPublishTo = new HashSet<>(clusterState.nodes().size()); + public void publish(ClusterChangedEvent clusterChangedEvent, final Discovery.AckListener ackListener) { + Set nodesToPublishTo = new HashSet<>(clusterChangedEvent.state().nodes().size()); DiscoveryNode localNode = nodesProvider.nodes().localNode(); - for (final DiscoveryNode node : clusterState.nodes()) { + for (final DiscoveryNode node : clusterChangedEvent.state().nodes()) { if (node.equals(localNode)) { continue; } nodesToPublishTo.add(node); } - publish(clusterState, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); + publish(clusterChangedEvent, nodesToPublishTo, new AckClusterStatePublishResponseHandler(nodesToPublishTo, ackListener)); } - private void publish(final ClusterState clusterState, final Set nodesToPublishTo, + private void publish(final ClusterChangedEvent clusterChangedEvent, final Set nodesToPublishTo, final BlockingClusterStatePublishResponseHandler publishResponseHandler) { Map serializedStates = Maps.newHashMap(); + Map serializedDiffs = Maps.newHashMap(); + final ClusterState clusterState = clusterChangedEvent.state(); + final ClusterState previousState = clusterChangedEvent.previousState(); final AtomicBoolean timedOutWaitingForNodes = new AtomicBoolean(false); final TimeValue publishTimeout = discoverySettings.getPublishTimeout(); + final boolean sendFullVersion = !discoverySettings.getPublishDiff() || previousState == null; + Diff diff = null; for (final DiscoveryNode node : nodesToPublishTo) { // try and serialize the cluster state once (or per version), so we don't serialize it // per node when we send it over the wire, compress it while we are at it... - BytesReference bytes = serializedStates.get(node.version()); - if (bytes == null) { - try { - BytesStreamOutput bStream = new BytesStreamOutput(); - StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); - stream.setVersion(node.version()); - ClusterState.Builder.writeTo(clusterState, stream); - stream.close(); - bytes = bStream.bytes(); - serializedStates.put(node.version(), bytes); - } catch (Throwable e) { - logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node); - publishResponseHandler.onFailure(node, e); - continue; + // we don't send full version if node didn't exist in the previous version of cluster state + if (sendFullVersion || !previousState.nodes().nodeExists(node.id())) { + sendFullClusterState(clusterState, serializedStates, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); + } else { + if (diff == null) { + diff = clusterState.diff(previousState); } - } - try { - TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false); - // no need to put a timeout on the options here, because we want the response to eventually be received - // and not log an error if it arrives after the timeout - transportService.sendRequest(node, ACTION_NAME, - new BytesTransportRequest(bytes, node.version()), - options, // no need to compress, we already compressed the bytes - - new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - - @Override - public void handleResponse(TransportResponse.Empty response) { - if (timedOutWaitingForNodes.get()) { - logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, clusterState.version(), publishTimeout); - } - publishResponseHandler.onResponse(node); - } - - @Override - public void handleException(TransportException exp) { - logger.debug("failed to send cluster state to {}", exp, node); - publishResponseHandler.onFailure(node, exp); - } - }); - } catch (Throwable t) { - logger.debug("error sending cluster state to {}", t, node); - publishResponseHandler.onFailure(node, t); + sendClusterStateDiff(clusterState, diff, serializedDiffs, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); } } @@ -171,7 +148,107 @@ public class PublishClusterStateAction extends AbstractComponent { } } + private void sendFullClusterState(ClusterState clusterState, @Nullable Map serializedStates, + DiscoveryNode node, AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout, + BlockingClusterStatePublishResponseHandler publishResponseHandler) { + BytesReference bytes = null; + if (serializedStates != null) { + bytes = serializedStates.get(node.version()); + } + if (bytes == null) { + try { + bytes = serializeFullClusterState(clusterState, node.version()); + if (serializedStates != null) { + serializedStates.put(node.version(), bytes); + } + } catch (Throwable e) { + logger.warn("failed to serialize cluster_state before publishing it to node {}", e, node); + publishResponseHandler.onFailure(node, e); + return; + } + } + publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, false); + } + + private void sendClusterStateDiff(ClusterState clusterState, Diff diff, Map serializedDiffs, DiscoveryNode node, + AtomicBoolean timedOutWaitingForNodes, TimeValue publishTimeout, + BlockingClusterStatePublishResponseHandler publishResponseHandler) { + BytesReference bytes = serializedDiffs.get(node.version()); + if (bytes == null) { + try { + bytes = serializeDiffClusterState(diff, node.version()); + serializedDiffs.put(node.version(), bytes); + } catch (Throwable e) { + logger.warn("failed to serialize diff of cluster_state before publishing it to node {}", e, node); + publishResponseHandler.onFailure(node, e); + return; + } + } + publishClusterStateToNode(clusterState, bytes, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler, true); + } + + private void publishClusterStateToNode(final ClusterState clusterState, BytesReference bytes, + final DiscoveryNode node, final AtomicBoolean timedOutWaitingForNodes, + final TimeValue publishTimeout, + final BlockingClusterStatePublishResponseHandler publishResponseHandler, + final boolean sendDiffs) { + try { + TransportRequestOptions options = TransportRequestOptions.options().withType(TransportRequestOptions.Type.STATE).withCompress(false); + // no need to put a timeout on the options here, because we want the response to eventually be received + // and not log an error if it arrives after the timeout + transportService.sendRequest(node, ACTION_NAME, + new BytesTransportRequest(bytes, node.version()), + options, // no need to compress, we already compressed the bytes + + new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { + + @Override + public void handleResponse(TransportResponse.Empty response) { + if (timedOutWaitingForNodes.get()) { + logger.debug("node {} responded for cluster state [{}] (took longer than [{}])", node, clusterState.version(), publishTimeout); + } + publishResponseHandler.onResponse(node); + } + + @Override + public void handleException(TransportException exp) { + if (sendDiffs && exp.unwrapCause() instanceof IncompatibleClusterStateVersionException) { + logger.debug("resending full cluster state to node {} reason {}", node, exp.getDetailedMessage()); + sendFullClusterState(clusterState, null, node, timedOutWaitingForNodes, publishTimeout, publishResponseHandler); + } else { + logger.debug("failed to send cluster state to {}", exp, node); + publishResponseHandler.onFailure(node, exp); + } + } + }); + } catch (Throwable t) { + logger.warn("error sending cluster state to {}", t, node); + publishResponseHandler.onFailure(node, t); + } + } + + public static BytesReference serializeFullClusterState(ClusterState clusterState, Version nodeVersion) throws IOException { + BytesStreamOutput bStream = new BytesStreamOutput(); + StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); + stream.setVersion(nodeVersion); + stream.writeBoolean(true); + clusterState.writeTo(stream); + stream.close(); + return bStream.bytes(); + } + + public static BytesReference serializeDiffClusterState(Diff diff, Version nodeVersion) throws IOException { + BytesStreamOutput bStream = new BytesStreamOutput(); + StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); + stream.setVersion(nodeVersion); + stream.writeBoolean(false); + diff.writeTo(stream); + stream.close(); + return bStream.bytes(); + } + private class PublishClusterStateRequestHandler implements TransportRequestHandler { + private ClusterState lastSeenClusterState; @Override public void messageReceived(BytesTransportRequest request, final TransportChannel channel) throws Exception { @@ -183,11 +260,24 @@ public class PublishClusterStateAction extends AbstractComponent { in = request.bytes().streamInput(); } in.setVersion(request.version()); - ClusterState clusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); - clusterState.status(ClusterState.ClusterStateStatus.RECEIVED); - logger.debug("received cluster state version {}", clusterState.version()); + synchronized (this) { + // If true we received full cluster state - otherwise diffs + if (in.readBoolean()) { + lastSeenClusterState = ClusterState.Builder.readFrom(in, nodesProvider.nodes().localNode()); + logger.debug("received full cluster state version {} with size {}", lastSeenClusterState.version(), request.bytes().length()); + } else if (lastSeenClusterState != null) { + Diff diff = lastSeenClusterState.readDiffFrom(in); + lastSeenClusterState = diff.apply(lastSeenClusterState); + logger.debug("received diff cluster state version {} with uuid {}, diff size {}", lastSeenClusterState.version(), lastSeenClusterState.uuid(), request.bytes().length()); + } else { + logger.debug("received diff for but don't have any local cluster state - requesting full state"); + throw new IncompatibleClusterStateVersionException("have no local cluster state"); + } + lastSeenClusterState.status(ClusterState.ClusterStateStatus.RECEIVED); + } + try { - listener.onNewClusterState(clusterState, new NewClusterStateListener.NewStateProcessed() { + listener.onNewClusterState(lastSeenClusterState, new NewClusterStateListener.NewStateProcessed() { @Override public void onNewClusterStateProcessed() { try { @@ -207,7 +297,7 @@ public class PublishClusterStateAction extends AbstractComponent { } }); } catch (Exception e) { - logger.warn("unexpected error while processing cluster state version [{}]", e, clusterState.version()); + logger.warn("unexpected error while processing cluster state version [{}]", e, lastSeenClusterState.version()); try { channel.sendResponse(e); } catch (Throwable e1) { diff --git a/src/main/java/org/elasticsearch/gateway/Gateway.java b/src/main/java/org/elasticsearch/gateway/Gateway.java index cd15bccdc4a..139b5763489 100644 --- a/src/main/java/org/elasticsearch/gateway/Gateway.java +++ b/src/main/java/org/elasticsearch/gateway/Gateway.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.NodeEnvironment; -import org.elasticsearch.indices.IndicesService; + import java.nio.file.Path; diff --git a/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java b/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java index 43dec7edb51..5538ef6d043 100644 --- a/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java +++ b/src/main/java/org/elasticsearch/gateway/LocalAllocateDangledIndices.java @@ -198,7 +198,7 @@ public class LocalAllocateDangledIndices extends AbstractComponent { fromNode.writeTo(out); out.writeVInt(indices.length); for (IndexMetaData indexMetaData : indices) { - IndexMetaData.Builder.writeTo(indexMetaData, out); + indexMetaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java b/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java index 36372009f87..900a2e7ffc7 100644 --- a/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java +++ b/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayMetaState.java @@ -221,7 +221,7 @@ public class TransportNodesListGatewayMetaState extends TransportNodesOperationA out.writeBoolean(false); } else { out.writeBoolean(true); - MetaData.Builder.writeTo(metaData, out); + metaData.writeTo(out); } } } diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java index 486101f741f..fd4cce1c763 100644 --- a/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java @@ -223,7 +223,7 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent imple context.setMapUnmappedFieldAsString(mapUnmappedFieldsAsString ? true : false); return queryParserService.parseInnerQuery(context); } catch (IOException e) { - throw new QueryParsingException(queryParserService.index(), "Failed to parse", e); + throw new QueryParsingException(context, "Failed to parse", e); } finally { if (type != null) { QueryParseContext.setTypes(previousTypes); diff --git a/src/main/java/org/elasticsearch/index/query/AndFilterParser.java b/src/main/java/org/elasticsearch/index/query/AndFilterParser.java index 176a8c6dd7b..02322db9a0b 100644 --- a/src/main/java/org/elasticsearch/index/query/AndFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/AndFilterParser.java @@ -100,14 +100,14 @@ public class AndFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[and] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[and] filter does not support [" + currentFieldName + "]"); } } } } if (!filtersFound) { - throw new QueryParsingException(parseContext.index(), "[and] filter requires 'filters' to be set on it'"); + throw new QueryParsingException(parseContext, "[and] filter requires 'filters' to be set on it'"); } if (filters.isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/BoolFilterParser.java b/src/main/java/org/elasticsearch/index/query/BoolFilterParser.java index fcd2e68c8b4..71f8b8248f7 100644 --- a/src/main/java/org/elasticsearch/index/query/BoolFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/BoolFilterParser.java @@ -85,7 +85,7 @@ public class BoolFilterParser implements FilterParser { boolFilter.add(new BooleanClause(filter, BooleanClause.Occur.SHOULD)); } } else { - throw new QueryParsingException(parseContext.index(), "[bool] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[bool] filter does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if ("must".equals(currentFieldName)) { @@ -114,7 +114,7 @@ public class BoolFilterParser implements FilterParser { } } } else { - throw new QueryParsingException(parseContext.index(), "[bool] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[bool] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("_cache".equals(currentFieldName)) { @@ -124,13 +124,13 @@ public class BoolFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[bool] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[bool] filter does not support [" + currentFieldName + "]"); } } } if (!hasAnyFilter) { - throw new QueryParsingException(parseContext.index(), "[bool] filter has no inner should/must/must_not elements"); + throw new QueryParsingException(parseContext, "[bool] filter has no inner should/must/must_not elements"); } if (boolFilter.clauses().isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java b/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java index 29d4ba2edd5..b7c31647c94 100644 --- a/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/BoolQueryParser.java @@ -85,7 +85,7 @@ public class BoolQueryParser implements QueryParser { clauses.add(new BooleanClause(query, BooleanClause.Occur.SHOULD)); } } else { - throw new QueryParsingException(parseContext.index(), "[bool] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[bool] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if ("must".equals(currentFieldName)) { @@ -110,7 +110,7 @@ public class BoolQueryParser implements QueryParser { } } } else { - throw new QueryParsingException(parseContext.index(), "bool query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "bool query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("disable_coord".equals(currentFieldName) || "disableCoord".equals(currentFieldName)) { @@ -126,7 +126,7 @@ public class BoolQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[bool] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[bool] query does not support [" + currentFieldName + "]"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/BoostingQueryParser.java b/src/main/java/org/elasticsearch/index/query/BoostingQueryParser.java index a117256ece1..c160b2f9a4a 100644 --- a/src/main/java/org/elasticsearch/index/query/BoostingQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/BoostingQueryParser.java @@ -66,7 +66,7 @@ public class BoostingQueryParser implements QueryParser { negativeQuery = parseContext.parseInnerQuery(); negativeQueryFound = true; } else { - throw new QueryParsingException(parseContext.index(), "[boosting] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[boosting] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("negative_boost".equals(currentFieldName) || "negativeBoost".equals(currentFieldName)) { @@ -74,19 +74,19 @@ public class BoostingQueryParser implements QueryParser { } else if ("boost".equals(currentFieldName)) { boost = parser.floatValue(); } else { - throw new QueryParsingException(parseContext.index(), "[boosting] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[boosting] query does not support [" + currentFieldName + "]"); } } } if (positiveQuery == null && !positiveQueryFound) { - throw new QueryParsingException(parseContext.index(), "[boosting] query requires 'positive' query to be set'"); + throw new QueryParsingException(parseContext, "[boosting] query requires 'positive' query to be set'"); } if (negativeQuery == null && !negativeQueryFound) { - throw new QueryParsingException(parseContext.index(), "[boosting] query requires 'negative' query to be set'"); + throw new QueryParsingException(parseContext, "[boosting] query requires 'negative' query to be set'"); } if (negativeBoost == -1) { - throw new QueryParsingException(parseContext.index(), "[boosting] query requires 'negative_boost' to be set'"); + throw new QueryParsingException(parseContext, "[boosting] query requires 'negative_boost' to be set'"); } // parsers returned null diff --git a/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java b/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java index 223717084e6..3e313ce8408 100644 --- a/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java +++ b/src/main/java/org/elasticsearch/index/query/CommonTermsQueryBuilder.java @@ -19,6 +19,9 @@ package org.elasticsearch.index.query; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.similarities.Similarity; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; diff --git a/src/main/java/org/elasticsearch/index/query/CommonTermsQueryParser.java b/src/main/java/org/elasticsearch/index/query/CommonTermsQueryParser.java index 683b8dfd9ba..29945de5686 100644 --- a/src/main/java/org/elasticsearch/index/query/CommonTermsQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/CommonTermsQueryParser.java @@ -65,7 +65,7 @@ public class CommonTermsQueryParser implements QueryParser { XContentParser parser = parseContext.parser(); XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[common] query malformed, no field"); + throw new QueryParsingException(parseContext, "[common] query malformed, no field"); } String fieldName = parser.currentName(); Object value = null; @@ -96,12 +96,13 @@ public class CommonTermsQueryParser implements QueryParser { } else if ("high_freq".equals(innerFieldName) || "highFreq".equals(innerFieldName)) { highFreqMinimumShouldMatch = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[common] query does not support [" + innerFieldName + "] for [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[common] query does not support [" + innerFieldName + + "] for [" + currentFieldName + "]"); } } } } else { - throw new QueryParsingException(parseContext.index(), "[common] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[common] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("query".equals(currentFieldName)) { @@ -109,7 +110,7 @@ public class CommonTermsQueryParser implements QueryParser { } else if ("analyzer".equals(currentFieldName)) { String analyzer = parser.text(); if (parseContext.analysisService().analyzer(analyzer) == null) { - throw new QueryParsingException(parseContext.index(), "[common] analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[common] analyzer [" + parser.text() + "] not found"); } queryAnalyzer = analyzer; } else if ("disable_coord".equals(currentFieldName) || "disableCoord".equals(currentFieldName)) { @@ -123,7 +124,7 @@ public class CommonTermsQueryParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { highFreqOccur = BooleanClause.Occur.MUST; } else { - throw new QueryParsingException(parseContext.index(), + throw new QueryParsingException(parseContext, "[common] query requires operator to be either 'and' or 'or', not [" + op + "]"); } } else if ("low_freq_operator".equals(currentFieldName) || "lowFreqOperator".equals(currentFieldName)) { @@ -133,7 +134,7 @@ public class CommonTermsQueryParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { lowFreqOccur = BooleanClause.Occur.MUST; } else { - throw new QueryParsingException(parseContext.index(), + throw new QueryParsingException(parseContext, "[common] query requires operator to be either 'and' or 'or', not [" + op + "]"); } } else if ("minimum_should_match".equals(currentFieldName) || "minimumShouldMatch".equals(currentFieldName)) { @@ -143,7 +144,7 @@ public class CommonTermsQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[common] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[common] query does not support [" + currentFieldName + "]"); } } } @@ -154,13 +155,13 @@ public class CommonTermsQueryParser implements QueryParser { token = parser.nextToken(); if (token != XContentParser.Token.END_OBJECT) { throw new QueryParsingException( - parseContext.index(), + parseContext, "[common] query parsed in simplified form, with direct field name, but included more options than just the field name, possibly use its 'options' form, with 'query' element?"); } } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No text specified for text query"); + throw new QueryParsingException(parseContext, "No text specified for text query"); } FieldMapper mapper = null; String field; diff --git a/src/main/java/org/elasticsearch/index/query/ConstantScoreQueryParser.java b/src/main/java/org/elasticsearch/index/query/ConstantScoreQueryParser.java index 78c5879b63f..d89ff05b7fa 100644 --- a/src/main/java/org/elasticsearch/index/query/ConstantScoreQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/ConstantScoreQueryParser.java @@ -71,7 +71,7 @@ public class ConstantScoreQueryParser implements QueryParser { query = parseContext.parseInnerQuery(); queryFound = true; } else { - throw new QueryParsingException(parseContext.index(), "[constant_score] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[constant_score] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("boost".equals(currentFieldName)) { @@ -81,12 +81,12 @@ public class ConstantScoreQueryParser implements QueryParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[constant_score] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[constant_score] query does not support [" + currentFieldName + "]"); } } } if (!filterFound && !queryFound) { - throw new QueryParsingException(parseContext.index(), "[constant_score] requires either 'filter' or 'query' element"); + throw new QueryParsingException(parseContext, "[constant_score] requires either 'filter' or 'query' element"); } if (query == null && filter == null) { diff --git a/src/main/java/org/elasticsearch/index/query/DisMaxQueryParser.java b/src/main/java/org/elasticsearch/index/query/DisMaxQueryParser.java index 82feb9854a5..2747387fbd7 100644 --- a/src/main/java/org/elasticsearch/index/query/DisMaxQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/DisMaxQueryParser.java @@ -70,7 +70,7 @@ public class DisMaxQueryParser implements QueryParser { queries.add(query); } } else { - throw new QueryParsingException(parseContext.index(), "[dis_max] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[dis_max] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if ("queries".equals(currentFieldName)) { @@ -83,7 +83,7 @@ public class DisMaxQueryParser implements QueryParser { token = parser.nextToken(); } } else { - throw new QueryParsingException(parseContext.index(), "[dis_max] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[dis_max] query does not support [" + currentFieldName + "]"); } } else { if ("boost".equals(currentFieldName)) { @@ -93,13 +93,13 @@ public class DisMaxQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[dis_max] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[dis_max] query does not support [" + currentFieldName + "]"); } } } if (!queriesFound) { - throw new QueryParsingException(parseContext.index(), "[dis_max] requires 'queries' field"); + throw new QueryParsingException(parseContext, "[dis_max] requires 'queries' field"); } if (queries.isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/ExistsFilterParser.java b/src/main/java/org/elasticsearch/index/query/ExistsFilterParser.java index eb03586adf2..008f554a57f 100644 --- a/src/main/java/org/elasticsearch/index/query/ExistsFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/ExistsFilterParser.java @@ -23,8 +23,6 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Filter; import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryWrapperFilter; -import org.apache.lucene.search.TermRangeFilter; import org.apache.lucene.search.TermRangeQuery; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.HashedBytesRef; @@ -71,13 +69,13 @@ public class ExistsFilterParser implements FilterParser { } else if ("_name".equals(currentFieldName)) { filterName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[exists] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[exists] filter does not support [" + currentFieldName + "]"); } } } if (fieldPattern == null) { - throw new QueryParsingException(parseContext.index(), "exists must be provided with a [field]"); + throw new QueryParsingException(parseContext, "exists must be provided with a [field]"); } return newFilter(parseContext, fieldPattern, filterName); diff --git a/src/main/java/org/elasticsearch/index/query/FQueryFilterParser.java b/src/main/java/org/elasticsearch/index/query/FQueryFilterParser.java index cb821912ca9..d31e2f1a943 100644 --- a/src/main/java/org/elasticsearch/index/query/FQueryFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/FQueryFilterParser.java @@ -66,7 +66,7 @@ public class FQueryFilterParser implements FilterParser { queryFound = true; query = parseContext.parseInnerQuery(); } else { - throw new QueryParsingException(parseContext.index(), "[fquery] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[fquery] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("_name".equals(currentFieldName)) { @@ -76,12 +76,12 @@ public class FQueryFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[fquery] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[fquery] filter does not support [" + currentFieldName + "]"); } } } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[fquery] requires 'query' element"); + throw new QueryParsingException(parseContext, "[fquery] requires 'query' element"); } if (query == null) { return null; diff --git a/src/main/java/org/elasticsearch/index/query/FieldMaskingSpanQueryParser.java b/src/main/java/org/elasticsearch/index/query/FieldMaskingSpanQueryParser.java index 2b69cf61561..1e8fd7cfa03 100644 --- a/src/main/java/org/elasticsearch/index/query/FieldMaskingSpanQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/FieldMaskingSpanQueryParser.java @@ -64,11 +64,12 @@ public class FieldMaskingSpanQueryParser implements QueryParser { if ("query".equals(currentFieldName)) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "[field_masking_span] query] must be of type span query"); + throw new QueryParsingException(parseContext, "[field_masking_span] query] must be of type span query"); } inner = (SpanQuery) query; } else { - throw new QueryParsingException(parseContext.index(), "[field_masking_span] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[field_masking_span] query does not support [" + + currentFieldName + "]"); } } else { if ("boost".equals(currentFieldName)) { @@ -78,15 +79,15 @@ public class FieldMaskingSpanQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[field_masking_span] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[field_masking_span] query does not support [" + currentFieldName + "]"); } } } if (inner == null) { - throw new QueryParsingException(parseContext.index(), "field_masking_span must have [query] span query clause"); + throw new QueryParsingException(parseContext, "field_masking_span must have [query] span query clause"); } if (field == null) { - throw new QueryParsingException(parseContext.index(), "field_masking_span must have [field] set for it"); + throw new QueryParsingException(parseContext, "field_masking_span must have [field] set for it"); } FieldMapper mapper = parseContext.fieldMapper(field); diff --git a/src/main/java/org/elasticsearch/index/query/FilteredQueryParser.java b/src/main/java/org/elasticsearch/index/query/FilteredQueryParser.java index e1e27eec64b..f6ec14313b1 100644 --- a/src/main/java/org/elasticsearch/index/query/FilteredQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/FilteredQueryParser.java @@ -73,7 +73,7 @@ public class FilteredQueryParser implements QueryParser { filterFound = true; filter = parseContext.parseInnerFilter(); } else { - throw new QueryParsingException(parseContext.index(), "[filtered] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[filtered] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("strategy".equals(currentFieldName)) { @@ -93,7 +93,7 @@ public class FilteredQueryParser implements QueryParser { } else if ("leap_frog_filter_first".equals(value) || "leapFrogFilterFirst".equals(value)) { filterStrategy = FilteredQuery.LEAP_FROG_FILTER_FIRST_STRATEGY; } else { - throw new QueryParsingException(parseContext.index(), "[filtered] strategy value not supported [" + value + "]"); + throw new QueryParsingException(parseContext, "[filtered] strategy value not supported [" + value + "]"); } } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); @@ -104,7 +104,7 @@ public class FilteredQueryParser implements QueryParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[filtered] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[filtered] query does not support [" + currentFieldName + "]"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java b/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java index 243f86534cd..229fcc95c72 100644 --- a/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/FuzzyQueryParser.java @@ -57,7 +57,7 @@ public class FuzzyQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[fuzzy] query malformed, no field"); + throw new QueryParsingException(parseContext, "[fuzzy] query malformed, no field"); } String fieldName = parser.currentName(); @@ -95,7 +95,7 @@ public class FuzzyQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[fuzzy] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[fuzzy] query does not support [" + currentFieldName + "]"); } } } @@ -107,7 +107,7 @@ public class FuzzyQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for fuzzy query"); + throw new QueryParsingException(parseContext, "No value specified for fuzzy query"); } Query query = null; diff --git a/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxFilterParser.java b/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxFilterParser.java index 8f68dbea074..107e3a507dd 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxFilterParser.java @@ -147,7 +147,7 @@ public class GeoBoundingBoxFilterParser implements FilterParser { } else if ("type".equals(currentFieldName)) { type = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[geo_bbox] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_bbox] filter does not support [" + currentFieldName + "]"); } } } @@ -169,11 +169,11 @@ public class GeoBoundingBoxFilterParser implements FilterParser { MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "failed to find geo_point field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "failed to find geo_point field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.mapper(); if (!(mapper instanceof GeoPointFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "field [" + fieldName + "] is not a geo_point field"); + throw new QueryParsingException(parseContext, "field [" + fieldName + "] is not a geo_point field"); } GeoPointFieldMapper geoMapper = ((GeoPointFieldMapper) mapper); @@ -184,7 +184,8 @@ public class GeoBoundingBoxFilterParser implements FilterParser { IndexGeoPointFieldData indexFieldData = parseContext.getForField(mapper); filter = new InMemoryGeoBoundingBoxFilter(topLeft, bottomRight, indexFieldData); } else { - throw new QueryParsingException(parseContext.index(), "geo bounding box type [" + type + "] not supported, either 'indexed' or 'memory' are allowed"); + throw new QueryParsingException(parseContext, "geo bounding box type [" + type + + "] not supported, either 'indexed' or 'memory' are allowed"); } if (cache != null) { diff --git a/src/main/java/org/elasticsearch/index/query/GeoDistanceFilterParser.java b/src/main/java/org/elasticsearch/index/query/GeoDistanceFilterParser.java index 252afdf25cf..a7859977388 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoDistanceFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoDistanceFilterParser.java @@ -98,7 +98,8 @@ public class GeoDistanceFilterParser implements FilterParser { } else if (currentName.equals(GeoPointFieldMapper.Names.GEOHASH)) { GeoHashUtils.decode(parser.text(), point); } else { - throw new QueryParsingException(parseContext.index(), "[geo_distance] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_distance] filter does not support [" + currentFieldName + + "]"); } } } @@ -141,7 +142,7 @@ public class GeoDistanceFilterParser implements FilterParser { } if (vDistance == null) { - throw new QueryParsingException(parseContext.index(), "geo_distance requires 'distance' to be specified"); + throw new QueryParsingException(parseContext, "geo_distance requires 'distance' to be specified"); } else if (vDistance instanceof Number) { distance = DistanceUnit.DEFAULT.convert(((Number) vDistance).doubleValue(), unit); } else { @@ -155,11 +156,11 @@ public class GeoDistanceFilterParser implements FilterParser { MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "failed to find geo_point field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "failed to find geo_point field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.mapper(); if (!(mapper instanceof GeoPointFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "field [" + fieldName + "] is not a geo_point field"); + throw new QueryParsingException(parseContext, "field [" + fieldName + "] is not a geo_point field"); } GeoPointFieldMapper geoMapper = ((GeoPointFieldMapper) mapper); diff --git a/src/main/java/org/elasticsearch/index/query/GeoDistanceRangeFilterParser.java b/src/main/java/org/elasticsearch/index/query/GeoDistanceRangeFilterParser.java index b7452bec0f1..113c59d2c83 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoDistanceRangeFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoDistanceRangeFilterParser.java @@ -196,11 +196,11 @@ public class GeoDistanceRangeFilterParser implements FilterParser { MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "failed to find geo_point field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "failed to find geo_point field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.mapper(); if (!(mapper instanceof GeoPointFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "field [" + fieldName + "] is not a geo_point field"); + throw new QueryParsingException(parseContext, "field [" + fieldName + "] is not a geo_point field"); } GeoPointFieldMapper geoMapper = ((GeoPointFieldMapper) mapper); diff --git a/src/main/java/org/elasticsearch/index/query/GeoPolygonFilterParser.java b/src/main/java/org/elasticsearch/index/query/GeoPolygonFilterParser.java index fefa37c07e3..e63c6012ede 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoPolygonFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoPolygonFilterParser.java @@ -96,10 +96,12 @@ public class GeoPolygonFilterParser implements FilterParser { shell.add(GeoUtils.parseGeoPoint(parser)); } } else { - throw new QueryParsingException(parseContext.index(), "[geo_polygon] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_polygon] filter does not support [" + currentFieldName + + "]"); } } else { - throw new QueryParsingException(parseContext.index(), "[geo_polygon] filter does not support token type [" + token.name() + "] under [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_polygon] filter does not support token type [" + token.name() + + "] under [" + currentFieldName + "]"); } } } else if (token.isValue()) { @@ -113,25 +115,25 @@ public class GeoPolygonFilterParser implements FilterParser { normalizeLat = parser.booleanValue(); normalizeLon = parser.booleanValue(); } else { - throw new QueryParsingException(parseContext.index(), "[geo_polygon] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_polygon] filter does not support [" + currentFieldName + "]"); } } else { - throw new QueryParsingException(parseContext.index(), "[geo_polygon] unexpected token type [" + token.name() + "]"); + throw new QueryParsingException(parseContext, "[geo_polygon] unexpected token type [" + token.name() + "]"); } } if (shell.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "no points defined for geo_polygon filter"); + throw new QueryParsingException(parseContext, "no points defined for geo_polygon filter"); } else { if (shell.size() < 3) { - throw new QueryParsingException(parseContext.index(), "too few points defined for geo_polygon filter"); + throw new QueryParsingException(parseContext, "too few points defined for geo_polygon filter"); } GeoPoint start = shell.get(0); if (!start.equals(shell.get(shell.size() - 1))) { shell.add(start); } if (shell.size() < 4) { - throw new QueryParsingException(parseContext.index(), "too few points defined for geo_polygon filter"); + throw new QueryParsingException(parseContext, "too few points defined for geo_polygon filter"); } } @@ -143,11 +145,11 @@ public class GeoPolygonFilterParser implements FilterParser { MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "failed to find geo_point field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "failed to find geo_point field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.mapper(); if (!(mapper instanceof GeoPointFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "field [" + fieldName + "] is not a geo_point field"); + throw new QueryParsingException(parseContext, "field [" + fieldName + "] is not a geo_point field"); } IndexGeoPointFieldData indexFieldData = parseContext.getForField(mapper); diff --git a/src/main/java/org/elasticsearch/index/query/GeoShapeFilterParser.java b/src/main/java/org/elasticsearch/index/query/GeoShapeFilterParser.java index 72eba62854e..5a5e45736cd 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoShapeFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoShapeFilterParser.java @@ -113,7 +113,7 @@ public class GeoShapeFilterParser implements FilterParser { } else if ("relation".equals(currentFieldName)) { shapeRelation = ShapeRelation.getRelationByName(parser.text()); if (shapeRelation == null) { - throw new QueryParsingException(parseContext.index(), "Unknown shape operation [" + parser.text() + "]"); + throw new QueryParsingException(parseContext, "Unknown shape operation [" + parser.text() + "]"); } } else if ("strategy".equals(currentFieldName)) { strategyName = parser.text(); @@ -134,13 +134,13 @@ public class GeoShapeFilterParser implements FilterParser { } } if (id == null) { - throw new QueryParsingException(parseContext.index(), "ID for indexed shape not provided"); + throw new QueryParsingException(parseContext, "ID for indexed shape not provided"); } else if (type == null) { - throw new QueryParsingException(parseContext.index(), "Type for indexed shape not provided"); + throw new QueryParsingException(parseContext, "Type for indexed shape not provided"); } shape = fetchService.fetch(id, type, index, shapePath); } else { - throw new QueryParsingException(parseContext.index(), "[geo_shape] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_shape] filter does not support [" + currentFieldName + "]"); } } } @@ -152,26 +152,26 @@ public class GeoShapeFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[geo_shape] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_shape] filter does not support [" + currentFieldName + "]"); } } } if (shape == null) { - throw new QueryParsingException(parseContext.index(), "No Shape defined"); + throw new QueryParsingException(parseContext, "No Shape defined"); } else if (shapeRelation == null) { - throw new QueryParsingException(parseContext.index(), "No Shape Relation defined"); + throw new QueryParsingException(parseContext, "No Shape Relation defined"); } MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName); if (smartNameFieldMappers == null || !smartNameFieldMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "Failed to find geo_shape field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "Failed to find geo_shape field [" + fieldName + "]"); } FieldMapper fieldMapper = smartNameFieldMappers.mapper(); // TODO: This isn't the nicest way to check this if (!(fieldMapper instanceof GeoShapeFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "Field [" + fieldName + "] is not a geo_shape"); + throw new QueryParsingException(parseContext, "Field [" + fieldName + "] is not a geo_shape"); } GeoShapeFieldMapper shapeFieldMapper = (GeoShapeFieldMapper) fieldMapper; diff --git a/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java b/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java index eeed9d270f0..ac3d4f59f92 100644 --- a/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/GeoShapeQueryParser.java @@ -93,7 +93,7 @@ public class GeoShapeQueryParser implements QueryParser { } else if ("relation".equals(currentFieldName)) { shapeRelation = ShapeRelation.getRelationByName(parser.text()); if (shapeRelation == null) { - throw new QueryParsingException(parseContext.index(), "Unknown shape operation [" + parser.text() + " ]"); + throw new QueryParsingException(parseContext, "Unknown shape operation [" + parser.text() + " ]"); } } else if ("indexed_shape".equals(currentFieldName) || "indexedShape".equals(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { @@ -112,13 +112,13 @@ public class GeoShapeQueryParser implements QueryParser { } } if (id == null) { - throw new QueryParsingException(parseContext.index(), "ID for indexed shape not provided"); + throw new QueryParsingException(parseContext, "ID for indexed shape not provided"); } else if (type == null) { - throw new QueryParsingException(parseContext.index(), "Type for indexed shape not provided"); + throw new QueryParsingException(parseContext, "Type for indexed shape not provided"); } shape = fetchService.fetch(id, type, index, shapePath); } else { - throw new QueryParsingException(parseContext.index(), "[geo_shape] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_shape] query does not support [" + currentFieldName + "]"); } } } @@ -128,26 +128,26 @@ public class GeoShapeQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[geo_shape] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[geo_shape] query does not support [" + currentFieldName + "]"); } } } if (shape == null) { - throw new QueryParsingException(parseContext.index(), "No Shape defined"); + throw new QueryParsingException(parseContext, "No Shape defined"); } else if (shapeRelation == null) { - throw new QueryParsingException(parseContext.index(), "No Shape Relation defined"); + throw new QueryParsingException(parseContext, "No Shape Relation defined"); } MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName); if (smartNameFieldMappers == null || !smartNameFieldMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "Failed to find geo_shape field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "Failed to find geo_shape field [" + fieldName + "]"); } FieldMapper fieldMapper = smartNameFieldMappers.mapper(); // TODO: This isn't the nicest way to check this if (!(fieldMapper instanceof GeoShapeFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "Field [" + fieldName + "] is not a geo_shape"); + throw new QueryParsingException(parseContext, "Field [" + fieldName + "] is not a geo_shape"); } GeoShapeFieldMapper shapeFieldMapper = (GeoShapeFieldMapper) fieldMapper; diff --git a/src/main/java/org/elasticsearch/index/query/GeohashCellFilter.java b/src/main/java/org/elasticsearch/index/query/GeohashCellFilter.java index f93cc2681b8..63ca22db644 100644 --- a/src/main/java/org/elasticsearch/index/query/GeohashCellFilter.java +++ b/src/main/java/org/elasticsearch/index/query/GeohashCellFilter.java @@ -265,22 +265,23 @@ public class GeohashCellFilter { } if (geohash == null) { - throw new QueryParsingException(parseContext.index(), "no geohash value provided to geohash_cell filter"); + throw new QueryParsingException(parseContext, "no geohash value provided to geohash_cell filter"); } MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "failed to find geo_point field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "failed to find geo_point field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.mapper(); if (!(mapper instanceof GeoPointFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "field [" + fieldName + "] is not a geo_point field"); + throw new QueryParsingException(parseContext, "field [" + fieldName + "] is not a geo_point field"); } GeoPointFieldMapper geoMapper = ((GeoPointFieldMapper) mapper); if (!geoMapper.isEnableGeohashPrefix()) { - throw new QueryParsingException(parseContext.index(), "can't execute geohash_cell on field [" + fieldName + "], geohash_prefix is not enabled"); + throw new QueryParsingException(parseContext, "can't execute geohash_cell on field [" + fieldName + + "], geohash_prefix is not enabled"); } if(levels > 0) { diff --git a/src/main/java/org/elasticsearch/index/query/HasChildFilterParser.java b/src/main/java/org/elasticsearch/index/query/HasChildFilterParser.java index c04e48b8e1e..d22a05f6a11 100644 --- a/src/main/java/org/elasticsearch/index/query/HasChildFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasChildFilterParser.java @@ -94,7 +94,7 @@ public class HasChildFilterParser implements FilterParser { } else if ("inner_hits".equals(currentFieldName)) { innerHits = innerHitsQueryParserHelper.parse(parseContext); } else { - throw new QueryParsingException(parseContext.index(), "[has_child] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_child] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "child_type".equals(currentFieldName) || "childType".equals(currentFieldName)) { @@ -112,15 +112,15 @@ public class HasChildFilterParser implements FilterParser { } else if ("max_children".equals(currentFieldName) || "maxChildren".equals(currentFieldName)) { maxChildren = parser.intValue(true); } else { - throw new QueryParsingException(parseContext.index(), "[has_child] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_child] filter does not support [" + currentFieldName + "]"); } } } if (!queryFound && !filterFound) { - throw new QueryParsingException(parseContext.index(), "[has_child] filter requires 'query' or 'filter' field"); + throw new QueryParsingException(parseContext, "[has_child] filter requires 'query' or 'filter' field"); } if (childType == null) { - throw new QueryParsingException(parseContext.index(), "[has_child] filter requires 'type' field"); + throw new QueryParsingException(parseContext, "[has_child] filter requires 'type' field"); } Query query; @@ -136,7 +136,7 @@ public class HasChildFilterParser implements FilterParser { DocumentMapper childDocMapper = parseContext.mapperService().documentMapper(childType); if (childDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "No mapping for for type [" + childType + "]"); + throw new QueryParsingException(parseContext, "No mapping for for type [" + childType + "]"); } if (innerHits != null) { InnerHitsContext.ParentChildInnerHits parentChildInnerHits = new InnerHitsContext.ParentChildInnerHits(innerHits.v2(), query, null, childDocMapper); @@ -145,7 +145,7 @@ public class HasChildFilterParser implements FilterParser { } ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper(); if (!parentFieldMapper.active()) { - throw new QueryParsingException(parseContext.index(), "Type [" + childType + "] does not have parent mapping"); + throw new QueryParsingException(parseContext, "Type [" + childType + "] does not have parent mapping"); } String parentType = parentFieldMapper.type(); @@ -154,11 +154,12 @@ public class HasChildFilterParser implements FilterParser { DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType); if (parentDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "[has_child] Type [" + childType + "] points to a non existent parent type [" + parentType + "]"); + throw new QueryParsingException(parseContext, "[has_child] Type [" + childType + "] points to a non existent parent type [" + + parentType + "]"); } if (maxChildren > 0 && maxChildren < minChildren) { - throw new QueryParsingException(parseContext.index(), "[has_child] 'max_children' is less than 'min_children'"); + throw new QueryParsingException(parseContext, "[has_child] 'max_children' is less than 'min_children'"); } BitDocIdSetFilter nonNestedDocsFilter = null; diff --git a/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java b/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java index 058f47d5eb7..e088b58a51a 100644 --- a/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasChildQueryParser.java @@ -92,7 +92,7 @@ public class HasChildQueryParser implements QueryParser { } else if ("inner_hits".equals(currentFieldName)) { innerHits = innerHitsQueryParserHelper.parse(parseContext); } else { - throw new QueryParsingException(parseContext.index(), "[has_child] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_child] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "child_type".equals(currentFieldName) || "childType".equals(currentFieldName)) { @@ -112,15 +112,15 @@ public class HasChildQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[has_child] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_child] query does not support [" + currentFieldName + "]"); } } } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[has_child] requires 'query' field"); + throw new QueryParsingException(parseContext, "[has_child] requires 'query' field"); } if (childType == null) { - throw new QueryParsingException(parseContext.index(), "[has_child] requires 'type' field"); + throw new QueryParsingException(parseContext, "[has_child] requires 'type' field"); } Query innerQuery = iq.asQuery(childType); @@ -132,10 +132,10 @@ public class HasChildQueryParser implements QueryParser { DocumentMapper childDocMapper = parseContext.mapperService().documentMapper(childType); if (childDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "[has_child] No mapping for for type [" + childType + "]"); + throw new QueryParsingException(parseContext, "[has_child] No mapping for for type [" + childType + "]"); } if (!childDocMapper.parentFieldMapper().active()) { - throw new QueryParsingException(parseContext.index(), "[has_child] Type [" + childType + "] does not have parent mapping"); + throw new QueryParsingException(parseContext, "[has_child] Type [" + childType + "] does not have parent mapping"); } if (innerHits != null) { @@ -146,18 +146,18 @@ public class HasChildQueryParser implements QueryParser { ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper(); if (!parentFieldMapper.active()) { - throw new QueryParsingException(parseContext.index(), "[has_child] _parent field not configured"); + throw new QueryParsingException(parseContext, "[has_child] _parent field not configured"); } String parentType = parentFieldMapper.type(); DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType); if (parentDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "[has_child] Type [" + childType - + "] points to a non existent parent type [" + parentType + "]"); + throw new QueryParsingException(parseContext, "[has_child] Type [" + childType + "] points to a non existent parent type [" + + parentType + "]"); } if (maxChildren > 0 && maxChildren < minChildren) { - throw new QueryParsingException(parseContext.index(), "[has_child] 'max_children' is less than 'min_children'"); + throw new QueryParsingException(parseContext, "[has_child] 'max_children' is less than 'min_children'"); } BitDocIdSetFilter nonNestedDocsFilter = null; diff --git a/src/main/java/org/elasticsearch/index/query/HasParentFilterParser.java b/src/main/java/org/elasticsearch/index/query/HasParentFilterParser.java index fd3335202f3..388f24d4ab0 100644 --- a/src/main/java/org/elasticsearch/index/query/HasParentFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasParentFilterParser.java @@ -83,7 +83,7 @@ public class HasParentFilterParser implements FilterParser { } else if ("inner_hits".equals(currentFieldName)) { innerHits = innerHitsQueryParserHelper.parse(parseContext); } else { - throw new QueryParsingException(parseContext.index(), "[has_parent] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_parent] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "parent_type".equals(currentFieldName) || "parentType".equals(currentFieldName)) { @@ -95,15 +95,15 @@ public class HasParentFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { // noop to be backwards compatible } else { - throw new QueryParsingException(parseContext.index(), "[has_parent] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_parent] filter does not support [" + currentFieldName + "]"); } } } if (!queryFound && !filterFound) { - throw new QueryParsingException(parseContext.index(), "[has_parent] filter requires 'query' or 'filter' field"); + throw new QueryParsingException(parseContext, "[has_parent] filter requires 'query' or 'filter' field"); } if (parentType == null) { - throw new QueryParsingException(parseContext.index(), "[has_parent] filter requires 'parent_type' field"); + throw new QueryParsingException(parseContext, "[has_parent] filter requires 'parent_type' field"); } Query innerQuery; diff --git a/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java b/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java index 9525064647b..d7d57b6ddd6 100644 --- a/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/HasParentQueryParser.java @@ -23,7 +23,6 @@ import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Filter; import org.apache.lucene.search.FilteredQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryWrapperFilter; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; @@ -88,7 +87,7 @@ public class HasParentQueryParser implements QueryParser { } else if ("inner_hits".equals(currentFieldName)) { innerHits = innerHitsQueryParserHelper.parse(parseContext); } else { - throw new QueryParsingException(parseContext.index(), "[has_parent] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_parent] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "parent_type".equals(currentFieldName) || "parentType".equals(currentFieldName)) { @@ -112,15 +111,15 @@ public class HasParentQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[has_parent] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[has_parent] query does not support [" + currentFieldName + "]"); } } } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[has_parent] query requires 'query' field"); + throw new QueryParsingException(parseContext, "[has_parent] query requires 'query' field"); } if (parentType == null) { - throw new QueryParsingException(parseContext.index(), "[has_parent] query requires 'parent_type' field"); + throw new QueryParsingException(parseContext, "[has_parent] query requires 'parent_type' field"); } Query innerQuery = iq.asQuery(parentType); @@ -145,7 +144,8 @@ public class HasParentQueryParser implements QueryParser { static Query createParentQuery(Query innerQuery, String parentType, boolean score, QueryParseContext parseContext, Tuple innerHits) { DocumentMapper parentDocMapper = parseContext.mapperService().documentMapper(parentType); if (parentDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "[has_parent] query configured 'parent_type' [" + parentType + "] is not a valid type"); + throw new QueryParsingException(parseContext, "[has_parent] query configured 'parent_type' [" + parentType + + "] is not a valid type"); } if (innerHits != null) { @@ -169,7 +169,7 @@ public class HasParentQueryParser implements QueryParser { } } if (parentChildIndexFieldData == null) { - throw new QueryParsingException(parseContext.index(), "[has_parent] no _parent field configured"); + throw new QueryParsingException(parseContext, "[has_parent] no _parent field configured"); } Filter parentFilter = null; diff --git a/src/main/java/org/elasticsearch/index/query/IdsFilterParser.java b/src/main/java/org/elasticsearch/index/query/IdsFilterParser.java index d0402aabf95..138557cd79a 100644 --- a/src/main/java/org/elasticsearch/index/query/IdsFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/IdsFilterParser.java @@ -68,7 +68,7 @@ public class IdsFilterParser implements FilterParser { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { BytesRef value = parser.utf8BytesOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for term filter"); + throw new QueryParsingException(parseContext, "No value specified for term filter"); } ids.add(value); } @@ -77,12 +77,12 @@ public class IdsFilterParser implements FilterParser { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { String value = parser.textOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No type specified for term filter"); + throw new QueryParsingException(parseContext, "No type specified for term filter"); } types.add(value); } } else { - throw new QueryParsingException(parseContext.index(), "[ids] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[ids] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "_type".equals(currentFieldName)) { @@ -90,13 +90,13 @@ public class IdsFilterParser implements FilterParser { } else if ("_name".equals(currentFieldName)) { filterName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[ids] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[ids] filter does not support [" + currentFieldName + "]"); } } } if (!idsProvided) { - throw new QueryParsingException(parseContext.index(), "[ids] filter requires providing a values element"); + throw new QueryParsingException(parseContext, "[ids] filter requires providing a values element"); } if (ids.isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/IdsQueryParser.java b/src/main/java/org/elasticsearch/index/query/IdsQueryParser.java index d0345944c66..3789b3039c0 100644 --- a/src/main/java/org/elasticsearch/index/query/IdsQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/IdsQueryParser.java @@ -74,12 +74,12 @@ public class IdsQueryParser implements QueryParser { (token == XContentParser.Token.VALUE_NUMBER)) { BytesRef value = parser.utf8BytesOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for term filter"); + throw new QueryParsingException(parseContext, "No value specified for term filter"); } ids.add(value); } else { - throw new QueryParsingException(parseContext.index(), - "Illegal value for id, expecting a string or number, got: " + token); + throw new QueryParsingException(parseContext, "Illegal value for id, expecting a string or number, got: " + + token); } } } else if ("types".equals(currentFieldName) || "type".equals(currentFieldName)) { @@ -87,12 +87,12 @@ public class IdsQueryParser implements QueryParser { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { String value = parser.textOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No type specified for term filter"); + throw new QueryParsingException(parseContext, "No type specified for term filter"); } types.add(value); } } else { - throw new QueryParsingException(parseContext.index(), "[ids] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[ids] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName) || "_type".equals(currentFieldName)) { @@ -102,13 +102,13 @@ public class IdsQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[ids] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[ids] query does not support [" + currentFieldName + "]"); } } } if (!idsProvided) { - throw new QueryParsingException(parseContext.index(), "[ids] query, no ids values provided"); + throw new QueryParsingException(parseContext, "[ids] query, no ids values provided"); } if (ids.isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/IndexQueryParserService.java b/src/main/java/org/elasticsearch/index/query/IndexQueryParserService.java index 93c94ee98da..e2bcd353e11 100644 --- a/src/main/java/org/elasticsearch/index/query/IndexQueryParserService.java +++ b/src/main/java/org/elasticsearch/index/query/IndexQueryParserService.java @@ -210,7 +210,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { } catch (QueryParsingException e) { throw e; } catch (Exception e) { - throw new QueryParsingException(index, "Failed to parse", e); + throw new QueryParsingException(getParseContext(), "Failed to parse", e); } finally { if (parser != null) { parser.close(); @@ -230,7 +230,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { } catch (QueryParsingException e) { throw e; } catch (Exception e) { - throw new QueryParsingException(index, "Failed to parse", e); + throw new QueryParsingException(getParseContext(), "Failed to parse", e); } finally { if (parser != null) { parser.close(); @@ -250,7 +250,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { } catch (QueryParsingException e) { throw e; } catch (Exception e) { - throw new QueryParsingException(index, "Failed to parse", e); + throw new QueryParsingException(context, "Failed to parse", e); } finally { if (parser != null) { parser.close(); @@ -266,7 +266,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { } catch (QueryParsingException e) { throw e; } catch (Exception e) { - throw new QueryParsingException(index, "Failed to parse [" + source + "]", e); + throw new QueryParsingException(getParseContext(), "Failed to parse [" + source + "]", e); } finally { if (parser != null) { parser.close(); @@ -282,7 +282,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { try { return innerParse(context, parser); } catch (IOException e) { - throw new QueryParsingException(index, "Failed to parse", e); + throw new QueryParsingException(context, "Failed to parse", e); } } @@ -359,7 +359,7 @@ public class IndexQueryParserService extends AbstractIndexComponent { XContentParser qSourceParser = XContentFactory.xContent(querySource).createParser(querySource); parsedQuery = parse(qSourceParser); } else { - throw new QueryParsingException(index(), "request does not support [" + fieldName + "]"); + throw new QueryParsingException(getParseContext(), "request does not support [" + fieldName + "]"); } } } @@ -369,10 +369,10 @@ public class IndexQueryParserService extends AbstractIndexComponent { } catch (QueryParsingException e) { throw e; } catch (Throwable e) { - throw new QueryParsingException(index, "Failed to parse", e); + throw new QueryParsingException(getParseContext(), "Failed to parse", e); } - throw new QueryParsingException(index(), "Required query is missing"); + throw new QueryParsingException(getParseContext(), "Required query is missing"); } private ParsedQuery innerParse(QueryParseContext parseContext, XContentParser parser) throws IOException, QueryParsingException { diff --git a/src/main/java/org/elasticsearch/index/query/IndicesFilterParser.java b/src/main/java/org/elasticsearch/index/query/IndicesFilterParser.java index c1f5b804f94..7bd39dad947 100644 --- a/src/main/java/org/elasticsearch/index/query/IndicesFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/IndicesFilterParser.java @@ -83,30 +83,30 @@ public class IndicesFilterParser implements FilterParser { noMatchFilter = parseContext.parseInnerFilter(); } } else { - throw new QueryParsingException(parseContext.index(), "[indices] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] filter does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if ("indices".equals(currentFieldName)) { if (indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] indices or index already specified"); + throw new QueryParsingException(parseContext, "[indices] indices or index already specified"); } indicesFound = true; Collection indices = new ArrayList<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { String value = parser.textOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "[indices] no value specified for 'indices' entry"); + throw new QueryParsingException(parseContext, "[indices] no value specified for 'indices' entry"); } indices.add(value); } currentIndexMatchesIndices = matchesIndices(parseContext.index().name(), indices.toArray(new String[indices.size()])); } else { - throw new QueryParsingException(parseContext.index(), "[indices] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("index".equals(currentFieldName)) { if (indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] indices or index already specified"); + throw new QueryParsingException(parseContext, "[indices] indices or index already specified"); } indicesFound = true; currentIndexMatchesIndices = matchesIndices(parseContext.index().name(), parser.text()); @@ -120,15 +120,15 @@ public class IndicesFilterParser implements FilterParser { } else if ("_name".equals(currentFieldName)) { filterName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[indices] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] filter does not support [" + currentFieldName + "]"); } } } if (!filterFound) { - throw new QueryParsingException(parseContext.index(), "[indices] requires 'filter' element"); + throw new QueryParsingException(parseContext, "[indices] requires 'filter' element"); } if (!indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] requires 'indices' or 'index' element"); + throw new QueryParsingException(parseContext, "[indices] requires 'indices' or 'index' element"); } Filter chosenFilter; diff --git a/src/main/java/org/elasticsearch/index/query/IndicesQueryParser.java b/src/main/java/org/elasticsearch/index/query/IndicesQueryParser.java index d5b5cefa149..a45fe9f88f6 100644 --- a/src/main/java/org/elasticsearch/index/query/IndicesQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/IndicesQueryParser.java @@ -76,30 +76,30 @@ public class IndicesQueryParser implements QueryParser { } else if ("no_match_query".equals(currentFieldName)) { innerNoMatchQuery = new XContentStructure.InnerQuery(parseContext, null); } else { - throw new QueryParsingException(parseContext.index(), "[indices] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if ("indices".equals(currentFieldName)) { if (indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] indices or index already specified"); + throw new QueryParsingException(parseContext, "[indices] indices or index already specified"); } indicesFound = true; Collection indices = new ArrayList<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { String value = parser.textOrNull(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "[indices] no value specified for 'indices' entry"); + throw new QueryParsingException(parseContext, "[indices] no value specified for 'indices' entry"); } indices.add(value); } currentIndexMatchesIndices = matchesIndices(parseContext.index().name(), indices.toArray(new String[indices.size()])); } else { - throw new QueryParsingException(parseContext.index(), "[indices] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("index".equals(currentFieldName)) { if (indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] indices or index already specified"); + throw new QueryParsingException(parseContext, "[indices] indices or index already specified"); } indicesFound = true; currentIndexMatchesIndices = matchesIndices(parseContext.index().name(), parser.text()); @@ -113,15 +113,15 @@ public class IndicesQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[indices] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[indices] query does not support [" + currentFieldName + "]"); } } } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[indices] requires 'query' element"); + throw new QueryParsingException(parseContext, "[indices] requires 'query' element"); } if (!indicesFound) { - throw new QueryParsingException(parseContext.index(), "[indices] requires 'indices' or 'index' element"); + throw new QueryParsingException(parseContext, "[indices] requires 'indices' or 'index' element"); } Query chosenQuery; diff --git a/src/main/java/org/elasticsearch/index/query/LimitFilterParser.java b/src/main/java/org/elasticsearch/index/query/LimitFilterParser.java index 858b23c6693..f4f8fde7427 100644 --- a/src/main/java/org/elasticsearch/index/query/LimitFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/LimitFilterParser.java @@ -53,13 +53,13 @@ public class LimitFilterParser implements FilterParser { if ("value".equals(currentFieldName)) { limit = parser.intValue(); } else { - throw new QueryParsingException(parseContext.index(), "[limit] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[limit] filter does not support [" + currentFieldName + "]"); } } } if (limit == -1) { - throw new QueryParsingException(parseContext.index(), "No value specified for limit filter"); + throw new QueryParsingException(parseContext, "No value specified for limit filter"); } // this filter is deprecated and parses to a filter that matches everything diff --git a/src/main/java/org/elasticsearch/index/query/MatchAllQueryParser.java b/src/main/java/org/elasticsearch/index/query/MatchAllQueryParser.java index 2017b940921..933d3d35631 100644 --- a/src/main/java/org/elasticsearch/index/query/MatchAllQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/MatchAllQueryParser.java @@ -59,7 +59,7 @@ public class MatchAllQueryParser implements QueryParser { if ("boost".equals(currentFieldName)) { boost = parser.floatValue(); } else { - throw new QueryParsingException(parseContext.index(), "[match_all] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[match_all] query does not support [" + currentFieldName + "]"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/MatchQueryParser.java b/src/main/java/org/elasticsearch/index/query/MatchQueryParser.java index a0f595a6626..8dd35c84b4d 100644 --- a/src/main/java/org/elasticsearch/index/query/MatchQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/MatchQueryParser.java @@ -65,7 +65,7 @@ public class MatchQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[match] query malformed, no field"); + throw new QueryParsingException(parseContext, "[match] query malformed, no field"); } String fieldName = parser.currentName(); @@ -93,12 +93,12 @@ public class MatchQueryParser implements QueryParser { } else if ("phrase_prefix".equals(tStr) || "phrasePrefix".equals(currentFieldName)) { type = MatchQuery.Type.PHRASE_PREFIX; } else { - throw new QueryParsingException(parseContext.index(), "[match] query does not support type " + tStr); + throw new QueryParsingException(parseContext, "[match] query does not support type " + tStr); } } else if ("analyzer".equals(currentFieldName)) { String analyzer = parser.text(); if (parseContext.analysisService().analyzer(analyzer) == null) { - throw new QueryParsingException(parseContext.index(), "[match] analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[match] analyzer [" + parser.text() + "] not found"); } matchQuery.setAnalyzer(analyzer); } else if ("boost".equals(currentFieldName)) { @@ -118,7 +118,8 @@ public class MatchQueryParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { matchQuery.setOccur(BooleanClause.Occur.MUST); } else { - throw new QueryParsingException(parseContext.index(), "text query requires operator to be either 'and' or 'or', not [" + op + "]"); + throw new QueryParsingException(parseContext, "text query requires operator to be either 'and' or 'or', not [" + + op + "]"); } } else if ("minimum_should_match".equals(currentFieldName) || "minimumShouldMatch".equals(currentFieldName)) { minimumShouldMatch = parser.textOrNull(); @@ -139,12 +140,12 @@ public class MatchQueryParser implements QueryParser { } else if ("all".equalsIgnoreCase(zeroTermsDocs)) { matchQuery.setZeroTermsQuery(MatchQuery.ZeroTermsQuery.ALL); } else { - throw new QueryParsingException(parseContext.index(), "Unsupported zero_terms_docs value [" + zeroTermsDocs + "]"); + throw new QueryParsingException(parseContext, "Unsupported zero_terms_docs value [" + zeroTermsDocs + "]"); } } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[match] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[match] query does not support [" + currentFieldName + "]"); } } } @@ -154,12 +155,13 @@ public class MatchQueryParser implements QueryParser { // move to the next token token = parser.nextToken(); if (token != XContentParser.Token.END_OBJECT) { - throw new QueryParsingException(parseContext.index(), "[match] query parsed in simplified form, with direct field name, but included more options than just the field name, possibly use its 'options' form, with 'query' element?"); + throw new QueryParsingException(parseContext, + "[match] query parsed in simplified form, with direct field name, but included more options than just the field name, possibly use its 'options' form, with 'query' element?"); } } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No text specified for text query"); + throw new QueryParsingException(parseContext, "No text specified for text query"); } Query query = matchQuery.parse(type, fieldName, value); diff --git a/src/main/java/org/elasticsearch/index/query/MissingFilterParser.java b/src/main/java/org/elasticsearch/index/query/MissingFilterParser.java index 10f0405b832..3f394ff735e 100644 --- a/src/main/java/org/elasticsearch/index/query/MissingFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/MissingFilterParser.java @@ -23,7 +23,6 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Filter; import org.apache.lucene.search.Query; -import org.apache.lucene.search.QueryWrapperFilter; import org.apache.lucene.search.TermRangeQuery; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.HashedBytesRef; @@ -78,13 +77,13 @@ public class MissingFilterParser implements FilterParser { } else if ("_name".equals(currentFieldName)) { filterName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[missing] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[missing] filter does not support [" + currentFieldName + "]"); } } } if (fieldPattern == null) { - throw new QueryParsingException(parseContext.index(), "missing must be provided with a [field]"); + throw new QueryParsingException(parseContext, "missing must be provided with a [field]"); } return newFilter(parseContext, fieldPattern, existence, nullValue, filterName); @@ -92,7 +91,7 @@ public class MissingFilterParser implements FilterParser { public static Filter newFilter(QueryParseContext parseContext, String fieldPattern, boolean existence, boolean nullValue, String filterName) { if (!existence && !nullValue) { - throw new QueryParsingException(parseContext.index(), "missing must have either existence, or null_value, or both set to true"); + throw new QueryParsingException(parseContext, "missing must have either existence, or null_value, or both set to true"); } final FieldMappers fieldNamesMappers = parseContext.mapperService().fullName(FieldNamesFieldMapper.NAME); diff --git a/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java b/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java index 0050b7199a1..b726d4f0159 100644 --- a/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryParser.java @@ -155,7 +155,7 @@ public class MoreLikeThisQueryParser implements QueryParser { } else if (Fields.INCLUDE.match(currentFieldName, parseContext.parseFlags())) { include = parser.booleanValue(); } else { - throw new QueryParsingException(parseContext.index(), "[mlt] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[mlt] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_ARRAY) { if (Fields.STOP_WORDS.match(currentFieldName, parseContext.parseFlags())) { @@ -192,7 +192,7 @@ public class MoreLikeThisQueryParser implements QueryParser { parseLikeField(parser, ignoreTexts, ignoreItems); } } else { - throw new QueryParsingException(parseContext.index(), "[mlt] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[mlt] query does not support [" + currentFieldName + "]"); } } else if (token == XContentParser.Token.START_OBJECT) { if (Fields.LIKE.match(currentFieldName, parseContext.parseFlags())) { @@ -201,16 +201,16 @@ public class MoreLikeThisQueryParser implements QueryParser { else if (Fields.IGNORE_LIKE.match(currentFieldName, parseContext.parseFlags())) { parseLikeField(parser, ignoreTexts, ignoreItems); } else { - throw new QueryParsingException(parseContext.index(), "[mlt] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[mlt] query does not support [" + currentFieldName + "]"); } } } if (likeTexts.isEmpty() && likeItems.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "more_like_this requires 'like' to be specified"); + throw new QueryParsingException(parseContext, "more_like_this requires 'like' to be specified"); } if (moreLikeFields != null && moreLikeFields.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "more_like_this requires 'fields' to be non-empty"); + throw new QueryParsingException(parseContext, "more_like_this requires 'fields' to be non-empty"); } // set analyzer @@ -258,8 +258,9 @@ public class MoreLikeThisQueryParser implements QueryParser { } if (item.type() == null) { if (parseContext.queryTypes().size() > 1) { - throw new QueryParsingException(parseContext.index(), - "ambiguous type for item with id: " + item.id() + " and index: " + item.index()); + throw new QueryParsingException(parseContext, + "ambiguous type for item with id: " + item.id() + + " and index: " + item.index()); } else { item.type(parseContext.queryTypes().iterator().next()); } diff --git a/src/main/java/org/elasticsearch/index/query/MultiMatchQueryParser.java b/src/main/java/org/elasticsearch/index/query/MultiMatchQueryParser.java index 3fbd43651de..976dd15dc7b 100644 --- a/src/main/java/org/elasticsearch/index/query/MultiMatchQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/MultiMatchQueryParser.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.query; import com.google.common.collect.Maps; + import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Query; import org.elasticsearch.common.inject.Inject; @@ -77,8 +78,7 @@ public class MultiMatchQueryParser implements QueryParser { } else if (token.isValue()) { extractFieldAndBoost(parseContext, parser, fieldNameWithBoosts); } else { - throw new QueryParsingException(parseContext.index(), "[" + NAME + "] query does not support [" + currentFieldName - + "]"); + throw new QueryParsingException(parseContext, "[" + NAME + "] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("query".equals(currentFieldName)) { @@ -88,7 +88,7 @@ public class MultiMatchQueryParser implements QueryParser { } else if ("analyzer".equals(currentFieldName)) { String analyzer = parser.text(); if (parseContext.analysisService().analyzer(analyzer) == null) { - throw new QueryParsingException(parseContext.index(), "["+ NAME +"] analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[" + NAME + "] analyzer [" + parser.text() + "] not found"); } multiMatchQuery.setAnalyzer(analyzer); } else if ("boost".equals(currentFieldName)) { @@ -108,7 +108,8 @@ public class MultiMatchQueryParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { multiMatchQuery.setOccur(BooleanClause.Occur.MUST); } else { - throw new QueryParsingException(parseContext.index(), "text query requires operator to be either 'and' or 'or', not [" + op + "]"); + throw new QueryParsingException(parseContext, "text query requires operator to be either 'and' or 'or', not [" + op + + "]"); } } else if ("minimum_should_match".equals(currentFieldName) || "minimumShouldMatch".equals(currentFieldName)) { minimumShouldMatch = parser.textOrNull(); @@ -131,22 +132,22 @@ public class MultiMatchQueryParser implements QueryParser { } else if ("all".equalsIgnoreCase(zeroTermsDocs)) { multiMatchQuery.setZeroTermsQuery(MatchQuery.ZeroTermsQuery.ALL); } else { - throw new QueryParsingException(parseContext.index(), "Unsupported zero_terms_docs value [" + zeroTermsDocs + "]"); + throw new QueryParsingException(parseContext, "Unsupported zero_terms_docs value [" + zeroTermsDocs + "]"); } } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[match] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[match] query does not support [" + currentFieldName + "]"); } } } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No text specified for multi_match query"); + throw new QueryParsingException(parseContext, "No text specified for multi_match query"); } if (fieldNameWithBoosts.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "No fields specified for multi_match query"); + throw new QueryParsingException(parseContext, "No fields specified for multi_match query"); } if (type == null) { type = MultiMatchQueryBuilder.Type.BEST_FIELDS; diff --git a/src/main/java/org/elasticsearch/index/query/NestedFilterParser.java b/src/main/java/org/elasticsearch/index/query/NestedFilterParser.java index f6cad0a57e0..fc2237d6630 100644 --- a/src/main/java/org/elasticsearch/index/query/NestedFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/NestedFilterParser.java @@ -70,7 +70,7 @@ public class NestedFilterParser implements FilterParser { } else if ("inner_hits".equals(currentFieldName)) { builder.setInnerHits(innerHitsQueryParserHelper.parse(parseContext)); } else { - throw new QueryParsingException(parseContext.index(), "[nested] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[nested] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("path".equals(currentFieldName)) { @@ -84,7 +84,7 @@ public class NestedFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[nested] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[nested] filter does not support [" + currentFieldName + "]"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java b/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java index 989388b79d4..ba9bcf07d46 100644 --- a/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/NestedQueryParser.java @@ -75,7 +75,7 @@ public class NestedQueryParser implements QueryParser { } else if ("inner_hits".equals(currentFieldName)) { builder.setInnerHits(innerHitsQueryParserHelper.parse(parseContext)); } else { - throw new QueryParsingException(parseContext.index(), "[nested] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[nested] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("path".equals(currentFieldName)) { @@ -93,12 +93,12 @@ public class NestedQueryParser implements QueryParser { } else if ("none".equals(sScoreMode)) { scoreMode = ScoreMode.None; } else { - throw new QueryParsingException(parseContext.index(), "illegal score_mode for nested query [" + sScoreMode + "]"); + throw new QueryParsingException(parseContext, "illegal score_mode for nested query [" + sScoreMode + "]"); } } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[nested] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[nested] query does not support [" + currentFieldName + "]"); } } } @@ -144,7 +144,7 @@ public class NestedQueryParser implements QueryParser { innerQuery = null; } } else { - throw new QueryParsingException(parseContext.index(), "[nested] requires either 'query' or 'filter' field"); + throw new QueryParsingException(parseContext, "[nested] requires either 'query' or 'filter' field"); } if (innerHits != null) { diff --git a/src/main/java/org/elasticsearch/index/query/NotFilterParser.java b/src/main/java/org/elasticsearch/index/query/NotFilterParser.java index db8adccc5dd..38bff1997bb 100644 --- a/src/main/java/org/elasticsearch/index/query/NotFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/NotFilterParser.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.query; import org.apache.lucene.search.Filter; -import org.apache.lucene.search.QueryWrapperFilter; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.HashedBytesRef; import org.elasticsearch.common.lucene.search.Queries; @@ -80,13 +79,13 @@ public class NotFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[not] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[not] filter does not support [" + currentFieldName + "]"); } } } if (!filterFound) { - throw new QueryParsingException(parseContext.index(), "filter is required when using `not` filter"); + throw new QueryParsingException(parseContext, "filter is required when using `not` filter"); } if (filter == null) { diff --git a/src/main/java/org/elasticsearch/index/query/OrFilterParser.java b/src/main/java/org/elasticsearch/index/query/OrFilterParser.java index 9c3ad615105..22932ac8290 100644 --- a/src/main/java/org/elasticsearch/index/query/OrFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/OrFilterParser.java @@ -100,14 +100,14 @@ public class OrFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[or] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[or] filter does not support [" + currentFieldName + "]"); } } } } if (!filtersFound) { - throw new QueryParsingException(parseContext.index(), "[or] filter requires 'filters' to be set on it'"); + throw new QueryParsingException(parseContext, "[or] filter requires 'filters' to be set on it'"); } if (filters.isEmpty()) { diff --git a/src/main/java/org/elasticsearch/index/query/PrefixFilterParser.java b/src/main/java/org/elasticsearch/index/query/PrefixFilterParser.java index e6bc4e3437f..c6bf3fe0a95 100644 --- a/src/main/java/org/elasticsearch/index/query/PrefixFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/PrefixFilterParser.java @@ -78,7 +78,7 @@ public class PrefixFilterParser implements FilterParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for prefix filter"); + throw new QueryParsingException(parseContext, "No value specified for prefix filter"); } Filter filter = null; diff --git a/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java b/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java index 0cecb0aa651..dc59007c461 100644 --- a/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/PrefixQueryParser.java @@ -53,7 +53,7 @@ public class PrefixQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[prefix] query malformed, no field"); + throw new QueryParsingException(parseContext, "[prefix] query malformed, no field"); } String fieldName = parser.currentName(); String rewriteMethod = null; @@ -80,7 +80,7 @@ public class PrefixQueryParser implements QueryParser { queryName = parser.text(); } } else { - throw new QueryParsingException(parseContext.index(), "[prefix] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[prefix] query does not support [" + currentFieldName + "]"); } } parser.nextToken(); @@ -90,7 +90,7 @@ public class PrefixQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for prefix query"); + throw new QueryParsingException(parseContext, "No value specified for prefix query"); } MultiTermQuery.RewriteMethod method = QueryParsers.parseRewriteMethod(rewriteMethod, null); diff --git a/src/main/java/org/elasticsearch/index/query/QueryParseContext.java b/src/main/java/org/elasticsearch/index/query/QueryParseContext.java index 2f43985444c..39c0543759b 100644 --- a/src/main/java/org/elasticsearch/index/query/QueryParseContext.java +++ b/src/main/java/org/elasticsearch/index/query/QueryParseContext.java @@ -292,23 +292,23 @@ public class QueryParseContext { if (parser.currentToken() != XContentParser.Token.START_OBJECT) { token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new QueryParsingException(index, "[_na] query malformed, must start with start_object"); + throw new QueryParsingException(this, "[_na] query malformed, must start with start_object"); } } token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(index, "[_na] query malformed, no field after start_object"); + throw new QueryParsingException(this, "[_na] query malformed, no field after start_object"); } String queryName = parser.currentName(); // move to the next START_OBJECT token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT && token != XContentParser.Token.START_ARRAY) { - throw new QueryParsingException(index, "[_na] query malformed, no field after start_object"); + throw new QueryParsingException(this, "[_na] query malformed, no field after start_object"); } QueryParser queryParser = indexQueryParser.queryParser(queryName); if (queryParser == null) { - throw new QueryParsingException(index, "No query registered for [" + queryName + "]"); + throw new QueryParsingException(this, "No query registered for [" + queryName + "]"); } Query result = queryParser.parse(this); if (parser.currentToken() == XContentParser.Token.END_OBJECT || parser.currentToken() == XContentParser.Token.END_ARRAY) { @@ -335,7 +335,7 @@ public class QueryParseContext { if (parser.currentToken() != XContentParser.Token.START_OBJECT) { token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new QueryParsingException(index, "[_na] filter malformed, must start with start_object"); + throw new QueryParsingException(this, "[_na] filter malformed, must start with start_object"); } } token = parser.nextToken(); @@ -344,18 +344,18 @@ public class QueryParseContext { if (token == XContentParser.Token.END_OBJECT || token == XContentParser.Token.VALUE_NULL) { return null; } - throw new QueryParsingException(index, "[_na] filter malformed, no field after start_object"); + throw new QueryParsingException(this, "[_na] filter malformed, no field after start_object"); } String filterName = parser.currentName(); // move to the next START_OBJECT or START_ARRAY token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT && token != XContentParser.Token.START_ARRAY) { - throw new QueryParsingException(index, "[_na] filter malformed, no field after start_object"); + throw new QueryParsingException(this, "[_na] filter malformed, no field after start_object"); } FilterParser filterParser = indexQueryParser.filterParser(filterName); if (filterParser == null) { - throw new QueryParsingException(index, "No filter registered for [" + filterName + "]"); + throw new QueryParsingException(this, "No filter registered for [" + filterName + "]"); } Filter result = executeFilterParser(filterParser); if (parser.currentToken() == XContentParser.Token.END_OBJECT || parser.currentToken() == XContentParser.Token.END_ARRAY) { @@ -368,7 +368,7 @@ public class QueryParseContext { public Filter parseInnerFilter(String filterName) throws IOException, QueryParsingException { FilterParser filterParser = indexQueryParser.filterParser(filterName); if (filterParser == null) { - throw new QueryParsingException(index, "No filter registered for [" + filterName + "]"); + throw new QueryParsingException(this, "No filter registered for [" + filterName + "]"); } return executeFilterParser(filterParser); } @@ -432,7 +432,8 @@ public class QueryParseContext { } else { Version indexCreatedVersion = indexQueryParser.getIndexCreatedVersion(); if (fieldMapping == null && indexCreatedVersion.onOrAfter(Version.V_1_4_0_Beta1)) { - throw new QueryParsingException(index, "Strict field resolution and no field mapping can be found for the field with name [" + name + "]"); + throw new QueryParsingException(this, "Strict field resolution and no field mapping can be found for the field with name [" + + name + "]"); } else { return fieldMapping; } diff --git a/src/main/java/org/elasticsearch/index/query/QueryParsingException.java b/src/main/java/org/elasticsearch/index/query/QueryParsingException.java index 5bf1407a107..b9b0381e90e 100644 --- a/src/main/java/org/elasticsearch/index/query/QueryParsingException.java +++ b/src/main/java/org/elasticsearch/index/query/QueryParsingException.java @@ -19,21 +19,67 @@ package org.elasticsearch.index.query; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentLocation; +import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexException; import org.elasticsearch.rest.RestStatus; +import java.io.IOException; + /** * */ public class QueryParsingException extends IndexException { - public QueryParsingException(Index index, String msg) { - super(index, msg); + static final int UNKNOWN_POSITION = -1; + private int lineNumber = UNKNOWN_POSITION; + private int columnNumber = UNKNOWN_POSITION; + + public QueryParsingException(QueryParseContext parseContext, String msg) { + this(parseContext, msg, null); } - public QueryParsingException(Index index, String msg, Throwable cause) { + public QueryParsingException(QueryParseContext parseContext, String msg, Throwable cause) { + super(parseContext.index(), msg, cause); + + XContentParser parser = parseContext.parser(); + if (parser != null) { + XContentLocation location = parser.getTokenLocation(); + if (location != null) { + lineNumber = location.lineNumber; + columnNumber = location.columnNumber; + } + } + } + + /** + * This constructor is provided for use in unit tests where a + * {@link QueryParseContext} may not be available + */ + QueryParsingException(Index index, int line, int col, String msg, Throwable cause) { super(index, msg, cause); + this.lineNumber = line; + this.columnNumber = col; + } + + /** + * Line number of the location of the error + * + * @return the line number or -1 if unknown + */ + public int getLineNumber() { + return lineNumber; + } + + /** + * Column number of the location of the error + * + * @return the column number or -1 if unknown + */ + public int getColumnNumber() { + return columnNumber; } @Override @@ -41,4 +87,13 @@ public class QueryParsingException extends IndexException { return RestStatus.BAD_REQUEST; } + @Override + protected void innerToXContent(XContentBuilder builder, Params params) throws IOException { + if (lineNumber != UNKNOWN_POSITION) { + builder.field("line", lineNumber); + builder.field("col", columnNumber); + } + super.innerToXContent(builder, params); + } + } diff --git a/src/main/java/org/elasticsearch/index/query/QueryStringQueryParser.java b/src/main/java/org/elasticsearch/index/query/QueryStringQueryParser.java index d0b07941888..402080789f4 100644 --- a/src/main/java/org/elasticsearch/index/query/QueryStringQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/QueryStringQueryParser.java @@ -126,7 +126,8 @@ public class QueryStringQueryParser implements QueryParser { } } } else { - throw new QueryParsingException(parseContext.index(), "[query_string] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[query_string] query does not support [" + currentFieldName + + "]"); } } else if (token.isValue()) { if ("query".equals(currentFieldName)) { @@ -140,18 +141,19 @@ public class QueryStringQueryParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { qpSettings.defaultOperator(org.apache.lucene.queryparser.classic.QueryParser.Operator.AND); } else { - throw new QueryParsingException(parseContext.index(), "Query default operator [" + op + "] is not allowed"); + throw new QueryParsingException(parseContext, "Query default operator [" + op + "] is not allowed"); } } else if ("analyzer".equals(currentFieldName)) { NamedAnalyzer analyzer = parseContext.analysisService().analyzer(parser.text()); if (analyzer == null) { - throw new QueryParsingException(parseContext.index(), "[query_string] analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[query_string] analyzer [" + parser.text() + "] not found"); } qpSettings.forcedAnalyzer(analyzer); } else if ("quote_analyzer".equals(currentFieldName) || "quoteAnalyzer".equals(currentFieldName)) { NamedAnalyzer analyzer = parseContext.analysisService().analyzer(parser.text()); if (analyzer == null) { - throw new QueryParsingException(parseContext.index(), "[query_string] quote_analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[query_string] quote_analyzer [" + parser.text() + + "] not found"); } qpSettings.forcedQuoteAnalyzer(analyzer); } else if ("allow_leading_wildcard".equals(currentFieldName) || "allowLeadingWildcard".equals(currentFieldName)) { @@ -199,17 +201,19 @@ public class QueryStringQueryParser implements QueryParser { try { qpSettings.timeZone(DateTimeZone.forID(parser.text())); } catch (IllegalArgumentException e) { - throw new QueryParsingException(parseContext.index(), "[query_string] time_zone [" + parser.text() + "] is unknown"); + throw new QueryParsingException(parseContext, + "[query_string] time_zone [" + parser.text() + "] is unknown"); } } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[query_string] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[query_string] query does not support [" + currentFieldName + + "]"); } } } if (qpSettings.queryString() == null) { - throw new QueryParsingException(parseContext.index(), "query_string must be provided with a [query]"); + throw new QueryParsingException(parseContext, "query_string must be provided with a [query]"); } qpSettings.defaultAnalyzer(parseContext.mapperService().searchAnalyzer()); qpSettings.defaultQuoteAnalyzer(parseContext.mapperService().searchQuoteAnalyzer()); @@ -239,7 +243,7 @@ public class QueryStringQueryParser implements QueryParser { } return query; } catch (org.apache.lucene.queryparser.classic.ParseException e) { - throw new QueryParsingException(parseContext.index(), "Failed to parse query [" + qpSettings.queryString() + "]", e); + throw new QueryParsingException(parseContext, "Failed to parse query [" + qpSettings.queryString() + "]", e); } } } diff --git a/src/main/java/org/elasticsearch/index/query/RangeFilterParser.java b/src/main/java/org/elasticsearch/index/query/RangeFilterParser.java index 300ed66e6d8..8b5f557d0ba 100644 --- a/src/main/java/org/elasticsearch/index/query/RangeFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/RangeFilterParser.java @@ -105,7 +105,7 @@ public class RangeFilterParser implements FilterParser { } else if ("format".equals(currentFieldName)) { forcedDateParser = new DateMathParser(Joda.forPattern(parser.text()), DateFieldMapper.Defaults.TIME_UNIT); } else { - throw new QueryParsingException(parseContext.index(), "[range] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[range] filter does not support [" + currentFieldName + "]"); } } } @@ -119,13 +119,13 @@ public class RangeFilterParser implements FilterParser { } else if ("execution".equals(currentFieldName)) { execution = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[range] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[range] filter does not support [" + currentFieldName + "]"); } } } if (fieldName == null) { - throw new QueryParsingException(parseContext.index(), "[range] filter no field specified for range filter"); + throw new QueryParsingException(parseContext, "[range] filter no field specified for range filter"); } Filter filter = null; @@ -136,33 +136,39 @@ public class RangeFilterParser implements FilterParser { FieldMapper mapper = smartNameFieldMappers.mapper(); if (mapper instanceof DateFieldMapper) { if ((from instanceof Number || to instanceof Number) && timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + fieldName + "]"); + throw new QueryParsingException(parseContext, + "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + + fieldName + "]"); } filter = ((DateFieldMapper) mapper).rangeFilter(from, to, includeLower, includeUpper, timeZone, forcedDateParser, parseContext); } else { if (timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone can not be applied to non date field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field [" + + fieldName + "]"); } filter = mapper.rangeFilter(from, to, includeLower, includeUpper, parseContext); } } else if ("fielddata".equals(execution)) { FieldMapper mapper = smartNameFieldMappers.mapper(); if (!(mapper instanceof NumberFieldMapper)) { - throw new QueryParsingException(parseContext.index(), "[range] filter field [" + fieldName + "] is not a numeric type"); + throw new QueryParsingException(parseContext, "[range] filter field [" + fieldName + "] is not a numeric type"); } if (mapper instanceof DateFieldMapper) { if ((from instanceof Number || to instanceof Number) && timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + fieldName + "]"); + throw new QueryParsingException(parseContext, + "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + + fieldName + "]"); } filter = ((DateFieldMapper) mapper).rangeFilter(parseContext, from, to, includeLower, includeUpper, timeZone, forcedDateParser, parseContext); } else { if (timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone can not be applied to non date field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field [" + + fieldName + "]"); } filter = ((NumberFieldMapper) mapper).rangeFilter(parseContext, from, to, includeLower, includeUpper, parseContext); } } else { - throw new QueryParsingException(parseContext.index(), "[range] filter doesn't support [" + execution + "] execution"); + throw new QueryParsingException(parseContext, "[range] filter doesn't support [" + execution + "] execution"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java b/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java index cfc600832ec..354da1df704 100644 --- a/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/RangeQueryParser.java @@ -55,12 +55,12 @@ public class RangeQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[range] query malformed, no field to indicate field name"); + throw new QueryParsingException(parseContext, "[range] query malformed, no field to indicate field name"); } String fieldName = parser.currentName(); token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new QueryParsingException(parseContext.index(), "[range] query malformed, after field missing start object"); + throw new QueryParsingException(parseContext, "[range] query malformed, after field missing start object"); } Object from = null; @@ -106,7 +106,7 @@ public class RangeQueryParser implements QueryParser { } else if ("format".equals(currentFieldName)) { forcedDateParser = new DateMathParser(Joda.forPattern(parser.text()), DateFieldMapper.Defaults.TIME_UNIT); } else { - throw new QueryParsingException(parseContext.index(), "[range] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[range] query does not support [" + currentFieldName + "]"); } } } @@ -114,7 +114,7 @@ public class RangeQueryParser implements QueryParser { // move to the next end object, to close the field name token = parser.nextToken(); if (token != XContentParser.Token.END_OBJECT) { - throw new QueryParsingException(parseContext.index(), "[range] query malformed, does not end with an object"); + throw new QueryParsingException(parseContext, "[range] query malformed, does not end with an object"); } Query query = null; @@ -124,12 +124,15 @@ public class RangeQueryParser implements QueryParser { FieldMapper mapper = smartNameFieldMappers.mapper(); if (mapper instanceof DateFieldMapper) { if ((from instanceof Number || to instanceof Number) && timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + fieldName + "]"); + throw new QueryParsingException(parseContext, + "[range] time_zone when using ms since epoch format as it's UTC based can not be applied to [" + fieldName + + "]"); } query = ((DateFieldMapper) mapper).rangeQuery(from, to, includeLower, includeUpper, timeZone, forcedDateParser, parseContext); } else { if (timeZone != null) { - throw new QueryParsingException(parseContext.index(), "[range] time_zone can not be applied to non date field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "[range] time_zone can not be applied to non date field [" + + fieldName + "]"); } //LUCENE 4 UPGRADE Mapper#rangeQuery should use bytesref as well? query = mapper.rangeQuery(from, to, includeLower, includeUpper, parseContext); diff --git a/src/main/java/org/elasticsearch/index/query/RegexpFilterParser.java b/src/main/java/org/elasticsearch/index/query/RegexpFilterParser.java index 76db069af17..5f1d9174fc7 100644 --- a/src/main/java/org/elasticsearch/index/query/RegexpFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/RegexpFilterParser.java @@ -84,7 +84,7 @@ public class RegexpFilterParser implements FilterParser { } else if ("flags_value".equals(currentFieldName)) { flagsValue = parser.intValue(); } else { - throw new QueryParsingException(parseContext.index(), "[regexp] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[regexp] filter does not support [" + currentFieldName + "]"); } } } @@ -108,7 +108,7 @@ public class RegexpFilterParser implements FilterParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for regexp filter"); + throw new QueryParsingException(parseContext, "No value specified for regexp filter"); } Filter filter = null; diff --git a/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java b/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java index 41d53316a57..a1ec2996332 100644 --- a/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/RegexpQueryParser.java @@ -55,7 +55,7 @@ public class RegexpQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[regexp] query malformed, no field"); + throw new QueryParsingException(parseContext, "[regexp] query malformed, no field"); } String fieldName = parser.currentName(); String rewriteMethod = null; @@ -92,7 +92,7 @@ public class RegexpQueryParser implements QueryParser { queryName = parser.text(); } } else { - throw new QueryParsingException(parseContext.index(), "[regexp] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[regexp] query does not support [" + currentFieldName + "]"); } } parser.nextToken(); @@ -102,7 +102,7 @@ public class RegexpQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for regexp query"); + throw new QueryParsingException(parseContext, "No value specified for regexp query"); } MultiTermQuery.RewriteMethod method = QueryParsers.parseRewriteMethod(rewriteMethod, null); diff --git a/src/main/java/org/elasticsearch/index/query/ScriptFilterParser.java b/src/main/java/org/elasticsearch/index/query/ScriptFilterParser.java index a619b3d63ba..54dbe6cc1db 100644 --- a/src/main/java/org/elasticsearch/index/query/ScriptFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/ScriptFilterParser.java @@ -85,7 +85,7 @@ public class ScriptFilterParser implements FilterParser { if ("params".equals(currentFieldName)) { params = parser.map(); } else { - throw new QueryParsingException(parseContext.index(), "[script] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[script] filter does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("_name".equals(currentFieldName)) { @@ -95,7 +95,7 @@ public class ScriptFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else if (!scriptParameterParser.token(currentFieldName, token, parser)){ - throw new QueryParsingException(parseContext.index(), "[script] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[script] filter does not support [" + currentFieldName + "]"); } } } @@ -108,7 +108,7 @@ public class ScriptFilterParser implements FilterParser { scriptLang = scriptParameterParser.lang(); if (script == null) { - throw new QueryParsingException(parseContext.index(), "script must be provided with a [script] filter"); + throw new QueryParsingException(parseContext, "script must be provided with a [script] filter"); } if (params == null) { params = newHashMap(); diff --git a/src/main/java/org/elasticsearch/index/query/SimpleQueryStringParser.java b/src/main/java/org/elasticsearch/index/query/SimpleQueryStringParser.java index 43e64ce0280..446dbc95b57 100644 --- a/src/main/java/org/elasticsearch/index/query/SimpleQueryStringParser.java +++ b/src/main/java/org/elasticsearch/index/query/SimpleQueryStringParser.java @@ -139,8 +139,9 @@ public class SimpleQueryStringParser implements QueryParser { } } } else { - throw new QueryParsingException(parseContext.index(), - "[" + NAME + "] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, + "[" + NAME + "] query does not support [" + currentFieldName + + "]"); } } else if (token.isValue()) { if ("query".equals(currentFieldName)) { @@ -148,7 +149,7 @@ public class SimpleQueryStringParser implements QueryParser { } else if ("analyzer".equals(currentFieldName)) { analyzer = parseContext.analysisService().analyzer(parser.text()); if (analyzer == null) { - throw new QueryParsingException(parseContext.index(), "[" + NAME + "] analyzer [" + parser.text() + "] not found"); + throw new QueryParsingException(parseContext, "[" + NAME + "] analyzer [" + parser.text() + "] not found"); } } else if ("field".equals(currentFieldName)) { field = parser.text(); @@ -159,8 +160,7 @@ public class SimpleQueryStringParser implements QueryParser { } else if ("and".equalsIgnoreCase(op)) { defaultOperator = BooleanClause.Occur.MUST; } else { - throw new QueryParsingException(parseContext.index(), - "[" + NAME + "] default operator [" + op + "] is not allowed"); + throw new QueryParsingException(parseContext, "[" + NAME + "] default operator [" + op + "] is not allowed"); } } else if ("flags".equals(currentFieldName)) { if (parser.currentToken() != XContentParser.Token.VALUE_NUMBER) { @@ -188,14 +188,14 @@ public class SimpleQueryStringParser implements QueryParser { } else if ("minimum_should_match".equals(currentFieldName)) { minimumShouldMatch = parser.textOrNull(); } else { - throw new QueryParsingException(parseContext.index(), "[" + NAME + "] unsupported field [" + parser.currentName() + "]"); + throw new QueryParsingException(parseContext, "[" + NAME + "] unsupported field [" + parser.currentName() + "]"); } } } // Query text is required if (queryBody == null) { - throw new QueryParsingException(parseContext.index(), "[" + NAME + "] query text missing"); + throw new QueryParsingException(parseContext, "[" + NAME + "] query text missing"); } // Support specifying only a field instead of a map diff --git a/src/main/java/org/elasticsearch/index/query/SpanFirstQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanFirstQueryParser.java index ea8ff3d3923..5a302eb17d7 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanFirstQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanFirstQueryParser.java @@ -63,11 +63,11 @@ public class SpanFirstQueryParser implements QueryParser { if ("match".equals(currentFieldName)) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "spanFirst [match] must be of type span query"); + throw new QueryParsingException(parseContext, "spanFirst [match] must be of type span query"); } match = (SpanQuery) query; } else { - throw new QueryParsingException(parseContext.index(), "[span_first] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_first] query does not support [" + currentFieldName + "]"); } } else { if ("boost".equals(currentFieldName)) { @@ -77,15 +77,15 @@ public class SpanFirstQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[span_first] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_first] query does not support [" + currentFieldName + "]"); } } } if (match == null) { - throw new QueryParsingException(parseContext.index(), "spanFirst must have [match] span query clause"); + throw new QueryParsingException(parseContext, "spanFirst must have [match] span query clause"); } if (end == -1) { - throw new QueryParsingException(parseContext.index(), "spanFirst must have [end] set for it"); + throw new QueryParsingException(parseContext, "spanFirst must have [end] set for it"); } SpanFirstQuery query = new SpanFirstQuery(match, end); diff --git a/src/main/java/org/elasticsearch/index/query/SpanMultiTermQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanMultiTermQueryParser.java index 7c9b2a67277..a44580a5176 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanMultiTermQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanMultiTermQueryParser.java @@ -51,17 +51,17 @@ public class SpanMultiTermQueryParser implements QueryParser { Token token = parser.nextToken(); if (!MATCH_NAME.equals(parser.currentName()) || token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "spanMultiTerm must have [" + MATCH_NAME + "] multi term query clause"); + throw new QueryParsingException(parseContext, "spanMultiTerm must have [" + MATCH_NAME + "] multi term query clause"); } token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new QueryParsingException(parseContext.index(), "spanMultiTerm must have [" + MATCH_NAME + "] multi term query clause"); + throw new QueryParsingException(parseContext, "spanMultiTerm must have [" + MATCH_NAME + "] multi term query clause"); } Query subQuery = parseContext.parseInnerQuery(); if (!(subQuery instanceof MultiTermQuery)) { - throw new QueryParsingException(parseContext.index(), "spanMultiTerm [" + MATCH_NAME + "] must be of type multi term query"); + throw new QueryParsingException(parseContext, "spanMultiTerm [" + MATCH_NAME + "] must be of type multi term query"); } parser.nextToken(); diff --git a/src/main/java/org/elasticsearch/index/query/SpanNearQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanNearQueryParser.java index 84283fce1a4..6ecf1b70bea 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanNearQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanNearQueryParser.java @@ -69,12 +69,12 @@ public class SpanNearQueryParser implements QueryParser { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "spanNear [clauses] must be of type span query"); + throw new QueryParsingException(parseContext, "spanNear [clauses] must be of type span query"); } clauses.add((SpanQuery) query); } } else { - throw new QueryParsingException(parseContext.index(), "[span_near] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_near] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("in_order".equals(currentFieldName) || "inOrder".equals(currentFieldName)) { @@ -88,17 +88,17 @@ public class SpanNearQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[span_near] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_near] query does not support [" + currentFieldName + "]"); } } else { - throw new QueryParsingException(parseContext.index(), "[span_near] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_near] query does not support [" + currentFieldName + "]"); } } if (clauses.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "span_near must include [clauses]"); + throw new QueryParsingException(parseContext, "span_near must include [clauses]"); } if (slop == null) { - throw new QueryParsingException(parseContext.index(), "span_near must include [slop]"); + throw new QueryParsingException(parseContext, "span_near must include [slop]"); } SpanNearQuery query = new SpanNearQuery(clauses.toArray(new SpanQuery[clauses.size()]), slop.intValue(), inOrder, collectPayloads); diff --git a/src/main/java/org/elasticsearch/index/query/SpanNotQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanNotQueryParser.java index afadf4c68ef..bcb62e7a224 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanNotQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanNotQueryParser.java @@ -68,17 +68,17 @@ public class SpanNotQueryParser implements QueryParser { if ("include".equals(currentFieldName)) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "spanNot [include] must be of type span query"); + throw new QueryParsingException(parseContext, "spanNot [include] must be of type span query"); } include = (SpanQuery) query; } else if ("exclude".equals(currentFieldName)) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "spanNot [exclude] must be of type span query"); + throw new QueryParsingException(parseContext, "spanNot [exclude] must be of type span query"); } exclude = (SpanQuery) query; } else { - throw new QueryParsingException(parseContext.index(), "[span_not] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_not] query does not support [" + currentFieldName + "]"); } } else { if ("dist".equals(currentFieldName)) { @@ -92,18 +92,18 @@ public class SpanNotQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[span_not] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_not] query does not support [" + currentFieldName + "]"); } } } if (include == null) { - throw new QueryParsingException(parseContext.index(), "spanNot must have [include] span query clause"); + throw new QueryParsingException(parseContext, "spanNot must have [include] span query clause"); } if (exclude == null) { - throw new QueryParsingException(parseContext.index(), "spanNot must have [exclude] span query clause"); + throw new QueryParsingException(parseContext, "spanNot must have [exclude] span query clause"); } if (dist != null && (pre != null || post != null)) { - throw new QueryParsingException(parseContext.index(), "spanNot can either use [dist] or [pre] & [post] (or none)"); + throw new QueryParsingException(parseContext, "spanNot can either use [dist] or [pre] & [post] (or none)"); } // set appropriate defaults diff --git a/src/main/java/org/elasticsearch/index/query/SpanOrQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanOrQueryParser.java index a9d12f6d941..db58d4cca82 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanOrQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanOrQueryParser.java @@ -66,12 +66,12 @@ public class SpanOrQueryParser implements QueryParser { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { Query query = parseContext.parseInnerQuery(); if (!(query instanceof SpanQuery)) { - throw new QueryParsingException(parseContext.index(), "spanOr [clauses] must be of type span query"); + throw new QueryParsingException(parseContext, "spanOr [clauses] must be of type span query"); } clauses.add((SpanQuery) query); } } else { - throw new QueryParsingException(parseContext.index(), "[span_or] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_or] query does not support [" + currentFieldName + "]"); } } else { if ("boost".equals(currentFieldName)) { @@ -79,12 +79,12 @@ public class SpanOrQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[span_or] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_or] query does not support [" + currentFieldName + "]"); } } } if (clauses.isEmpty()) { - throw new QueryParsingException(parseContext.index(), "spanOr must include [clauses]"); + throw new QueryParsingException(parseContext, "spanOr must include [clauses]"); } SpanOrQuery query = new SpanOrQuery(clauses.toArray(new SpanQuery[clauses.size()])); diff --git a/src/main/java/org/elasticsearch/index/query/SpanTermQueryParser.java b/src/main/java/org/elasticsearch/index/query/SpanTermQueryParser.java index 0203bb26051..535b626306a 100644 --- a/src/main/java/org/elasticsearch/index/query/SpanTermQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/SpanTermQueryParser.java @@ -77,7 +77,7 @@ public class SpanTermQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[span_term] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[span_term] query does not support [" + currentFieldName + "]"); } } } @@ -89,7 +89,7 @@ public class SpanTermQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for term query"); + throw new QueryParsingException(parseContext, "No value specified for term query"); } BytesRef valueBytes = null; diff --git a/src/main/java/org/elasticsearch/index/query/TermFilterParser.java b/src/main/java/org/elasticsearch/index/query/TermFilterParser.java index f03a8a43cae..ca077b91ee3 100644 --- a/src/main/java/org/elasticsearch/index/query/TermFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermFilterParser.java @@ -81,7 +81,7 @@ public class TermFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[term] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[term] filter does not support [" + currentFieldName + "]"); } } } @@ -100,11 +100,11 @@ public class TermFilterParser implements FilterParser { } if (fieldName == null) { - throw new QueryParsingException(parseContext.index(), "No field specified for term filter"); + throw new QueryParsingException(parseContext, "No field specified for term filter"); } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for term filter"); + throw new QueryParsingException(parseContext, "No value specified for term filter"); } Filter filter = null; diff --git a/src/main/java/org/elasticsearch/index/query/TermQueryParser.java b/src/main/java/org/elasticsearch/index/query/TermQueryParser.java index 2c016973b6e..3a913fc21ad 100644 --- a/src/main/java/org/elasticsearch/index/query/TermQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermQueryParser.java @@ -51,7 +51,7 @@ public class TermQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[term] query malformed, no field"); + throw new QueryParsingException(parseContext, "[term] query malformed, no field"); } String fieldName = parser.currentName(); @@ -74,7 +74,7 @@ public class TermQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[term] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[term] query does not support [" + currentFieldName + "]"); } } } @@ -86,7 +86,7 @@ public class TermQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for term query"); + throw new QueryParsingException(parseContext, "No value specified for term query"); } Query query = null; diff --git a/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java b/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java index 3c5ecd15106..46c52b80f64 100644 --- a/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java @@ -90,14 +90,14 @@ public class TermsFilterParser implements FilterParser { currentFieldName = parser.currentName(); } else if (token == XContentParser.Token.START_ARRAY) { if (fieldName != null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter does not support multiple fields"); + throw new QueryParsingException(parseContext, "[terms] filter does not support multiple fields"); } fieldName = currentFieldName; while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { Object value = parser.objectBytes(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for terms filter"); + throw new QueryParsingException(parseContext, "No value specified for terms filter"); } terms.add(value); } @@ -118,18 +118,19 @@ public class TermsFilterParser implements FilterParser { } else if ("routing".equals(currentFieldName)) { lookupRouting = parser.textOrNull(); } else { - throw new QueryParsingException(parseContext.index(), "[terms] filter does not support [" + currentFieldName + "] within lookup element"); + throw new QueryParsingException(parseContext, "[terms] filter does not support [" + currentFieldName + + "] within lookup element"); } } } if (lookupType == null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the type"); + throw new QueryParsingException(parseContext, "[terms] filter lookup element requires specifying the type"); } if (lookupId == null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the id"); + throw new QueryParsingException(parseContext, "[terms] filter lookup element requires specifying the id"); } if (lookupPath == null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the path"); + throw new QueryParsingException(parseContext, "[terms] filter lookup element requires specifying the path"); } } else if (token.isValue()) { if (EXECUTION_KEY.equals(currentFieldName)) { @@ -141,13 +142,13 @@ public class TermsFilterParser implements FilterParser { } else if ("_cache_key".equals(currentFieldName) || "_cacheKey".equals(currentFieldName)) { cacheKey = new HashedBytesRef(parser.text()); } else { - throw new QueryParsingException(parseContext.index(), "[terms] filter does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[terms] filter does not support [" + currentFieldName + "]"); } } } if (fieldName == null) { - throw new QueryParsingException(parseContext.index(), "terms filter requires a field name, followed by array of terms"); + throw new QueryParsingException(parseContext, "terms filter requires a field name, followed by array of terms"); } FieldMapper fieldMapper = null; diff --git a/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java b/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java index 15c9f18388e..dcf078d19b1 100644 --- a/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermsQueryParser.java @@ -75,13 +75,13 @@ public class TermsQueryParser implements QueryParser { currentFieldName = parser.currentName(); } else if (token == XContentParser.Token.START_ARRAY) { if (fieldName != null) { - throw new QueryParsingException(parseContext.index(), "[terms] query does not support multiple fields"); + throw new QueryParsingException(parseContext, "[terms] query does not support multiple fields"); } fieldName = currentFieldName; while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { Object value = parser.objectBytes(); if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for terms query"); + throw new QueryParsingException(parseContext, "No value specified for terms query"); } values.add(value); } @@ -97,15 +97,15 @@ public class TermsQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[terms] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[terms] query does not support [" + currentFieldName + "]"); } } else { - throw new QueryParsingException(parseContext.index(), "[terms] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[terms] query does not support [" + currentFieldName + "]"); } } if (fieldName == null) { - throw new QueryParsingException(parseContext.index(), "No field specified for terms query"); + throw new QueryParsingException(parseContext, "No field specified for terms query"); } FieldMapper mapper = null; diff --git a/src/main/java/org/elasticsearch/index/query/TopChildrenQueryParser.java b/src/main/java/org/elasticsearch/index/query/TopChildrenQueryParser.java index a44239e863e..095a849b792 100644 --- a/src/main/java/org/elasticsearch/index/query/TopChildrenQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/TopChildrenQueryParser.java @@ -78,7 +78,7 @@ public class TopChildrenQueryParser implements QueryParser { iq = new XContentStructure.InnerQuery(parseContext, childType == null ? null : new String[] {childType}); queryFound = true; } else { - throw new QueryParsingException(parseContext.index(), "[top_children] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[top_children] query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if ("type".equals(currentFieldName)) { @@ -96,15 +96,15 @@ public class TopChildrenQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[top_children] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[top_children] query does not support [" + currentFieldName + "]"); } } } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[top_children] requires 'query' field"); + throw new QueryParsingException(parseContext, "[top_children] requires 'query' field"); } if (childType == null) { - throw new QueryParsingException(parseContext.index(), "[top_children] requires 'type' field"); + throw new QueryParsingException(parseContext, "[top_children] requires 'type' field"); } Query innerQuery = iq.asQuery(childType); @@ -115,11 +115,11 @@ public class TopChildrenQueryParser implements QueryParser { DocumentMapper childDocMapper = parseContext.mapperService().documentMapper(childType); if (childDocMapper == null) { - throw new QueryParsingException(parseContext.index(), "No mapping for for type [" + childType + "]"); + throw new QueryParsingException(parseContext, "No mapping for for type [" + childType + "]"); } ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper(); if (!parentFieldMapper.active()) { - throw new QueryParsingException(parseContext.index(), "Type [" + childType + "] does not have parent mapping"); + throw new QueryParsingException(parseContext, "Type [" + childType + "] does not have parent mapping"); } String parentType = childDocMapper.parentFieldMapper().type(); diff --git a/src/main/java/org/elasticsearch/index/query/TypeFilterParser.java b/src/main/java/org/elasticsearch/index/query/TypeFilterParser.java index e4ae0b957e0..a6248a4e228 100644 --- a/src/main/java/org/elasticsearch/index/query/TypeFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/TypeFilterParser.java @@ -50,15 +50,15 @@ public class TypeFilterParser implements FilterParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[type] filter should have a value field, and the type name"); + throw new QueryParsingException(parseContext, "[type] filter should have a value field, and the type name"); } String fieldName = parser.currentName(); if (!fieldName.equals("value")) { - throw new QueryParsingException(parseContext.index(), "[type] filter should have a value field, and the type name"); + throw new QueryParsingException(parseContext, "[type] filter should have a value field, and the type name"); } token = parser.nextToken(); if (token != XContentParser.Token.VALUE_STRING) { - throw new QueryParsingException(parseContext.index(), "[type] filter should have a value field, and the type name"); + throw new QueryParsingException(parseContext, "[type] filter should have a value field, and the type name"); } BytesRef type = parser.utf8Bytes(); // move to the next token diff --git a/src/main/java/org/elasticsearch/index/query/WildcardQueryParser.java b/src/main/java/org/elasticsearch/index/query/WildcardQueryParser.java index 6a641e96219..a661c185878 100644 --- a/src/main/java/org/elasticsearch/index/query/WildcardQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/WildcardQueryParser.java @@ -52,7 +52,7 @@ public class WildcardQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[wildcard] query malformed, no field"); + throw new QueryParsingException(parseContext, "[wildcard] query malformed, no field"); } String fieldName = parser.currentName(); String rewriteMethod = null; @@ -78,7 +78,7 @@ public class WildcardQueryParser implements QueryParser { } else if ("_name".equals(currentFieldName)) { queryName = parser.text(); } else { - throw new QueryParsingException(parseContext.index(), "[wildcard] query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, "[wildcard] query does not support [" + currentFieldName + "]"); } } } @@ -89,7 +89,7 @@ public class WildcardQueryParser implements QueryParser { } if (value == null) { - throw new QueryParsingException(parseContext.index(), "No value specified for prefix query"); + throw new QueryParsingException(parseContext, "No value specified for prefix query"); } BytesRef valueBytes; diff --git a/src/main/java/org/elasticsearch/index/query/WrapperFilterParser.java b/src/main/java/org/elasticsearch/index/query/WrapperFilterParser.java index 2346d65943d..35ca8724453 100644 --- a/src/main/java/org/elasticsearch/index/query/WrapperFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/WrapperFilterParser.java @@ -48,11 +48,11 @@ public class WrapperFilterParser implements FilterParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[wrapper] filter malformed"); + throw new QueryParsingException(parseContext, "[wrapper] filter malformed"); } String fieldName = parser.currentName(); if (!fieldName.equals("filter")) { - throw new QueryParsingException(parseContext.index(), "[wrapper] filter malformed"); + throw new QueryParsingException(parseContext, "[wrapper] filter malformed"); } parser.nextToken(); diff --git a/src/main/java/org/elasticsearch/index/query/WrapperQueryParser.java b/src/main/java/org/elasticsearch/index/query/WrapperQueryParser.java index 3fc16d7af74..f7b98ad3dd5 100644 --- a/src/main/java/org/elasticsearch/index/query/WrapperQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/WrapperQueryParser.java @@ -48,11 +48,11 @@ public class WrapperQueryParser implements QueryParser { XContentParser.Token token = parser.nextToken(); if (token != XContentParser.Token.FIELD_NAME) { - throw new QueryParsingException(parseContext.index(), "[wrapper] query malformed"); + throw new QueryParsingException(parseContext, "[wrapper] query malformed"); } String fieldName = parser.currentName(); if (!fieldName.equals("query")) { - throw new QueryParsingException(parseContext.index(), "[wrapper] query malformed"); + throw new QueryParsingException(parseContext, "[wrapper] query malformed"); } parser.nextToken(); diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionParser.java b/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionParser.java index 6268f4a5c74..001bdf05854 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionParser.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/DecayFunctionParser.java @@ -154,7 +154,7 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser { // the doc later MapperService.SmartNameFieldMappers smartMappers = parseContext.smartFieldMappers(fieldName); if (smartMappers == null || !smartMappers.hasMapper()) { - throw new QueryParsingException(parseContext.index(), "Unknown field [" + fieldName + "]"); + throw new QueryParsingException(parseContext, "Unknown field [" + fieldName + "]"); } FieldMapper mapper = smartMappers.fieldMappers().mapper(); @@ -167,7 +167,7 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser { } else if (mapper instanceof NumberFieldMapper) { return parseNumberVariable(fieldName, parser, parseContext, (NumberFieldMapper) mapper, mode); } else { - throw new QueryParsingException(parseContext.index(), "Field " + fieldName + " is of type " + mapper.fieldType() + throw new QueryParsingException(parseContext, "Field " + fieldName + " is of type " + mapper.fieldType() + ", but only numeric types are supported."); } } diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java b/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java index 10d4c7f3d55..734ab2f7759 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/FunctionScoreQueryParser.java @@ -134,7 +134,7 @@ public class FunctionScoreQueryParser implements QueryParser { // we try to parse a score function. If there is no score // function for the current field name, // functionParserMapper.get() will throw an Exception. - scoreFunction = functionParserMapper.get(parseContext.index(), currentFieldName).parse(parseContext, parser); + scoreFunction = functionParserMapper.get(parseContext, currentFieldName).parse(parseContext, parser); } if (functionArrayFound) { String errorString = "Found \"functions\": [...] already, now encountering \"" + currentFieldName + "\"."; @@ -202,8 +202,8 @@ public class FunctionScoreQueryParser implements QueryParser { ScoreFunction scoreFunction = null; Float functionWeight = null; if (token != XContentParser.Token.START_OBJECT) { - throw new QueryParsingException(parseContext.index(), NAME + ": malformed query, expected a " - + XContentParser.Token.START_OBJECT + " while parsing functions but got a " + token); + throw new QueryParsingException(parseContext, NAME + ": malformed query, expected a " + XContentParser.Token.START_OBJECT + + " while parsing functions but got a " + token); } else { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { @@ -217,7 +217,7 @@ public class FunctionScoreQueryParser implements QueryParser { // do not need to check null here, // functionParserMapper throws exception if parser // non-existent - ScoreFunctionParser functionParser = functionParserMapper.get(parseContext.index(), currentFieldName); + ScoreFunctionParser functionParser = functionParserMapper.get(parseContext, currentFieldName); scoreFunction = functionParser.parse(parseContext, parser); } } @@ -253,7 +253,7 @@ public class FunctionScoreQueryParser implements QueryParser { } else if ("first".equals(scoreMode)) { return FiltersFunctionScoreQuery.ScoreMode.First; } else { - throw new QueryParsingException(parseContext.index(), NAME + " illegal score_mode [" + scoreMode + "]"); + throw new QueryParsingException(parseContext, NAME + " illegal score_mode [" + scoreMode + "]"); } } @@ -261,7 +261,7 @@ public class FunctionScoreQueryParser implements QueryParser { String boostMode = parser.text(); CombineFunction cf = combineFunctionsMap.get(boostMode); if (cf == null) { - throw new QueryParsingException(parseContext.index(), NAME + " illegal boost_mode [" + boostMode + "]"); + throw new QueryParsingException(parseContext, NAME + " illegal boost_mode [" + boostMode + "]"); } return cf; } diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/ScoreFunctionParserMapper.java b/src/main/java/org/elasticsearch/index/query/functionscore/ScoreFunctionParserMapper.java index 4f7d9de390b..abe8b5c4e35 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/ScoreFunctionParserMapper.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/ScoreFunctionParserMapper.java @@ -20,9 +20,10 @@ package org.elasticsearch.index.query.functionscore; import com.google.common.collect.ImmutableMap; + import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.index.Index; +import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParsingException; import java.util.Set; @@ -42,10 +43,10 @@ public class ScoreFunctionParserMapper { this.functionParsers = builder.immutableMap(); } - public ScoreFunctionParser get(Index index, String parserName) { + public ScoreFunctionParser get(QueryParseContext parseContext, String parserName) { ScoreFunctionParser functionParser = get(parserName); if (functionParser == null) { - throw new QueryParsingException(index, "No function with the name [" + parserName + "] is registered."); + throw new QueryParsingException(parseContext, "No function with the name [" + parserName + "] is registered.", null); } return functionParser; } diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/fieldvaluefactor/FieldValueFactorFunctionParser.java b/src/main/java/org/elasticsearch/index/query/functionscore/fieldvaluefactor/FieldValueFactorFunctionParser.java index c5f454ef40a..90c4b953bed 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/fieldvaluefactor/FieldValueFactorFunctionParser.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/fieldvaluefactor/FieldValueFactorFunctionParser.java @@ -72,15 +72,15 @@ public class FieldValueFactorFunctionParser implements ScoreFunctionParser { } else if ("missing".equals(currentFieldName)) { missing = parser.doubleValue(); } else { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, NAMES[0] + " query does not support [" + currentFieldName + "]"); } } else if("factor".equals(currentFieldName) && (token == XContentParser.Token.START_ARRAY || token == XContentParser.Token.START_OBJECT)) { - throw new QueryParsingException(parseContext.index(), "[" + NAMES[0] + "] field 'factor' does not support lists or objects"); + throw new QueryParsingException(parseContext, "[" + NAMES[0] + "] field 'factor' does not support lists or objects"); } } if (field == null) { - throw new QueryParsingException(parseContext.index(), "[" + NAMES[0] + "] required field 'field' missing"); + throw new QueryParsingException(parseContext, "[" + NAMES[0] + "] required field 'field' missing"); } SearchContext searchContext = SearchContext.current(); diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/random/RandomScoreFunctionParser.java b/src/main/java/org/elasticsearch/index/query/functionscore/random/RandomScoreFunctionParser.java index 8bdc3074f3f..e4b26822d66 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/random/RandomScoreFunctionParser.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/random/RandomScoreFunctionParser.java @@ -21,6 +21,7 @@ package org.elasticsearch.index.query.functionscore.random; import com.google.common.primitives.Longs; + import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.search.function.RandomScoreFunction; import org.elasticsearch.common.lucene.search.function.ScoreFunction; @@ -66,15 +67,17 @@ public class RandomScoreFunctionParser implements ScoreFunctionParser { } else if (parser.numberType() == XContentParser.NumberType.LONG) { seed = Longs.hashCode(parser.longValue()); } else { - throw new QueryParsingException(parseContext.index(), "random_score seed must be an int, long or string, not '" + token.toString() + "'"); + throw new QueryParsingException(parseContext, "random_score seed must be an int, long or string, not '" + + token.toString() + "'"); } } else if (token == XContentParser.Token.VALUE_STRING) { seed = parser.text().hashCode(); } else { - throw new QueryParsingException(parseContext.index(), "random_score seed must be an int/long or string, not '" + token.toString() + "'"); + throw new QueryParsingException(parseContext, "random_score seed must be an int/long or string, not '" + + token.toString() + "'"); } } else { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, NAMES[0] + " query does not support [" + currentFieldName + "]"); } } } diff --git a/src/main/java/org/elasticsearch/index/query/functionscore/script/ScriptScoreFunctionParser.java b/src/main/java/org/elasticsearch/index/query/functionscore/script/ScriptScoreFunctionParser.java index aaa9bec3fac..b01eaee3615 100644 --- a/src/main/java/org/elasticsearch/index/query/functionscore/script/ScriptScoreFunctionParser.java +++ b/src/main/java/org/elasticsearch/index/query/functionscore/script/ScriptScoreFunctionParser.java @@ -28,7 +28,9 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParsingException; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; -import org.elasticsearch.script.*; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptContext; +import org.elasticsearch.script.ScriptParameterParser; import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue; import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.SearchScript; @@ -67,11 +69,11 @@ public class ScriptScoreFunctionParser implements ScoreFunctionParser { if ("params".equals(currentFieldName)) { vars = parser.map(); } else { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, NAMES[0] + " query does not support [" + currentFieldName + "]"); } } else if (token.isValue()) { if (!scriptParameterParser.token(currentFieldName, token, parser)) { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " query does not support [" + currentFieldName + "]"); + throw new QueryParsingException(parseContext, NAMES[0] + " query does not support [" + currentFieldName + "]"); } } } @@ -82,7 +84,7 @@ public class ScriptScoreFunctionParser implements ScoreFunctionParser { scriptType = scriptValue.scriptType(); } if (script == null) { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " requires 'script' field"); + throw new QueryParsingException(parseContext, NAMES[0] + " requires 'script' field"); } SearchScript searchScript; @@ -90,7 +92,7 @@ public class ScriptScoreFunctionParser implements ScoreFunctionParser { searchScript = parseContext.scriptService().search(parseContext.lookup(), new Script(scriptParameterParser.lang(), script, scriptType, vars), ScriptContext.Standard.SEARCH); return new ScriptScoreFunction(script, vars, searchScript); } catch (Exception e) { - throw new QueryParsingException(parseContext.index(), NAMES[0] + " the script could not be loaded", e); + throw new QueryParsingException(parseContext, NAMES[0] + " the script could not be loaded", e); } } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java b/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java index 149b47eadf9..ae839c41d1c 100644 --- a/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java +++ b/src/main/java/org/elasticsearch/index/query/support/InnerHitsQueryParserHelper.java @@ -72,7 +72,7 @@ public class InnerHitsQueryParserHelper { } } } catch (Exception e) { - throw new QueryParsingException(parserContext.index(), "Failed to parse [_inner_hits]", e); + throw new QueryParsingException(parserContext, "Failed to parse [_inner_hits]", e); } return new Tuple<>(innerHitName, subSearchContext); } diff --git a/src/main/java/org/elasticsearch/index/query/support/NestedInnerQueryParseSupport.java b/src/main/java/org/elasticsearch/index/query/support/NestedInnerQueryParseSupport.java index 17eb059e1d0..c96fdb7e103 100644 --- a/src/main/java/org/elasticsearch/index/query/support/NestedInnerQueryParseSupport.java +++ b/src/main/java/org/elasticsearch/index/query/support/NestedInnerQueryParseSupport.java @@ -106,10 +106,10 @@ public class NestedInnerQueryParseSupport { return innerQuery; } else { if (path == null) { - throw new QueryParsingException(parseContext.index(), "[nested] requires 'path' field"); + throw new QueryParsingException(parseContext, "[nested] requires 'path' field"); } if (!queryFound) { - throw new QueryParsingException(parseContext.index(), "[nested] requires either 'query' or 'filter' field"); + throw new QueryParsingException(parseContext, "[nested] requires either 'query' or 'filter' field"); } XContentParser old = parseContext.parser(); @@ -135,10 +135,10 @@ public class NestedInnerQueryParseSupport { return innerFilter; } else { if (path == null) { - throw new QueryParsingException(parseContext.index(), "[nested] requires 'path' field"); + throw new QueryParsingException(parseContext, "[nested] requires 'path' field"); } if (!filterFound) { - throw new QueryParsingException(parseContext.index(), "[nested] requires either 'query' or 'filter' field"); + throw new QueryParsingException(parseContext, "[nested] requires either 'query' or 'filter' field"); } setPathLevel(); @@ -160,15 +160,15 @@ public class NestedInnerQueryParseSupport { this.path = path; MapperService.SmartNameObjectMapper smart = parseContext.smartObjectMapper(path); if (smart == null) { - throw new QueryParsingException(parseContext.index(), "[nested] failed to find nested object under path [" + path + "]"); + throw new QueryParsingException(parseContext, "[nested] failed to find nested object under path [" + path + "]"); } childDocumentMapper = smart.docMapper(); nestedObjectMapper = smart.mapper(); if (nestedObjectMapper == null) { - throw new QueryParsingException(parseContext.index(), "[nested] failed to find nested object under path [" + path + "]"); + throw new QueryParsingException(parseContext, "[nested] failed to find nested object under path [" + path + "]"); } if (!nestedObjectMapper.nested().isNested()) { - throw new QueryParsingException(parseContext.index(), "[nested] nested object under path [" + path + "] is not of nested type"); + throw new QueryParsingException(parseContext, "[nested] nested object under path [" + path + "] is not of nested type"); } } diff --git a/src/main/java/org/elasticsearch/indices/store/IndicesStore.java b/src/main/java/org/elasticsearch/indices/store/IndicesStore.java index 32e695a828c..36c9be862ee 100644 --- a/src/main/java/org/elasticsearch/indices/store/IndicesStore.java +++ b/src/main/java/org/elasticsearch/indices/store/IndicesStore.java @@ -332,56 +332,57 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe // make sure shard is really there before register cluster state observer if (indexShard == null) { channel.sendResponse(new ShardActiveResponse(false, clusterService.localNode())); - } - // create observer here. we need to register it here because we need to capture the current cluster state - // which will then be compared to the one that is applied when we call waitForNextChange(). if we create it - // later we might miss an update and wait forever in case no new cluster state comes in. - // in general, using a cluster state observer here is a workaround for the fact that we cannot listen on shard state changes explicitly. - // instead we wait for the cluster state changes because we know any shard state change will trigger or be - // triggered by a cluster state change. - ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout, logger); - // check if shard is active. if so, all is good - boolean shardActive = shardActive(indexShard); - if (shardActive) { - channel.sendResponse(new ShardActiveResponse(true, clusterService.localNode())); } else { - // shard is not active, might be POST_RECOVERY so check if cluster state changed inbetween or wait for next change - observer.waitForNextChange(new ClusterStateObserver.Listener() { - @Override - public void onNewClusterState(ClusterState state) { - sendResult(shardActive(getShard(request))); - } - - @Override - public void onClusterServiceClose() { - sendResult(false); - } - - @Override - public void onTimeout(TimeValue timeout) { - sendResult(shardActive(getShard(request))); - } - - public void sendResult(boolean shardActive) { - try { - channel.sendResponse(new ShardActiveResponse(shardActive, clusterService.localNode())); - } catch (IOException e) { - logger.error("failed send response for shard active while trying to delete shard {} - shard will probably not be removed", e, request.shardId); - } catch (EsRejectedExecutionException e) { - logger.error("failed send response for shard active while trying to delete shard {} - shard will probably not be removed", e, request.shardId); + // create observer here. we need to register it here because we need to capture the current cluster state + // which will then be compared to the one that is applied when we call waitForNextChange(). if we create it + // later we might miss an update and wait forever in case no new cluster state comes in. + // in general, using a cluster state observer here is a workaround for the fact that we cannot listen on shard state changes explicitly. + // instead we wait for the cluster state changes because we know any shard state change will trigger or be + // triggered by a cluster state change. + ClusterStateObserver observer = new ClusterStateObserver(clusterService, request.timeout, logger); + // check if shard is active. if so, all is good + boolean shardActive = shardActive(indexShard); + if (shardActive) { + channel.sendResponse(new ShardActiveResponse(true, clusterService.localNode())); + } else { + // shard is not active, might be POST_RECOVERY so check if cluster state changed inbetween or wait for next change + observer.waitForNextChange(new ClusterStateObserver.Listener() { + @Override + public void onNewClusterState(ClusterState state) { + sendResult(shardActive(getShard(request))); } - } - }, new ClusterStateObserver.ValidationPredicate() { - @Override - protected boolean validate(ClusterState newState) { - // the shard is not there in which case we want to send back a false (shard is not active), so the cluster state listener must be notified - // or the shard is active in which case we want to send back that the shard is active - // here we could also evaluate the cluster state and get the information from there. we - // don't do it because we would have to write another method for this that would have the same effect - IndexShard indexShard = getShard(request); - return indexShard == null || shardActive(indexShard); - } - }); + + @Override + public void onClusterServiceClose() { + sendResult(false); + } + + @Override + public void onTimeout(TimeValue timeout) { + sendResult(shardActive(getShard(request))); + } + + public void sendResult(boolean shardActive) { + try { + channel.sendResponse(new ShardActiveResponse(shardActive, clusterService.localNode())); + } catch (IOException e) { + logger.error("failed send response for shard active while trying to delete shard {} - shard will probably not be removed", e, request.shardId); + } catch (EsRejectedExecutionException e) { + logger.error("failed send response for shard active while trying to delete shard {} - shard will probably not be removed", e, request.shardId); + } + } + }, new ClusterStateObserver.ValidationPredicate() { + @Override + protected boolean validate(ClusterState newState) { + // the shard is not there in which case we want to send back a false (shard is not active), so the cluster state listener must be notified + // or the shard is active in which case we want to send back that the shard is active + // here we could also evaluate the cluster state and get the information from there. we + // don't do it because we would have to write another method for this that would have the same effect + IndexShard indexShard = getShard(request); + return indexShard == null || shardActive(indexShard); + } + }); + } } } diff --git a/src/main/java/org/elasticsearch/percolator/PercolatorService.java b/src/main/java/org/elasticsearch/percolator/PercolatorService.java index 4c79c2164d8..96e7f326e4c 100644 --- a/src/main/java/org/elasticsearch/percolator/PercolatorService.java +++ b/src/main/java/org/elasticsearch/percolator/PercolatorService.java @@ -19,6 +19,7 @@ package org.elasticsearch.percolator; import com.carrotsearch.hppc.ByteObjectOpenHashMap; +import com.google.common.collect.Lists; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.ReaderUtil; @@ -85,8 +86,11 @@ import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.SearchParseElement; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.AggregationPhase; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.SiblingReducer; import org.elasticsearch.search.highlight.HighlightField; import org.elasticsearch.search.highlight.HighlightPhase; import org.elasticsearch.search.internal.SearchContext; @@ -846,15 +850,24 @@ public class PercolatorService extends AbstractComponent { return null; } - if (shardResults.size() == 1) { - return shardResults.get(0).aggregations(); - } - List aggregationsList = new ArrayList<>(shardResults.size()); for (PercolateShardResponse shardResult : shardResults) { aggregationsList.add(shardResult.aggregations()); } - return InternalAggregations.reduce(aggregationsList, new ReduceContext(bigArrays, scriptService)); + InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, new ReduceContext(bigArrays, scriptService)); + if (aggregations != null) { + List reducers = shardResults.get(0).reducers(); + if (reducers != null) { + List newAggs = new ArrayList<>(Lists.transform(aggregations.asList(), Reducer.AGGREGATION_TRANFORM_FUNCTION)); + for (SiblingReducer reducer : reducers) { + InternalAggregation newAgg = reducer.doReduce(new InternalAggregations(newAggs), new ReduceContext(bigArrays, + scriptService)); + newAggs.add(newAgg); + } + aggregations = new InternalAggregations(newAggs); + } + } + return aggregations; } } diff --git a/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java index be4e1b4e3f3..85b46925b5f 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/cluster/repositories/get/RestGetRepositoriesAction.java @@ -58,7 +58,7 @@ public class RestGetRepositoriesAction extends BaseRestHandler { public RestResponse buildResponse(GetRepositoriesResponse response, XContentBuilder builder) throws Exception { builder.startObject(); for (RepositoryMetaData repositoryMetaData : response.repositories()) { - RepositoriesMetaData.FACTORY.toXContent(repositoryMetaData, builder, request); + RepositoriesMetaData.toXContent(repositoryMetaData, builder, request); } builder.endObject(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java index 7e4e56710b7..dd1dca34bbc 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/get/RestGetIndicesAction.java @@ -146,7 +146,7 @@ public class RestGetIndicesAction extends BaseRestHandler { builder.startObject(Fields.WARMERS); if (warmers != null) { for (IndexWarmersMetaData.Entry warmer : warmers) { - IndexWarmersMetaData.FACTORY.toXContent(warmer, builder, params); + IndexWarmersMetaData.toXContent(warmer, builder, params); } } builder.endObject(); diff --git a/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java b/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java index 7023eecedd4..be83ccbe4b5 100644 --- a/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java +++ b/src/main/java/org/elasticsearch/rest/action/admin/indices/warmer/get/RestGetWarmerAction.java @@ -72,7 +72,7 @@ public class RestGetWarmerAction extends BaseRestHandler { builder.startObject(entry.key, XContentBuilder.FieldCaseConversion.NONE); builder.startObject(IndexWarmersMetaData.TYPE, XContentBuilder.FieldCaseConversion.NONE); for (IndexWarmersMetaData.Entry warmerEntry : entry.value) { - IndexWarmersMetaData.FACTORY.toXContent(warmerEntry, builder, request); + IndexWarmersMetaData.toXContent(warmerEntry, builder, request); } builder.endObject(); builder.endObject(); diff --git a/src/main/java/org/elasticsearch/search/SearchParseException.java b/src/main/java/org/elasticsearch/search/SearchParseException.java index 923532373a5..15c6bfd6f90 100644 --- a/src/main/java/org/elasticsearch/search/SearchParseException.java +++ b/src/main/java/org/elasticsearch/search/SearchParseException.java @@ -19,24 +19,64 @@ package org.elasticsearch.search; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.internal.SearchContext; +import java.io.IOException; + /** * */ public class SearchParseException extends SearchContextException { - public SearchParseException(SearchContext context, String msg) { - super(context, msg); + public static final int UNKNOWN_POSITION = -1; + private int lineNumber = UNKNOWN_POSITION; + private int columnNumber = UNKNOWN_POSITION; + + public SearchParseException(SearchContext context, String msg, @Nullable XContentLocation location) { + this(context, msg, location, null); } - public SearchParseException(SearchContext context, String msg, Throwable cause) { + public SearchParseException(SearchContext context, String msg, @Nullable XContentLocation location, Throwable cause) { super(context, msg, cause); + if (location != null) { + lineNumber = location.lineNumber; + columnNumber = location.columnNumber; + } } @Override public RestStatus status() { return RestStatus.BAD_REQUEST; } + + @Override + protected void innerToXContent(XContentBuilder builder, Params params) throws IOException { + if (lineNumber != UNKNOWN_POSITION) { + builder.field("line", lineNumber); + builder.field("col", columnNumber); + } + super.innerToXContent(builder, params); + } + + /** + * Line number of the location of the error + * + * @return the line number or -1 if unknown + */ + public int getLineNumber() { + return lineNumber; + } + + /** + * Column number of the location of the error + * + * @return the column number or -1 if unknown + */ + public int getColumnNumber() { + return columnNumber; + } } diff --git a/src/main/java/org/elasticsearch/search/SearchService.java b/src/main/java/org/elasticsearch/search/SearchService.java index 369c2cb499b..38f4e03a0f1 100644 --- a/src/main/java/org/elasticsearch/search/SearchService.java +++ b/src/main/java/org/elasticsearch/search/SearchService.java @@ -24,6 +24,7 @@ import com.carrotsearch.hppc.ObjectSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableMap; + import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -77,10 +78,23 @@ import org.elasticsearch.script.mustache.MustacheScriptEngineService; import org.elasticsearch.search.dfs.CachedDfSource; import org.elasticsearch.search.dfs.DfsPhase; import org.elasticsearch.search.dfs.DfsSearchResult; -import org.elasticsearch.search.fetch.*; -import org.elasticsearch.search.internal.*; +import org.elasticsearch.search.fetch.FetchPhase; +import org.elasticsearch.search.fetch.FetchSearchResult; +import org.elasticsearch.search.fetch.QueryFetchSearchResult; +import org.elasticsearch.search.fetch.ScrollQueryFetchSearchResult; +import org.elasticsearch.search.fetch.ShardFetchRequest; +import org.elasticsearch.search.internal.DefaultSearchContext; +import org.elasticsearch.search.internal.InternalScrollSearchRequest; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.SearchContext.Lifetime; -import org.elasticsearch.search.query.*; +import org.elasticsearch.search.internal.ShardSearchLocalRequest; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.QueryPhase; +import org.elasticsearch.search.query.QueryPhaseExecutionException; +import org.elasticsearch.search.query.QuerySearchRequest; +import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.query.QuerySearchResultProvider; +import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.search.warmer.IndexWarmersMetaData; import org.elasticsearch.threadpool.ThreadPool; @@ -718,7 +732,7 @@ public class SearchService extends AbstractLifecycleComponent { parser.nextToken(); SearchParseElement element = elementParsers.get(fieldName); if (element == null) { - throw new SearchParseException(context, "No parser for element [" + fieldName + "]"); + throw new SearchParseException(context, "No parser for element [" + fieldName + "]", parser.getTokenLocation()); } element.parse(parser, context); } else { @@ -736,7 +750,7 @@ public class SearchService extends AbstractLifecycleComponent { } catch (Throwable e1) { // ignore } - throw new SearchParseException(context, "Failed to parse source [" + sSource + "]", e); + throw new SearchParseException(context, "Failed to parse source [" + sSource + "]", parser.getTokenLocation(), e); } finally { if (parser != null) { parser.close(); diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java index 5b9fab55aa4..d41daa7363f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilder.java @@ -20,6 +20,7 @@ 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; diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java b/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java index 607757fb682..803b52bc0bf 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregationModule.java @@ -56,6 +56,11 @@ import org.elasticsearch.search.aggregations.metrics.stats.extended.ExtendedStat import org.elasticsearch.search.aggregations.metrics.sum.SumParser; import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsParser; import org.elasticsearch.search.aggregations.metrics.valuecount.ValueCountParser; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.MaxBucketParser; +import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeParser; +import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgParser; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelModule; import java.util.List; @@ -64,40 +69,45 @@ import java.util.List; */ public class AggregationModule extends AbstractModule implements SpawnModules{ - private List> parsers = Lists.newArrayList(); + private List> aggParsers = Lists.newArrayList(); + private List> reducerParsers = 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(PercentilesParser.class); - parsers.add(PercentileRanksParser.class); - parsers.add(CardinalityParser.class); + aggParsers.add(AvgParser.class); + aggParsers.add(SumParser.class); + aggParsers.add(MinParser.class); + aggParsers.add(MaxParser.class); + aggParsers.add(StatsParser.class); + aggParsers.add(ExtendedStatsParser.class); + aggParsers.add(ValueCountParser.class); + aggParsers.add(PercentilesParser.class); + aggParsers.add(PercentileRanksParser.class); + aggParsers.add(CardinalityParser.class); - parsers.add(GlobalParser.class); - parsers.add(MissingParser.class); - parsers.add(FilterParser.class); - parsers.add(FiltersParser.class); - parsers.add(SamplerParser.class); - parsers.add(TermsParser.class); - parsers.add(SignificantTermsParser.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(GeoHashGridParser.class); - parsers.add(NestedParser.class); - parsers.add(ReverseNestedParser.class); - parsers.add(TopHitsParser.class); - parsers.add(GeoBoundsParser.class); - parsers.add(ScriptedMetricParser.class); - parsers.add(ChildrenParser.class); + aggParsers.add(GlobalParser.class); + aggParsers.add(MissingParser.class); + aggParsers.add(FilterParser.class); + aggParsers.add(FiltersParser.class); + aggParsers.add(SamplerParser.class); + aggParsers.add(TermsParser.class); + aggParsers.add(SignificantTermsParser.class); + aggParsers.add(RangeParser.class); + aggParsers.add(DateRangeParser.class); + aggParsers.add(IpRangeParser.class); + aggParsers.add(HistogramParser.class); + aggParsers.add(DateHistogramParser.class); + aggParsers.add(GeoDistanceParser.class); + aggParsers.add(GeoHashGridParser.class); + aggParsers.add(NestedParser.class); + aggParsers.add(ReverseNestedParser.class); + aggParsers.add(TopHitsParser.class); + aggParsers.add(GeoBoundsParser.class); + aggParsers.add(ScriptedMetricParser.class); + aggParsers.add(ChildrenParser.class); + + reducerParsers.add(DerivativeParser.class); + reducerParsers.add(MaxBucketParser.class); + reducerParsers.add(MovAvgParser.class); } /** @@ -106,14 +116,18 @@ public class AggregationModule extends AbstractModule implements SpawnModules{ * @param parser The parser for the custom aggregator. */ public void addAggregatorParser(Class parser) { - parsers.add(parser); + aggParsers.add(parser); } @Override protected void configure() { - Multibinder multibinder = Multibinder.newSetBinder(binder(), Aggregator.Parser.class); - for (Class parser : parsers) { - multibinder.addBinding().to(parser); + Multibinder multibinderAggParser = Multibinder.newSetBinder(binder(), Aggregator.Parser.class); + for (Class parser : aggParsers) { + multibinderAggParser.addBinding().to(parser); + } + Multibinder multibinderReducerParser = Multibinder.newSetBinder(binder(), Reducer.Parser.class); + for (Class parser : reducerParsers) { + multibinderReducerParser.addBinding().to(parser); } bind(AggregatorParsers.class).asEagerSingleton(); bind(AggregationParseElement.class).asEagerSingleton(); @@ -122,7 +136,7 @@ public class AggregationModule extends AbstractModule implements SpawnModules{ @Override public Iterable spawnModules() { - return ImmutableList.of(new SignificantTermsHeuristicModule()); + return ImmutableList.of(new SignificantTermsHeuristicModule(), new MovAvgModelModule()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index a687eb23bd7..abc0827a3cc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -29,6 +29,8 @@ import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.search.SearchParseElement; import org.elasticsearch.search.SearchPhase; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.SiblingReducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.query.QueryPhaseExecutionException; @@ -74,7 +76,8 @@ public class AggregationPhase implements SearchPhase { List collectors = new ArrayList<>(); Aggregator[] aggregators; try { - aggregators = context.aggregations().factories().createTopLevelAggregators(aggregationContext); + AggregatorFactories factories = context.aggregations().factories(); + aggregators = factories.createTopLevelAggregators(aggregationContext); for (int i = 0; i < aggregators.length; i++) { if (aggregators[i] instanceof GlobalAggregator == false) { collectors.add(aggregators[i]); @@ -138,6 +141,21 @@ public class AggregationPhase implements SearchPhase { } } context.queryResult().aggregations(new InternalAggregations(aggregations)); + try { + List reducers = context.aggregations().factories().createReducers(); + List siblingReducers = new ArrayList<>(reducers.size()); + for (Reducer reducer : reducers) { + if (reducer instanceof SiblingReducer) { + siblingReducers.add((SiblingReducer) reducer); + } else { + throw new AggregationExecutionException("Invalid reducer named [" + reducer.name() + "] of type [" + + reducer.type().name() + "]. Only sibling reducers are allowed at the top level"); + } + } + context.queryResult().reducers(siblingReducers); + } catch (IOException e) { + throw new AggregationExecutionException("Failed to build top level reducers", e); + } // disable aggregations so that they don't run on next pages in case of scrolling context.aggregations(null); diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java index 9a2fa3a8a57..bb1e18be09d 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorBase.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations; import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.search.aggregations.bucket.BestBucketsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext.Lifetime; import org.elasticsearch.search.query.QueryPhaseExecutionException; @@ -46,6 +47,7 @@ public abstract class AggregatorBase extends Aggregator { private Map subAggregatorbyName; private DeferringBucketCollector recordingWrapper; + private final List reducers; /** * Constructs a new Aggregator. @@ -56,8 +58,10 @@ public abstract class AggregatorBase extends Aggregator { * @param parent The parent aggregator (may be {@code null} for top level aggregators) * @param metaData The metaData associated with this aggregator */ - protected AggregatorBase(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, Map metaData) throws IOException { + protected AggregatorBase(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, + List reducers, Map metaData) throws IOException { this.name = name; + this.reducers = reducers; this.metaData = metaData; this.parent = parent; this.context = context; @@ -112,6 +116,10 @@ public abstract class AggregatorBase extends Aggregator { return this.metaData; } + public List reducers() { + return this.reducers; + } + /** * Get a {@link LeafBucketCollector} for the given ctx, which should * delegate to the given collector. diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java index 99aa5f6aefd..ba464531b8b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactories.java @@ -18,12 +18,18 @@ */ package org.elasticsearch.search.aggregations; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.AggregationPath; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -33,18 +39,30 @@ public class AggregatorFactories { public static final AggregatorFactories EMPTY = new Empty(); + private AggregatorFactory parent; private AggregatorFactory[] factories; + private List reducerFactories; public static Builder builder() { return new Builder(); } - private AggregatorFactories(AggregatorFactory[] factories) { + private AggregatorFactories(AggregatorFactory[] factories, List reducers) { this.factories = factories; + this.reducerFactories = reducers; + } + + public List createReducers() throws IOException { + List reducers = new ArrayList<>(); + for (ReducerFactory factory : this.reducerFactories) { + reducers.add(factory.create()); + } + return reducers; } /** - * Create all aggregators so that they can be consumed with multiple buckets. + * Create all aggregators so that they can be consumed with multiple + * buckets. */ public Aggregator[] createSubAggregators(Aggregator parent) throws IOException { Aggregator[] aggregators = new Aggregator[count()]; @@ -75,6 +93,7 @@ public class AggregatorFactories { } void setParent(AggregatorFactory parent) { + this.parent = parent; for (AggregatorFactory factory : factories) { factory.parent = parent; } @@ -84,15 +103,19 @@ public class AggregatorFactories { for (AggregatorFactory factory : factories) { factory.validate(); } + for (ReducerFactory factory : reducerFactories) { + factory.validate(parent, factories, reducerFactories); + } } 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 static final List EMPTY_REDUCERS = new ArrayList<>(); private Empty() { - super(EMPTY_FACTORIES); + super(EMPTY_FACTORIES, EMPTY_REDUCERS); } @Override @@ -111,8 +134,9 @@ public class AggregatorFactories { private final Set names = new HashSet<>(); private final List factories = new ArrayList<>(); + private final List reducerFactories = new ArrayList<>(); - public Builder add(AggregatorFactory factory) { + public Builder addAggregator(AggregatorFactory factory) { if (!names.add(factory.name)) { throw new IllegalArgumentException("Two sibling aggregations cannot have the same name: [" + factory.name + "]"); } @@ -120,11 +144,65 @@ public class AggregatorFactories { return this; } + public Builder addReducer(ReducerFactory reducerFactory) { + this.reducerFactories.add(reducerFactory); + return this; + } + public AggregatorFactories build() { - if (factories.isEmpty()) { + if (factories.isEmpty() && reducerFactories.isEmpty()) { return EMPTY; } - return new AggregatorFactories(factories.toArray(new AggregatorFactory[factories.size()])); + List orderedReducers = resolveReducerOrder(this.reducerFactories, this.factories); + return new AggregatorFactories(factories.toArray(new AggregatorFactory[factories.size()]), orderedReducers); + } + + private List resolveReducerOrder(List reducerFactories, List aggFactories) { + Map reducerFactoriesMap = new HashMap<>(); + for (ReducerFactory factory : reducerFactories) { + reducerFactoriesMap.put(factory.getName(), factory); + } + Set aggFactoryNames = new HashSet<>(); + for (AggregatorFactory aggFactory : aggFactories) { + aggFactoryNames.add(aggFactory.name); + } + List orderedReducers = new LinkedList<>(); + List unmarkedFactories = new ArrayList(reducerFactories); + Set temporarilyMarked = new HashSet(); + while (!unmarkedFactories.isEmpty()) { + ReducerFactory factory = unmarkedFactories.get(0); + resolveReducerOrder(aggFactoryNames, reducerFactoriesMap, orderedReducers, unmarkedFactories, temporarilyMarked, factory); + } + return orderedReducers; + } + + private void resolveReducerOrder(Set aggFactoryNames, Map reducerFactoriesMap, + List orderedReducers, List unmarkedFactories, Set temporarilyMarked, + ReducerFactory factory) { + if (temporarilyMarked.contains(factory)) { + throw new IllegalStateException("Cyclical dependancy found with reducer [" + factory.getName() + "]"); + } else if (unmarkedFactories.contains(factory)) { + temporarilyMarked.add(factory); + String[] bucketsPaths = factory.getBucketsPaths(); + for (String bucketsPath : bucketsPaths) { + List bucketsPathElements = AggregationPath.parse(bucketsPath).getPathElementsAsStringList(); + String firstAggName = bucketsPathElements.get(0); + if (bucketsPath.equals("_count") || bucketsPath.equals("_key") || aggFactoryNames.contains(firstAggName)) { + continue; + } else { + ReducerFactory matchingFactory = reducerFactoriesMap.get(firstAggName); + if (matchingFactory != null) { + resolveReducerOrder(aggFactoryNames, reducerFactoriesMap, orderedReducers, unmarkedFactories, + temporarilyMarked, matchingFactory); + } else { + throw new IllegalStateException("No aggregation found for path [" + bucketsPath + "]"); + } + } + } + unmarkedFactories.remove(factory); + temporarilyMarked.remove(factory); + orderedReducers.add(factory); + } } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java index 256700bada5..f69e54ee710 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorFactory.java @@ -23,10 +23,12 @@ import org.apache.lucene.search.Scorer; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ObjectArray; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext.Lifetime; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -64,6 +66,10 @@ public abstract class AggregatorFactory { return this; } + public String name() { + return name; + } + /** * Validates the state of this factory (makes sure the factory is properly configured) */ @@ -79,7 +85,8 @@ public abstract class AggregatorFactory { return parent; } - protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException; + protected abstract Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException; /** * Creates the aggregator @@ -92,7 +99,7 @@ public abstract class AggregatorFactory { * @return The created aggregator */ public final Aggregator create(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket) throws IOException { - return createInternal(context, parent, collectsFromSingleBucket, this.metaData); + return createInternal(context, parent, collectsFromSingleBucket, this.factories.createReducers(), this.metaData); } public void doValidate() { @@ -102,6 +109,8 @@ public abstract class AggregatorFactory { this.metaData = metaData; } + + /** * Utility method. Given an {@link AggregatorFactory} that creates {@link Aggregator}s that only know how * to collect bucket 0, this returns an aggregator that can collect any bucket. diff --git a/src/main/java/org/elasticsearch/search/aggregations/AggregatorParsers.java b/src/main/java/org/elasticsearch/search/aggregations/AggregatorParsers.java index b55f6a4f022..32969104733 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/AggregatorParsers.java +++ b/src/main/java/org/elasticsearch/search/aggregations/AggregatorParsers.java @@ -19,10 +19,13 @@ 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.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -37,21 +40,30 @@ import java.util.regex.Pattern; public class AggregatorParsers { public static final Pattern VALID_AGG_NAME = Pattern.compile("[^\\[\\]>]+"); - private final ImmutableMap parsers; + private final ImmutableMap aggParsers; + private final ImmutableMap reducerParsers; /** * 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}). + * @param aggParsers + * The available aggregator parsers (dynamically injected by the + * {@link org.elasticsearch.search.aggregations.AggregationModule} + * ). */ @Inject - public AggregatorParsers(Set parsers) { - MapBuilder builder = MapBuilder.newMapBuilder(); - for (Aggregator.Parser parser : parsers) { - builder.put(parser.type(), parser); + public AggregatorParsers(Set aggParsers, Set reducerParsers) { + MapBuilder aggParsersBuilder = MapBuilder.newMapBuilder(); + for (Aggregator.Parser parser : aggParsers) { + aggParsersBuilder.put(parser.type(), parser); } - this.parsers = builder.immutableMap(); + this.aggParsers = aggParsersBuilder.immutableMap(); + MapBuilder reducerParsersBuilder = MapBuilder.newMapBuilder(); + for (Reducer.Parser parser : reducerParsers) { + reducerParsersBuilder.put(parser.type(), parser); + } + this.reducerParsers = reducerParsersBuilder.immutableMap(); } /** @@ -61,7 +73,18 @@ public class AggregatorParsers { * @return The parser associated with the given aggregation type. */ public Aggregator.Parser parser(String type) { - return parsers.get(type); + return aggParsers.get(type); + } + + /** + * Returns the parser that is registered under the given reducer type. + * + * @param type + * The reducer type + * @return The parser associated with the given reducer type. + */ + public Reducer.Parser reducer(String type) { + return reducerParsers.get(type); } /** @@ -86,32 +109,39 @@ public class AggregatorParsers { XContentParser.Token token = null; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token != XContentParser.Token.FIELD_NAME) { - throw new SearchParseException(context, "Unexpected token " + token + " in [aggs]: aggregations definitions must start with the name of the aggregation."); + throw new SearchParseException(context, "Unexpected token " + token + + " in [aggs]: aggregations definitions must start with the name of the aggregation.", parser.getTokenLocation()); } final String aggregationName = parser.currentName(); if (!validAggMatcher.reset(aggregationName).matches()) { - throw new SearchParseException(context, "Invalid aggregation name [" + aggregationName + "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'"); + throw new SearchParseException(context, "Invalid aggregation name [" + aggregationName + + "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'", parser.getTokenLocation()); } token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new SearchParseException(context, "Aggregation definition for [" + aggregationName + " starts with a [" + token + "], expected a [" + XContentParser.Token.START_OBJECT + "]."); + throw new SearchParseException(context, "Aggregation definition for [" + aggregationName + " starts with a [" + token + + "], expected a [" + XContentParser.Token.START_OBJECT + "].", parser.getTokenLocation()); } - AggregatorFactory factory = null; + AggregatorFactory aggFactory = null; + ReducerFactory reducerFactory = null; AggregatorFactories subFactories = null; Map metaData = null; while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token != XContentParser.Token.FIELD_NAME) { - throw new SearchParseException(context, "Expected [" + XContentParser.Token.FIELD_NAME + "] under a [" + XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]"); + throw new SearchParseException(context, "Expected [" + XContentParser.Token.FIELD_NAME + "] under a [" + + XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]", + parser.getTokenLocation()); } final String fieldName = parser.currentName(); token = parser.nextToken(); if (token != XContentParser.Token.START_OBJECT) { - throw new SearchParseException(context, "Expected [" + XContentParser.Token.START_OBJECT + "] under [" + fieldName + "], but got a [" + token + "] in [" + aggregationName + "]"); + throw new SearchParseException(context, "Expected [" + XContentParser.Token.START_OBJECT + "] under [" + fieldName + + "], but got a [" + token + "] in [" + aggregationName + "]", parser.getTokenLocation()); } switch (fieldName) { @@ -121,39 +151,63 @@ public class AggregatorParsers { case "aggregations": case "aggs": if (subFactories != null) { - throw new SearchParseException(context, "Found two sub aggregation definitions under [" + aggregationName + "]"); + throw new SearchParseException(context, "Found two sub aggregation definitions under [" + aggregationName + "]", + parser.getTokenLocation()); } subFactories = parseAggregators(parser, context, level+1); break; default: - if (factory != null) { - throw new SearchParseException(context, "Found two aggregation type definitions in [" + aggregationName + "]: [" + factory.type + "] and [" + fieldName + "]"); + if (aggFactory != null) { + throw new SearchParseException(context, "Found two aggregation type definitions in [" + aggregationName + "]: [" + + aggFactory.type + "] and [" + fieldName + "]", parser.getTokenLocation()); } + if (reducerFactory != null) { + // TODO we would need a .type property on reducers too for this error message? + throw new SearchParseException(context, "Found two aggregation type definitions in [" + aggregationName + "]: [" + + reducerFactory + "] and [" + fieldName + "]", parser.getTokenLocation()); + } + Aggregator.Parser aggregatorParser = parser(fieldName); if (aggregatorParser == null) { - throw new SearchParseException(context, "Could not find aggregator type [" + fieldName + "] in [" + aggregationName + "]"); + Reducer.Parser reducerParser = reducer(fieldName); + if (reducerParser == null) { + throw new SearchParseException(context, "Could not find aggregator type [" + fieldName + "] in [" + + aggregationName + "]", parser.getTokenLocation()); + } else { + reducerFactory = reducerParser.parse(aggregationName, parser, context); + } + } else { + aggFactory = aggregatorParser.parse(aggregationName, parser, context); } - factory = aggregatorParser.parse(aggregationName, parser, context); } } - if (factory == null) { - throw new SearchParseException(context, "Missing definition for aggregation [" + aggregationName + "]"); - } - + if (aggFactory == null && reducerFactory == null) { + throw new SearchParseException(context, "Missing definition for aggregation [" + aggregationName + "]", + parser.getTokenLocation()); + } else if (aggFactory != null) { + assert reducerFactory == null; if (metaData != null) { - factory.setMetaData(metaData); + aggFactory.setMetaData(metaData); } if (subFactories != null) { - factory.subFactories(subFactories); + aggFactory.subFactories(subFactories); } if (level == 0) { - factory.validate(); + aggFactory.validate(); } - factories.add(factory); + factories.addAggregator(aggFactory); + } else { + assert reducerFactory != null; + if (subFactories != null) { + throw new SearchParseException(context, "Aggregation [" + aggregationName + "] cannot define sub-aggregations", + parser.getTokenLocation()); + } + factories.addReducer(reducerFactory); + } } return factories.build(); diff --git a/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java b/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java index 5fe69e74060..c9bb647c0f3 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java +++ b/src/main/java/org/elasticsearch/search/aggregations/InternalAggregation.java @@ -18,6 +18,9 @@ */ package org.elasticsearch.search.aggregations; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; @@ -28,6 +31,8 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerStreams; import org.elasticsearch.search.aggregations.support.AggregationPath; import java.io.IOException; @@ -110,6 +115,8 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St protected Map metaData; + private List reducers; + /** Constructs an un initialized addAggregation (used for serialization) **/ protected InternalAggregation() {} @@ -118,8 +125,9 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St * * @param name The name of the get. */ - protected InternalAggregation(String name, Map metaData) { + protected InternalAggregation(String name, List reducers, Map metaData) { this.name = name; + this.reducers = reducers; this.metaData = metaData; } @@ -139,7 +147,15 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St * try reusing an existing get instance (typically the first in the given list) to save on redundant object * construction. */ - public abstract InternalAggregation reduce(List aggregations, ReduceContext reduceContext); + public final InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + InternalAggregation aggResult = doReduce(aggregations, reduceContext); + for (Reducer reducer : reducers) { + aggResult = reducer.reduce(aggResult, reduceContext); + } + return aggResult; + } + + public abstract InternalAggregation doReduce(List aggregations, ReduceContext reduceContext); @Override public Object getProperty(String path) { @@ -172,6 +188,10 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St return metaData; } + public List reducers() { + return reducers; + } + @Override public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(name); @@ -190,6 +210,11 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St public final void writeTo(StreamOutput out) throws IOException { out.writeString(name); out.writeGenericValue(metaData); + out.writeVInt(reducers.size()); + for (Reducer reducer : reducers) { + out.writeBytesReference(reducer.type().stream()); + reducer.writeTo(out); + } doWriteTo(out); } @@ -199,6 +224,17 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St public final void readFrom(StreamInput in) throws IOException { name = in.readString(); metaData = in.readMap(); + int size = in.readVInt(); + if (size == 0) { + reducers = ImmutableList.of(); + } else { + reducers = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; i++) { + BytesReference type = in.readBytesReference(); + Reducer reducer = ReducerStreams.stream(type).readResult(in); + reducers.add(reducer); + } + } doReadFrom(in); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java b/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java index f7c4fc9809b..db2ac49bf38 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java +++ b/src/main/java/org/elasticsearch/search/aggregations/InternalMultiBucketAggregation.java @@ -20,19 +20,43 @@ package org.elasticsearch.search.aggregations; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.util.List; import java.util.Map; -public abstract class InternalMultiBucketAggregation extends InternalAggregation implements MultiBucketsAggregation { +public abstract class InternalMultiBucketAggregation + extends InternalAggregation implements MultiBucketsAggregation { public InternalMultiBucketAggregation() { } - public InternalMultiBucketAggregation(String name, Map metaData) { - super(name, metaData); + public InternalMultiBucketAggregation(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } + /** + * Create a new copy of this {@link Aggregation} with the same settings as + * this {@link Aggregation} and contains the provided buckets. + * + * @param buckets + * the buckets to use in the new {@link Aggregation} + * @return the new {@link Aggregation} + */ + public abstract A create(List buckets); + + /** + * Create a new {@link InternalBucket} using the provided prototype bucket + * and aggregations. + * + * @param aggregations + * the aggregations for the new bucket + * @param prototype + * the bucket to use as a prototype + * @return the new bucket + */ + public abstract B createBucket(InternalAggregations aggregations, B prototype); + @Override public Object getProperty(List path) { if (path.isEmpty()) { @@ -57,18 +81,19 @@ public abstract class InternalMultiBucketAggregation extends InternalAggregation String aggName = path.get(0); if (aggName.equals("_count")) { if (path.size() > 1) { - throw new IllegalArgumentException("_count must be the last element in the path"); + throw new InvalidAggregationPathException("_count must be the last element in the path"); } return getDocCount(); } else if (aggName.equals("_key")) { if (path.size() > 1) { - throw new IllegalArgumentException("_key must be the last element in the path"); + throw new InvalidAggregationPathException("_key must be the last element in the path"); } return getKey(); } InternalAggregation aggregation = aggregations.get(aggName); if (aggregation == null) { - throw new IllegalArgumentException("Cannot find an aggregation named [" + aggName + "] in [" + containingAggName + "]"); + throw new InvalidAggregationPathException("Cannot find an aggregation named [" + aggName + "] in [" + containingAggName + + "]"); } return aggregation.getProperty(path.subList(1, path.size())); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/InvalidAggregationPathException.java b/src/main/java/org/elasticsearch/search/aggregations/InvalidAggregationPathException.java new file mode 100644 index 00000000000..e2ab1f65245 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/InvalidAggregationPathException.java @@ -0,0 +1,33 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations; + +import org.elasticsearch.ElasticsearchException; + +public class InvalidAggregationPathException extends ElasticsearchException { + + public InvalidAggregationPathException(String msg) { + super(msg); + } + + public InvalidAggregationPathException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java index 33c4215e27a..9b64c647b38 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/NonCollectingAggregator.java @@ -20,9 +20,11 @@ package org.elasticsearch.search.aggregations; import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -31,12 +33,14 @@ import java.util.Map; */ public abstract class NonCollectingAggregator extends AggregatorBase { - protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, Map metaData) throws IOException { - super(name, subFactories, context, parent, metaData); + protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, AggregatorFactories subFactories, + List reducers, Map metaData) throws IOException { + super(name, subFactories, context, parent, reducers, metaData); } - protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - this(name, context, parent, AggregatorFactories.EMPTY, metaData); + protected NonCollectingAggregator(String name, AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + this(name, context, parent, AggregatorFactories.EMPTY, reducers, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java b/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java index a45b9b9857a..b0fe986a081 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java +++ b/src/main/java/org/elasticsearch/search/aggregations/TransportAggregationModule.java @@ -59,6 +59,12 @@ import org.elasticsearch.search.aggregations.metrics.stats.extended.InternalExte import org.elasticsearch.search.aggregations.metrics.sum.InternalSum; import org.elasticsearch.search.aggregations.metrics.tophits.InternalTopHits; import org.elasticsearch.search.aggregations.metrics.valuecount.InternalValueCount; +import org.elasticsearch.search.aggregations.reducers.InternalSimpleValue; +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBucketMetricValue; +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.MaxBucketReducer; +import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeReducer; +import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgReducer; +import org.elasticsearch.search.aggregations.reducers.movavg.models.TransportMovAvgModelModule; /** * A module that registers all the transport streams for the addAggregation @@ -93,7 +99,7 @@ public class TransportAggregationModule extends AbstractModule implements SpawnM SignificantStringTerms.registerStreams(); SignificantLongTerms.registerStreams(); UnmappedSignificantTerms.registerStreams(); - InternalGeoHashGrid.registerStreams(); + InternalGeoHashGrid.registerStreams(); DoubleTerms.registerStreams(); UnmappedTerms.registerStreams(); InternalRange.registerStream(); @@ -106,10 +112,17 @@ public class TransportAggregationModule extends AbstractModule implements SpawnM InternalTopHits.registerStreams(); InternalGeoBounds.registerStream(); InternalChildren.registerStream(); + + // Reducers + DerivativeReducer.registerStreams(); + InternalSimpleValue.registerStreams(); + InternalBucketMetricValue.registerStreams(); + MaxBucketReducer.registerStreams(); + MovAvgReducer.registerStreams(); } @Override public Iterable spawnModules() { - return ImmutableList.of(new TransportSignificantTermsHeuristicModule()); + return ImmutableList.of(new TransportSignificantTermsHeuristicModule(), new TransportMovAvgModelModule()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java index e4d0260cf93..041c15a5dc1 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/BucketsAggregator.java @@ -27,10 +27,12 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -41,9 +43,9 @@ public abstract class BucketsAggregator extends AggregatorBase { private final BigArrays bigArrays; private IntArray docCounts; - public BucketsAggregator(String name, AggregatorFactories factories, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, factories, context, parent, metaData); + public BucketsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, + List reducers, Map metaData) throws IOException { + super(name, factories, context, parent, reducers, metaData); bigArrays = context.bigArrays(); docCounts = bigArrays.newIntArray(1, true); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java index 29f39539293..b9ff232b626 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/InternalSingleBucketAggregation.java @@ -23,6 +23,7 @@ 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 org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -46,8 +47,8 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio * @param docCount The document count in the single bucket. * @param aggregations The already built sub-aggregations that are associated with the bucket. */ - protected InternalSingleBucketAggregation(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, metaData); + protected InternalSingleBucketAggregation(String name, long docCount, InternalAggregations aggregations, List reducers, Map metaData) { + super(name, reducers, metaData); this.docCount = docCount; this.aggregations = aggregations; } @@ -68,7 +69,7 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio protected abstract InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations); @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { long docCount = 0L; List subAggregationsList = new ArrayList<>(aggregations.size()); for (InternalAggregation aggregation : aggregations) { diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java index d8b884a88e4..2e032640f98 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/SingleBucketAggregator.java @@ -20,9 +20,11 @@ package org.elasticsearch.search.aggregations.bucket; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -31,8 +33,9 @@ import java.util.Map; public abstract class SingleBucketAggregator extends BucketsAggregator { protected SingleBucketAggregator(String name, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, + List reducers, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenParser.java index 4834774053b..aacd76b0b5b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenParser.java @@ -56,15 +56,18 @@ public class ChildrenParser implements Aggregator.Parser { if ("type".equals(currentFieldName)) { childType = parser.text(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (childType == null) { - throw new SearchParseException(context, "Missing [child_type] field for children aggregation [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [child_type] field for children aggregation [" + aggregationName + "]", + parser.getTokenLocation()); } ValuesSourceConfig config = new ValuesSourceConfig<>(ValuesSource.Bytes.WithOrdinals.ParentChild.class); @@ -76,7 +79,7 @@ public class ChildrenParser implements Aggregator.Parser { if (childDocMapper != null) { ParentFieldMapper parentFieldMapper = childDocMapper.parentFieldMapper(); if (!parentFieldMapper.active()) { - throw new SearchParseException(context, "[children] _parent field not configured"); + throw new SearchParseException(context, "[children] _parent field not configured", parser.getTokenLocation()); } parentType = parentFieldMapper.type(); DocumentMapper parentDocMapper = context.mapperService().documentMapper(parentType); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java index 427637b9da7..cfac7f834bc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/InternalChildren.java @@ -23,8 +23,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,9 @@ public class InternalChildren extends InternalSingleBucketAggregation implements public InternalChildren() { } - public InternalChildren(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, docCount, aggregations, metaData); + public InternalChildren(String name, long docCount, InternalAggregations aggregations, List reducers, + Map metaData) { + super(name, docCount, aggregations, reducers, metaData); } @Override @@ -60,6 +63,6 @@ public class InternalChildren extends InternalSingleBucketAggregation implements @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalChildren(name, docCount, subAggregations, getMetaData()); + return new InternalChildren(name, docCount, subAggregations, reducers(), getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java index 2f1864458d7..3775ab853a2 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ParentToChildrenAggregator.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.util.LongObjectPagedHashMap; import org.elasticsearch.index.search.child.ConstantScorer; import org.elasticsearch.search.aggregations.*; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -63,8 +64,9 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator { public ParentToChildrenAggregator(String name, AggregatorFactories factories, AggregationContext aggregationContext, Aggregator parent, String parentType, Filter childFilter, Filter parentFilter, - ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, long maxOrd, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, + long maxOrd, List reducers, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); this.parentType = parentType; // these two filters are cached in the parser this.childFilter = childFilter; @@ -77,12 +79,13 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalChildren(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalChildren(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } @Override @@ -192,21 +195,25 @@ public class ParentToChildrenAggregator extends SingleBucketAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new NonCollectingAggregator(name, aggregationContext, parent, reducers, metaData) { @Override public InternalAggregation buildEmptyAggregation() { - return new InternalChildren(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalChildren(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } }; } @Override - protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource.Bytes.WithOrdinals.ParentChild valuesSource, + AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, List reducers, + Map metaData) throws IOException { long maxOrd = valuesSource.globalMaxOrd(aggregationContext.searchContext().searcher(), parentType); - return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, valuesSource, maxOrd, metaData); + return new ParentToChildrenAggregator(name, factories, aggregationContext, parent, parentType, childFilter, parentFilter, + valuesSource, maxOrd, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java index d5b15dba1ca..6459ff83215 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregator.java @@ -22,6 +22,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Filter; import org.apache.lucene.util.Bits; import org.elasticsearch.common.lucene.docset.DocIdSets; +import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -29,9 +30,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -45,9 +48,9 @@ public class FilterAggregator extends SingleBucketAggregator { org.apache.lucene.search.Filter filter, AggregatorFactories factories, AggregationContext aggregationContext, - Aggregator parent, + Aggregator parent, List reducers, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + super(name, factories, aggregationContext, parent, reducers, metaData); this.filter = filter; } @@ -69,12 +72,13 @@ public class FilterAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalFilter(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalFilter(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalFilter(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } public static class Factory extends AggregatorFactory { @@ -87,8 +91,9 @@ public class FilterAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new FilterAggregator(name, filter, factories, context, parent, metaData); + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { + return new FilterAggregator(name, filter, factories, context, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilter.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilter.java index c3d84b9fe51..0429ea20a59 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilter.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/InternalFilter.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -48,8 +50,8 @@ public class InternalFilter extends InternalSingleBucketAggregation implements F InternalFilter() {} // for serialization - InternalFilter(String name, long docCount, InternalAggregations subAggregations, Map metaData) { - super(name, docCount, subAggregations, metaData); + InternalFilter(String name, long docCount, InternalAggregations subAggregations, List reducers, Map metaData) { + super(name, docCount, subAggregations, reducers, metaData); } @Override @@ -59,6 +61,6 @@ public class InternalFilter extends InternalSingleBucketAggregation implements F @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalFilter(name, docCount, subAggregations, getMetaData()); + return new InternalFilter(name, docCount, subAggregations, reducers(), getMetaData()); } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java index b97a5442ced..913d844cb6a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregator.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Filter; import org.apache.lucene.util.Bits; import org.elasticsearch.common.lucene.docset.DocIdSets; +import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -33,6 +34,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; @@ -59,8 +61,9 @@ public class FiltersAggregator extends BucketsAggregator { private final boolean keyed; public FiltersAggregator(String name, AggregatorFactories factories, List filters, boolean keyed, AggregationContext aggregationContext, - Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + Aggregator parent, List reducers, Map metaData) + throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); this.keyed = keyed; this.filters = filters.toArray(new KeyedFilter[filters.size()]); } @@ -73,7 +76,7 @@ public class FiltersAggregator extends BucketsAggregator { final Bits[] bits = new Bits[filters.length]; for (int i = 0; i < filters.length; ++i) { bits[i] = DocIdSets.asSequentialAccessBits(ctx.reader().maxDoc(), filters[i].filter.getDocIdSet(ctx, null)); - } + } return new LeafBucketCollectorBase(sub, null) { @Override public void collect(int doc, long bucket) throws IOException { @@ -95,7 +98,7 @@ public class FiltersAggregator extends BucketsAggregator { InternalFilters.Bucket bucket = new InternalFilters.Bucket(filter.key, bucketDocCount(bucketOrd), bucketAggregations(bucketOrd), keyed); buckets.add(bucket); } - return new InternalFilters(name, buckets, keyed, metaData()); + return new InternalFilters(name, buckets, keyed, reducers(), metaData()); } @Override @@ -106,7 +109,7 @@ public class FiltersAggregator extends BucketsAggregator { InternalFilters.Bucket bucket = new InternalFilters.Bucket(filters[i].key, 0, subAggs, keyed); buckets.add(bucket); } - return new InternalFilters(name, buckets, keyed, metaData()); + return new InternalFilters(name, buckets, keyed, reducers(), metaData()); } final long bucketOrd(long owningBucketOrdinal, int filterOrd) { @@ -125,8 +128,9 @@ public class FiltersAggregator extends BucketsAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new FiltersAggregator(name, factories, filters, keyed, context, parent, metaData); + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { + return new FiltersAggregator(name, factories, filters, keyed, context, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersParser.java index 49f43eafc36..e30fcc8a3a4 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersParser.java @@ -65,7 +65,8 @@ public class FiltersParser implements Aggregator.Parser { } } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_ARRAY) { if ("filters".equals(currentFieldName)) { @@ -78,10 +79,12 @@ public class FiltersParser implements Aggregator.Parser { idx++; } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/InternalFilters.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/InternalFilters.java index 91624557740..85477b0d70c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/InternalFilters.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/InternalFilters.java @@ -29,8 +29,10 @@ import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation.InternalBucket; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -41,7 +43,7 @@ import java.util.Map; /** * */ -public class InternalFilters extends InternalMultiBucketAggregation implements Filters { +public class InternalFilters extends InternalMultiBucketAggregation implements Filters { public final static Type TYPE = new Type("filters"); @@ -163,8 +165,8 @@ public class InternalFilters extends InternalMultiBucketAggregation implements F public InternalFilters() {} // for serialization - public InternalFilters(String name, List buckets, boolean keyed, Map metaData) { - super(name, metaData); + public InternalFilters(String name, List buckets, boolean keyed, List reducers, Map metaData) { + super(name, reducers, metaData); this.buckets = buckets; this.keyed = keyed; } @@ -174,6 +176,16 @@ public class InternalFilters extends InternalMultiBucketAggregation implements F return TYPE; } + @Override + public InternalFilters create(List buckets) { + return new InternalFilters(this.name, buckets, this.keyed, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, aggregations, prototype.keyed); + } + @Override public List getBuckets() { return buckets; @@ -191,7 +203,7 @@ public class InternalFilters extends InternalMultiBucketAggregation implements F } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { List> bucketsList = null; for (InternalAggregation aggregation : aggregations) { InternalFilters filters = (InternalFilters) aggregation; @@ -210,7 +222,7 @@ public class InternalFilters extends InternalMultiBucketAggregation implements F } } - InternalFilters reduced = new InternalFilters(name, new ArrayList(bucketsList.size()), keyed, getMetaData()); + InternalFilters reduced = new InternalFilters(name, new ArrayList(bucketsList.size()), keyed, reducers(), getMetaData()); for (List sameRangeList : bucketsList) { reduced.buckets.add((sameRangeList.get(0)).reduce(sameRangeList, reduceContext)); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java index 7e9f4682207..36448a103c1 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridAggregator.java @@ -28,12 +28,14 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,9 @@ public class GeoHashGridAggregator extends BucketsAggregator { private final LongHash bucketOrds; public GeoHashGridAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, - int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + int requiredSize, int shardSize, AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.requiredSize = requiredSize; this.shardSize = shardSize; @@ -126,12 +129,12 @@ public class GeoHashGridAggregator extends BucketsAggregator { bucket.aggregations = bucketAggregations(bucket.bucketOrd); list[i] = bucket; } - return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), metaData()); + return new InternalGeoHashGrid(name, requiredSize, Arrays.asList(list), reducers(), metaData()); } @Override public InternalGeoHashGrid buildEmptyAggregation() { - return new InternalGeoHashGrid(name, requiredSize, Collections.emptyList(), metaData()); + return new InternalGeoHashGrid(name, requiredSize, Collections. emptyList(), reducers(), metaData()); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java index 185e5e002f6..24b6d490c9f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoHashGridParser.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.BucketUtils; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -43,6 +44,7 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -123,10 +125,11 @@ public class GeoHashGridParser implements Aggregator.Parser { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, Collections.emptyList(), metaData); - return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { - @Override + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + final InternalAggregation aggregation = new InternalGeoHashGrid(name, requiredSize, + Collections. emptyList(), reducers, metaData); + return new NonCollectingAggregator(name, aggregationContext, parent, reducers, metaData) { public InternalAggregation buildEmptyAggregation() { return aggregation; } @@ -134,12 +137,15 @@ public class GeoHashGridParser implements Aggregator.Parser { } @Override - protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, + Aggregator parent, boolean collectsFromSingleBucket, List reducers, Map metaData) + throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, aggregationContext, parent); } ValuesSource.Numeric cellIdSource = new CellIdSource(valuesSource, precision); - return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, metaData); + return new GeoHashGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, aggregationContext, parent, reducers, + metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGrid.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGrid.java index 40ab098b624..c480aa667c7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGrid.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/InternalGeoHashGrid.java @@ -32,6 +32,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -45,7 +46,8 @@ import java.util.Map; * All geohashes in a grid are of the same precision and held internally as a single long * for efficiency's sake. */ -public class InternalGeoHashGrid extends InternalMultiBucketAggregation implements GeoHashGrid { +public class InternalGeoHashGrid extends InternalMultiBucketAggregation implements + GeoHashGrid { public static final Type TYPE = new Type("geohash_grid", "ghcells"); @@ -162,7 +164,6 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation implemen return builder; } } - private int requiredSize; private Collection buckets; protected Map bucketMap; @@ -170,8 +171,9 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation implemen InternalGeoHashGrid() { } // for serialization - public InternalGeoHashGrid(String name, int requiredSize, Collection buckets, Map metaData) { - super(name, metaData); + public InternalGeoHashGrid(String name, int requiredSize, Collection buckets, List reducers, + Map metaData) { + super(name, reducers, metaData); this.requiredSize = requiredSize; this.buckets = buckets; } @@ -181,6 +183,16 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation implemen return TYPE; } + @Override + public InternalGeoHashGrid create(List buckets) { + return new InternalGeoHashGrid(this.name, this.requiredSize, buckets, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.geohashAsLong, prototype.docCount, aggregations); + } + @Override public List getBuckets() { Object o = buckets; @@ -188,7 +200,7 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation implemen } @Override - public InternalGeoHashGrid reduce(List aggregations, ReduceContext reduceContext) { + public InternalGeoHashGrid doReduce(List aggregations, ReduceContext reduceContext) { LongObjectPagedHashMap> buckets = null; for (InternalAggregation aggregation : aggregations) { @@ -217,7 +229,7 @@ public class InternalGeoHashGrid extends InternalMultiBucketAggregation implemen for (int i = ordered.size() - 1; i >= 0; i--) { list[i] = ordered.pop(); } - return new InternalGeoHashGrid(getName(), requiredSize, Arrays.asList(list), getMetaData()); + return new InternalGeoHashGrid(getName(), requiredSize, Arrays.asList(list), reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java index da2f27fe852..1fa3e95d876 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregator.java @@ -27,9 +27,11 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -37,8 +39,9 @@ import java.util.Map; */ public class GlobalAggregator extends SingleBucketAggregator { - public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, Map metaData) throws IOException { - super(name, subFactories, aggregationContext, null, metaData); + public GlobalAggregator(String name, AggregatorFactories subFactories, AggregationContext aggregationContext, List reducers, + Map metaData) throws IOException { + super(name, subFactories, aggregationContext, null, reducers, metaData); } @Override @@ -56,7 +59,8 @@ public class GlobalAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0 : "global aggregator can only be a top level aggregator"; - return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalGlobal(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override @@ -71,7 +75,8 @@ public class GlobalAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { 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"); @@ -79,7 +84,7 @@ public class GlobalAggregator extends SingleBucketAggregator { if (collectsFromSingleBucket == false) { throw new IllegalStateException(); } - return new GlobalAggregator(name, factories, context, metaData); + return new GlobalAggregator(name, factories, context, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobal.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobal.java index 6e317f26952..157d2c5c7f9 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobal.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/global/InternalGlobal.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,8 @@ public class InternalGlobal extends InternalSingleBucketAggregation implements G InternalGlobal() {} // for serialization - InternalGlobal(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, docCount, aggregations, metaData); + InternalGlobal(String name, long docCount, InternalAggregations aggregations, List reducers, Map metaData) { + super(name, docCount, aggregations, reducers, metaData); } @Override @@ -60,6 +62,6 @@ public class InternalGlobal extends InternalSingleBucketAggregation implements G @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalGlobal(name, docCount, subAggregations, getMetaData()); + return new InternalGlobal(name, docCount, subAggregations, reducers(), getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java index 9d08d2ce81a..6f316d901db 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramParser.java @@ -108,13 +108,15 @@ public class DateHistogramParser implements Aggregator.Parser { } else if (INTERVAL.match(currentFieldName)) { interval = parser.text(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_BOOLEAN) { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_NUMBER) { if ("min_doc_count".equals(currentFieldName) || "minDocCount".equals(currentFieldName)) { @@ -122,7 +124,8 @@ public class DateHistogramParser implements Aggregator.Parser { } else if ("time_zone".equals(currentFieldName) || "timeZone".equals(currentFieldName)) { timeZone = DateTimeZone.forOffsetHours(parser.intValue()); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_OBJECT) { if ("order".equals(currentFieldName)) { @@ -147,7 +150,8 @@ public class DateHistogramParser implements Aggregator.Parser { } else if ("max".equals(currentFieldName)) { extendedBounds.maxAsStr = parser.text(); } else { - throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + + aggregationName + "]: [" + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_NUMBER) { if ("min".equals(currentFieldName)) { @@ -155,23 +159,28 @@ public class DateHistogramParser implements Aggregator.Parser { } else if ("max".equals(currentFieldName)) { extendedBounds.max = parser.longValue(); } else { - throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + + aggregationName + "]: [" + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (interval == null) { - throw new SearchParseException(context, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]"); + throw new SearchParseException(context, + "Missing required field [interval] for histogram aggregation [" + aggregationName + "]", parser.getTokenLocation()); } TimeZoneRounding.Builder tzRoundingBuilder; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java index b041ef34fdb..c703058b699 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBounds.java @@ -56,7 +56,7 @@ public class ExtendedBounds { } if (min != null && max != null && min.compareTo(max) > 0) { throw new SearchParseException(context, "[extended_bounds.min][" + min + "] cannot be greater than " + - "[extended_bounds.max][" + max + "] for histogram aggregation [" + aggName + "]"); + "[extended_bounds.max][" + max + "] for histogram aggregation [" + aggName + "]", null); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java index a39a488a615..44342366b3f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregator.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -56,15 +57,14 @@ public class HistogramAggregator extends BucketsAggregator { private final InternalHistogram.Factory histogramFactory; private final LongHash bucketOrds; - private SortedNumericDocValues values; public HistogramAggregator(String name, AggregatorFactories factories, Rounding rounding, InternalOrder order, boolean keyed, long minDocCount, @Nullable ExtendedBounds extendedBounds, @Nullable ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - InternalHistogram.Factory histogramFactory, - AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + InternalHistogram.Factory histogramFactory, AggregationContext aggregationContext, + Aggregator parent, List reducers, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + super(name, factories, aggregationContext, parent, reducers, metaData); this.rounding = rounding; this.order = order; this.keyed = keyed; @@ -130,13 +130,14 @@ public class HistogramAggregator extends BucketsAggregator { // value source will be null for unmapped fields InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; - return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, metaData()); + return histogramFactory.create(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { InternalHistogram.EmptyBucketInfo emptyBucketInfo = minDocCount == 0 ? new InternalHistogram.EmptyBucketInfo(rounding, buildEmptySubAggregations(), extendedBounds) : null; - return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, metaData()); + return histogramFactory.create(name, Collections.emptyList(), order, minDocCount, emptyBucketInfo, formatter, keyed, reducers(), + metaData()); } @Override @@ -166,13 +167,20 @@ public class HistogramAggregator extends BucketsAggregator { this.histogramFactory = histogramFactory; } - @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), histogramFactory, aggregationContext, parent, metaData); + public long minDocCount() { + return minDocCount; } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, null, null, config.formatter(), + histogramFactory, aggregationContext, parent, reducers, metaData); + } + + @Override + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, aggregationContext, parent); } @@ -185,7 +193,8 @@ public class HistogramAggregator extends BucketsAggregator { extendedBounds.processAndValidate(name, aggregationContext.searchContext(), config.parser()); roundedBounds = extendedBounds.round(rounding); } - return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, config.formatter(), histogramFactory, aggregationContext, parent, metaData); + return new HistogramAggregator(name, factories, rounding, order, keyed, minDocCount, roundedBounds, valuesSource, + config.formatter(), histogramFactory, aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java index f316237d734..c9c885be3f5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramParser.java @@ -75,7 +75,8 @@ public class HistogramParser implements Aggregator.Parser { } else if ("offset".equals(currentFieldName)) { offset = parser.longValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_OBJECT) { if ("order".equals(currentFieldName)) { @@ -86,7 +87,8 @@ public class HistogramParser implements Aggregator.Parser { String dir = parser.text(); boolean asc = "asc".equals(dir); if (!asc && !"desc".equals(dir)) { - throw new SearchParseException(context, "Unknown order direction [" + dir + "] in aggregation [" + aggregationName + "]. Should be either [asc] or [desc]"); + throw new SearchParseException(context, "Unknown order direction [" + dir + "] in aggregation [" + + aggregationName + "]. Should be either [asc] or [desc]", parser.getTokenLocation()); } order = resolveOrder(currentFieldName, asc); } @@ -102,21 +104,25 @@ public class HistogramParser implements Aggregator.Parser { } else if ("max".equals(currentFieldName)) { extendedBounds.max = parser.longValue(true); } else { - throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown extended_bounds key for a " + token + " in aggregation [" + + aggregationName + "]: [" + currentFieldName + "].", parser.getTokenLocation()); } } } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in aggregation [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in aggregation [" + aggregationName + "].", + parser.getTokenLocation()); } } if (interval < 1) { - throw new SearchParseException(context, "Missing required field [interval] for histogram aggregation [" + aggregationName + "]"); + throw new SearchParseException(context, + "Missing required field [interval] for histogram aggregation [" + aggregationName + "]", parser.getTokenLocation()); } Rounding rounding = new Rounding.Interval(interval); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java index 63cab59ad6b..a82a089066b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalDateHistogram.java @@ -19,16 +19,13 @@ package org.elasticsearch.search.aggregations.bucket.histogram; import org.elasticsearch.common.Nullable; +import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.EmptyBucketInfo; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import java.util.List; -import java.util.Map; - /** * */ @@ -74,14 +71,20 @@ public class InternalDateHistogram { } @Override - public InternalHistogram create(String name, List buckets, InternalOrder order, - long minDocCount, EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - return new InternalHistogram(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, this, metaData); + public InternalDateHistogram.Bucket createBucket(InternalAggregations aggregations, InternalDateHistogram.Bucket prototype) { + return new Bucket(prototype.key, prototype.docCount, aggregations, prototype.getKeyed(), prototype.formatter, this); } @Override - public InternalDateHistogram.Bucket createBucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) { - return new Bucket(key, docCount, aggregations, keyed, formatter, this); + public InternalDateHistogram.Bucket createBucket(Object key, long docCount, InternalAggregations aggregations, boolean keyed, + @Nullable ValueFormatter formatter) { + if (key instanceof Number) { + return new Bucket(((Number) key).longValue(), docCount, aggregations, keyed, formatter, this); + } else if (key instanceof DateTime) { + return new Bucket(((DateTime) key).getMillis(), docCount, aggregations, keyed, formatter, this); + } else { + throw new AggregationExecutionException("Expected key of type Number or DateTime but got [" + key + "]"); + } } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java index 518240cd1cf..1934676ac40 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/InternalHistogram.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.rounding.Rounding; import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -36,6 +37,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -49,7 +51,8 @@ import java.util.Map; /** * TODO should be renamed to InternalNumericHistogram (see comment on {@link Histogram})? */ -public class InternalHistogram extends InternalMultiBucketAggregation implements Histogram { +public class InternalHistogram extends InternalMultiBucketAggregation implements + Histogram { final static Type TYPE = new Type("histogram", "histo"); @@ -184,6 +187,14 @@ public class InternalHistogram extends Inter out.writeVLong(docCount); aggregations.writeTo(out); } + + public ValueFormatter getFormatter() { + return formatter; + } + + public boolean getKeyed() { + return keyed; + } } static class EmptyBucketInfo { @@ -222,7 +233,7 @@ public class InternalHistogram extends Inter } - static class Factory { + public static class Factory { protected Factory() { } @@ -232,12 +243,27 @@ public class InternalHistogram extends Inter } public InternalHistogram create(String name, List buckets, InternalOrder order, long minDocCount, - EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, this, metaData); + EmptyBucketInfo emptyBucketInfo, @Nullable ValueFormatter formatter, boolean keyed, List reducers, + Map metaData) { + return new InternalHistogram<>(name, buckets, order, minDocCount, emptyBucketInfo, formatter, keyed, this, reducers, metaData); } - public B createBucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) { - return (B) new Bucket(key, docCount, keyed, formatter, this, aggregations); + public InternalHistogram create(List buckets, InternalHistogram prototype) { + return new InternalHistogram<>(prototype.name, buckets, prototype.order, prototype.minDocCount, prototype.emptyBucketInfo, + prototype.formatter, prototype.keyed, this, prototype.reducers(), prototype.metaData); + } + + public B createBucket(InternalAggregations aggregations, B prototype) { + return (B) new Bucket(prototype.key, prototype.docCount, prototype.getKeyed(), prototype.formatter, this, aggregations); + } + + public B createBucket(Object key, long docCount, InternalAggregations aggregations, boolean keyed, + @Nullable ValueFormatter formatter) { + if (key instanceof Number) { + return (B) new Bucket(((Number) key).longValue(), docCount, keyed, formatter, this, aggregations); + } else { + throw new AggregationExecutionException("Expected key of type Number but got [" + key + "]"); + } } protected B createEmptyBucket(boolean keyed, @Nullable ValueFormatter formatter) { @@ -258,8 +284,8 @@ public class InternalHistogram extends Inter InternalHistogram(String name, List buckets, InternalOrder order, long minDocCount, EmptyBucketInfo emptyBucketInfo, - @Nullable ValueFormatter formatter, boolean keyed, Factory factory, Map metaData) { - super(name, metaData); + @Nullable ValueFormatter formatter, boolean keyed, Factory factory, List reducers, Map metaData) { + super(name, reducers, metaData); this.buckets = buckets; this.order = order; assert (minDocCount == 0) == (emptyBucketInfo != null); @@ -280,10 +306,20 @@ public class InternalHistogram extends Inter return buckets; } - protected Factory getFactory() { + public Factory getFactory() { return factory; } + @Override + public InternalHistogram create(List buckets) { + return getFactory().create(buckets, this); + } + + @Override + public B createBucket(InternalAggregations aggregations, B prototype) { + return getFactory().createBucket(aggregations, prototype); + } + private static class IteratorAndCurrent { private final Iterator iterator; @@ -410,7 +446,7 @@ public class InternalHistogram extends Inter } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { List reducedBuckets = reduceBuckets(aggregations, reduceContext); // adding empty buckets if needed @@ -430,7 +466,8 @@ public class InternalHistogram extends Inter CollectionUtil.introSort(reducedBuckets, order.comparator()); } - return getFactory().create(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, formatter, keyed, getMetaData()); + return getFactory().create(getName(), reducedBuckets, order, minDocCount, emptyBucketInfo, formatter, keyed, reducers(), + getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissing.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissing.java index d314e44e901..0245f117835 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissing.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/InternalMissing.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -50,8 +52,8 @@ public class InternalMissing extends InternalSingleBucketAggregation implements InternalMissing() { } - InternalMissing(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, docCount, aggregations, metaData); + InternalMissing(String name, long docCount, InternalAggregations aggregations, List reducers, Map metaData) { + super(name, docCount, aggregations, reducers, metaData); } @Override @@ -61,6 +63,6 @@ public class InternalMissing extends InternalSingleBucketAggregation implements @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalMissing(name, docCount, subAggregations, getMetaData()); + return new InternalMissing(name, docCount, subAggregations, reducers(), getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java index 1b65bde9904..b60c8510238 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregator.java @@ -26,12 +26,14 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -42,8 +44,9 @@ public class MissingAggregator extends SingleBucketAggregator { private final ValuesSource valuesSource; public MissingAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, - AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; } @@ -69,12 +72,13 @@ public class MissingAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalMissing(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalMissing(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalMissing(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory { @@ -84,13 +88,15 @@ public class MissingAggregator extends SingleBucketAggregator { } @Override - protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new MissingAggregator(name, factories, null, aggregationContext, parent, metaData); + protected MissingAggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new MissingAggregator(name, factories, null, aggregationContext, parent, reducers, metaData); } @Override - protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, metaData); + protected MissingAggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new MissingAggregator(name, factories, valuesSource, aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java index b37de4c743c..6ecdc129dd0 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingParser.java @@ -52,7 +52,8 @@ public class MissingParser implements Aggregator.Parser { } else if (vsParser.token(currentFieldName, token, parser)) { continue; } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNested.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNested.java index 8b434a3fd24..86ad26edab3 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNested.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalNested.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,9 @@ public class InternalNested extends InternalSingleBucketAggregation implements N public InternalNested() { } - public InternalNested(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, docCount, aggregations, metaData); + public InternalNested(String name, long docCount, InternalAggregations aggregations, List reducers, + Map metaData) { + super(name, docCount, aggregations, reducers, metaData); } @Override @@ -60,6 +63,6 @@ public class InternalNested extends InternalSingleBucketAggregation implements N @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalNested(name, docCount, subAggregations, getMetaData()); + return new InternalNested(name, docCount, subAggregations, reducers(), getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNested.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNested.java index eec7345d317..6dfaad42b03 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNested.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/InternalReverseNested.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,9 @@ public class InternalReverseNested extends InternalSingleBucketAggregation imple public InternalReverseNested() { } - public InternalReverseNested(String name, long docCount, InternalAggregations aggregations, Map metaData) { - super(name, docCount, aggregations, metaData); + public InternalReverseNested(String name, long docCount, InternalAggregations aggregations, List reducers, + Map metaData) { + super(name, docCount, aggregations, reducers, metaData); } @Override @@ -60,6 +63,6 @@ public class InternalReverseNested extends InternalSingleBucketAggregation imple @Override protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalReverseNested(name, docCount, subAggregations, getMetaData()); + return new InternalReverseNested(name, docCount, subAggregations, reducers(), getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java index 7795cfe197d..0acf04210a1 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregator.java @@ -39,9 +39,11 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -55,8 +57,8 @@ public class NestedAggregator extends SingleBucketAggregator { private DocIdSetIterator childDocs; private BitSet parentDocs; - public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, Map metaData, QueryCachingPolicy filterCachingPolicy) throws IOException { - super(name, factories, aggregationContext, parentAggregator, metaData); + public NestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parentAggregator, List reducers, Map metaData, QueryCachingPolicy filterCachingPolicy) throws IOException { + super(name, factories, aggregationContext, parentAggregator, reducers, metaData); childFilter = aggregationContext.searchContext().filterCache().cache(objectMapper.nestedTypeFilter(), null, filterCachingPolicy); } @@ -120,12 +122,13 @@ public class NestedAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalNested(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalNested(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } private static Filter findClosestNestedPath(Aggregator parent) { @@ -151,33 +154,35 @@ public class NestedAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, context, parent); } MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path); if (mapper == null) { - return new Unmapped(name, context, parent, metaData); + return new Unmapped(name, context, parent, reducers, metaData); } ObjectMapper objectMapper = mapper.mapper(); if (objectMapper == null) { - return new Unmapped(name, context, parent, metaData); + return new Unmapped(name, context, parent, reducers, metaData); } if (!objectMapper.nested().isNested()) { throw new AggregationExecutionException("[nested] nested path [" + path + "] is not nested"); } - return new NestedAggregator(name, factories, objectMapper, context, parent, metaData, queryCachingPolicy); + return new NestedAggregator(name, factories, objectMapper, context, parent, reducers, metaData, queryCachingPolicy); } private final static class Unmapped extends NonCollectingAggregator { - public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + public Unmapped(String name, AggregationContext context, Aggregator parent, List reducers, Map metaData) + throws IOException { + super(name, context, parent, reducers, metaData); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalNested(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalNested(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedParser.java index 61044fb4a28..56da7f51b17 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedParser.java @@ -49,16 +49,19 @@ public class NestedParser implements Aggregator.Parser { if ("path".equals(currentFieldName)) { path = parser.text(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (path == null) { // "field" doesn't exist, so we fall back to the context of the ancestors - throw new SearchParseException(context, "Missing [path] field for nested aggregation [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [path] field for nested aggregation [" + aggregationName + "]", + parser.getTokenLocation()); } return new NestedAggregator.Factory(aggregationName, path, context.queryParserService().autoFilterCachePolicy()); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java index 7466bec3b5b..a81aae9eef6 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregator.java @@ -40,9 +40,11 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -52,8 +54,10 @@ public class ReverseNestedAggregator extends SingleBucketAggregator { private final BitDocIdSetFilter parentFilter; - public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + public ReverseNestedAggregator(String name, AggregatorFactories factories, ObjectMapper objectMapper, + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData) + throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); if (objectMapper == null) { parentFilter = context.searchContext().bitsetFilterCache().getBitDocIdSetFilter(Queries.newNonNestedFilter()); } else { @@ -105,12 +109,13 @@ public class ReverseNestedAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { - return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalReverseNested(name, bucketDocCount(owningBucketOrdinal), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalReverseNested(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } Filter getParentFilter() { @@ -127,22 +132,24 @@ public class ReverseNestedAggregator extends SingleBucketAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { // Early validation NestedAggregator closestNestedAggregator = findClosestNestedAggregator(parent); if (closestNestedAggregator == null) { - throw new SearchParseException(context.searchContext(), "Reverse nested aggregation [" + name + "] can only be used inside a [nested] aggregation"); + throw new SearchParseException(context.searchContext(), "Reverse nested aggregation [" + name + + "] can only be used inside a [nested] aggregation", null); } final ObjectMapper objectMapper; if (path != null) { MapperService.SmartNameObjectMapper mapper = context.searchContext().smartNameObjectMapper(path); if (mapper == null) { - return new Unmapped(name, context, parent, metaData); + return new Unmapped(name, context, parent, reducers, metaData); } objectMapper = mapper.mapper(); if (objectMapper == null) { - return new Unmapped(name, context, parent, metaData); + return new Unmapped(name, context, parent, reducers, metaData); } if (!objectMapper.nested().isNested()) { throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested"); @@ -150,18 +157,19 @@ public class ReverseNestedAggregator extends SingleBucketAggregator { } else { objectMapper = null; } - return new ReverseNestedAggregator(name, factories, objectMapper, context, parent, metaData); + return new ReverseNestedAggregator(name, factories, objectMapper, context, parent, reducers, metaData); } private final static class Unmapped extends NonCollectingAggregator { - public Unmapped(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + public Unmapped(String name, AggregationContext context, Aggregator parent, List reducers, Map metaData) + throws IOException { + super(name, context, parent, reducers, metaData); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalReverseNested(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalReverseNested(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedParser.java index 0ab7cefc9e3..80ab9f5eebd 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedParser.java @@ -49,10 +49,12 @@ public class ReverseNestedParser implements Aggregator.Parser { if ("path".equals(currentFieldName)) { path = parser.text(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java index 5f8b7baa2ac..db0ccee33e5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalRange.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -42,7 +43,8 @@ import java.util.Map; /** * */ -public class InternalRange extends InternalMultiBucketAggregation implements Range { +public class InternalRange> extends InternalMultiBucketAggregation + implements Range { static final Factory FACTORY = new Factory(); @@ -123,6 +125,14 @@ public class InternalRange extends InternalMulti return to; } + public boolean getKeyed() { + return keyed; + } + + public ValueFormatter getFormatter() { + return formatter; + } + @Override public String getFromAsString() { if (Double.isInfinite(from)) { @@ -215,31 +225,44 @@ public class InternalRange extends InternalMulti } } - public static class Factory> { + public static class Factory> { public String type() { return TYPE.name(); } - public R create(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - return (R) new InternalRange<>(name, ranges, formatter, keyed, metaData); + public R create(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, List reducers, + Map metaData) { + return (R) new InternalRange<>(name, ranges, formatter, keyed, reducers, metaData); } - - public B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) { + public B createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, boolean keyed, + @Nullable ValueFormatter formatter) { return (B) new Bucket(key, from, to, docCount, aggregations, keyed, formatter); } + + public R create(List ranges, R prototype) { + return (R) new InternalRange<>(prototype.name, ranges, prototype.formatter, prototype.keyed, prototype.reducers(), + prototype.metaData); + } + + public B createBucket(InternalAggregations aggregations, B prototype) { + return (B) new Bucket(prototype.getKey(), prototype.from, prototype.to, prototype.getDocCount(), aggregations, prototype.keyed, + prototype.formatter); + } } private List ranges; private Map rangeMap; - private @Nullable ValueFormatter formatter; - private boolean keyed; + @Nullable + protected ValueFormatter formatter; + protected boolean keyed; public InternalRange() {} // for serialization - public InternalRange(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - super(name, metaData); + public InternalRange(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, List reducers, + Map metaData) { + super(name, reducers, metaData); this.ranges = ranges; this.formatter = formatter; this.keyed = keyed; @@ -255,19 +278,29 @@ public class InternalRange extends InternalMulti return ranges; } - protected Factory getFactory() { + public Factory getFactory() { return FACTORY; } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public R create(List buckets) { + return getFactory().create(buckets, (R) this); + } + + @Override + public B createBucket(InternalAggregations aggregations, B prototype) { + return getFactory().createBucket(aggregations, prototype); + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { @SuppressWarnings("unchecked") List[] rangeList = new List[ranges.size()]; for (int i = 0; i < rangeList.length; ++i) { rangeList[i] = new ArrayList(); } for (InternalAggregation aggregation : aggregations) { - InternalRange ranges = (InternalRange) aggregation; + InternalRange ranges = (InternalRange) aggregation; int i = 0; for (Bucket range : ranges.ranges) { rangeList[i++].add(range); @@ -278,7 +311,7 @@ public class InternalRange extends InternalMulti for (int i = 0; i < this.ranges.size(); ++i) { ranges.add((B) rangeList[i].get(0).reduce(rangeList[i], reduceContext)); } - return getFactory().create(name, ranges, formatter, keyed, getMetaData()); + return getFactory().create(name, ranges, formatter, keyed, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java index 47011b8dc49..d6d961a5998 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregator.java @@ -33,6 +33,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -104,10 +105,10 @@ public class RangeAggregator extends BucketsAggregator { List ranges, boolean keyed, AggregationContext aggregationContext, - Aggregator parent, + Aggregator parent, List reducers, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + super(name, factories, aggregationContext, parent, reducers, metaData); assert valuesSource != null; this.valuesSource = valuesSource; this.formatter = format != null ? format.formatter() : null; @@ -149,54 +150,54 @@ public class RangeAggregator extends BucketsAggregator { } } - private int collect(int doc, double value, long owningBucketOrdinal, int lowBound) throws IOException { - int lo = lowBound, hi = ranges.length - 1; // all candidates are between these indexes - int mid = (lo + hi) >>> 1; - while (lo <= hi) { - if (value < ranges[mid].from) { - hi = mid - 1; - } else if (value >= maxTo[mid]) { - lo = mid + 1; - } else { - break; - } - mid = (lo + hi) >>> 1; - } - if (lo > hi) return lo; // no potential candidate - - // binary search the lower bound - int startLo = lo, startHi = mid; - while (startLo <= startHi) { - final int startMid = (startLo + startHi) >>> 1; - if (value >= maxTo[startMid]) { - startLo = startMid + 1; - } else { - startHi = startMid - 1; - } - } - - // binary search the upper bound - int endLo = mid, endHi = hi; - while (endLo <= endHi) { - final int endMid = (endLo + endHi) >>> 1; - if (value < ranges[endMid].from) { - endHi = endMid - 1; - } else { - endLo = endMid + 1; - } - } - - assert startLo == lowBound || value >= maxTo[startLo - 1]; - assert endHi == ranges.length - 1 || value < ranges[endHi + 1].from; - - for (int i = startLo; i <= endHi; ++i) { - if (ranges[i].matches(value)) { - collectBucket(sub, doc, subBucketOrdinal(owningBucketOrdinal, i)); - } - } - - return endHi + 1; + private int collect(int doc, double value, long owningBucketOrdinal, int lowBound) throws IOException { + int lo = lowBound, hi = ranges.length - 1; // all candidates are between these indexes + int mid = (lo + hi) >>> 1; + while (lo <= hi) { + if (value < ranges[mid].from) { + hi = mid - 1; + } else if (value >= maxTo[mid]) { + lo = mid + 1; + } else { + break; } + mid = (lo + hi) >>> 1; + } + if (lo > hi) return lo; // no potential candidate + + // binary search the lower bound + int startLo = lo, startHi = mid; + while (startLo <= startHi) { + final int startMid = (startLo + startHi) >>> 1; + if (value >= maxTo[startMid]) { + startLo = startMid + 1; + } else { + startHi = startMid - 1; + } + } + + // binary search the upper bound + int endLo = mid, endHi = hi; + while (endLo <= endHi) { + final int endMid = (endLo + endHi) >>> 1; + if (value < ranges[endMid].from) { + endHi = endMid - 1; + } else { + endLo = endMid + 1; + } + } + + assert startLo == lowBound || value >= maxTo[startLo - 1]; + assert endHi == ranges.length - 1 || value < ranges[endHi + 1].from; + + for (int i = startLo; i <= endHi; ++i) { + if (ranges[i].matches(value)) { + collectBucket(sub, doc, subBucketOrdinal(owningBucketOrdinal, i)); + } + } + + return endHi + 1; + } }; } @@ -215,7 +216,7 @@ public class RangeAggregator extends BucketsAggregator { buckets.add(bucket); } // value source can be null in the case of unmapped fields - return rangeFactory.create(name, buckets, formatter, keyed, metaData()); + return rangeFactory.create(name, buckets, formatter, keyed, reducers(), metaData()); } @Override @@ -229,7 +230,7 @@ public class RangeAggregator extends BucketsAggregator { buckets.add(bucket); } // value source can be null in the case of unmapped fields - return rangeFactory.create(name, buckets, formatter, keyed, metaData()); + return rangeFactory.create(name, buckets, formatter, keyed, reducers(), metaData()); } private static final void sortRanges(final Range[] ranges) { @@ -266,10 +267,10 @@ public class RangeAggregator extends BucketsAggregator { ValueFormat format, AggregationContext context, Aggregator parent, - InternalRange.Factory factory, + InternalRange.Factory factory, List reducers, Map metaData) throws IOException { - super(name, context, parent, metaData); + super(name, context, parent, reducers, metaData); this.ranges = ranges; ValueParser parser = format != null ? format.parser() : ValueParser.RAW; for (Range range : this.ranges) { @@ -287,7 +288,7 @@ public class RangeAggregator extends BucketsAggregator { for (RangeAggregator.Range range : ranges) { buckets.add(factory.createBucket(range.key, range.from, range.to, 0, subAggs, keyed, formatter)); } - return factory.create(name, buckets, formatter, keyed, metaData()); + return factory.create(name, buckets, formatter, keyed, reducers(), metaData()); } } @@ -305,13 +306,15 @@ public class RangeAggregator extends BucketsAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new Unmapped(name, ranges, keyed, config.format(), aggregationContext, parent, rangeFactory, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new RangeAggregator(name, factories, valuesSource, config.format(), rangeFactory, ranges, keyed, aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeParser.java index dbe05df0998..e30b84bf1de 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeParser.java @@ -89,21 +89,25 @@ public class RangeParser implements Aggregator.Parser { ranges.add(new RangeAggregator.Range(key, from, fromAsStr, to, toAsStr)); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_BOOLEAN) { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (ranges == null) { - throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]", + parser.getTokenLocation()); } return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalRange.FACTORY, ranges, keyed); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeParser.java index 06dcba53b95..940e20a79a8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeParser.java @@ -79,7 +79,8 @@ public class DateRangeParser implements Aggregator.Parser { } else if ("to".equals(toOrFromOrKey)) { to = parser.doubleValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + + "]: [" + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_STRING) { if ("from".equals(toOrFromOrKey)) { @@ -89,7 +90,7 @@ public class DateRangeParser implements Aggregator.Parser { } else if ("key".equals(toOrFromOrKey)) { key = parser.text(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "].", parser.getTokenLocation()); } } } @@ -100,15 +101,18 @@ public class DateRangeParser implements Aggregator.Parser { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (ranges == null) { - throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]", + parser.getTokenLocation()); } return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalDateRange.FACTORY, ranges, keyed); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRange.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRange.java index 785df76e824..6444f53e527 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRange.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/InternalDateRange.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; import org.elasticsearch.search.aggregations.bucket.range.InternalRange; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -37,7 +38,7 @@ import java.util.Map; /** * */ -public class InternalDateRange extends InternalRange { +public class InternalDateRange extends InternalRange { public final static Type TYPE = new Type("date_range", "drange"); @@ -112,7 +113,7 @@ public class InternalDateRange extends InternalRange { } } - private static class Factory extends InternalRange.Factory { + public static class Factory extends InternalRange.Factory { @Override public String type() { @@ -120,20 +121,34 @@ public class InternalDateRange extends InternalRange { } @Override - public InternalDateRange create(String name, List ranges, ValueFormatter formatter, boolean keyed, Map metaData) { - return new InternalDateRange(name, ranges, formatter, keyed, metaData); + public InternalDateRange create(String name, List ranges, ValueFormatter formatter, boolean keyed, + List reducers, Map metaData) { + return new InternalDateRange(name, ranges, formatter, keyed, reducers, metaData); + } + + @Override + public InternalDateRange create(List ranges, InternalDateRange prototype) { + return new InternalDateRange(prototype.name, ranges, prototype.formatter, prototype.keyed, prototype.reducers(), + prototype.metaData); } @Override public Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, boolean keyed, ValueFormatter formatter) { return new Bucket(key, from, to, docCount, aggregations, keyed, formatter); } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.getKey(), ((Number) prototype.getFrom()).doubleValue(), ((Number) prototype.getTo()).doubleValue(), + prototype.getDocCount(), aggregations, prototype.getKeyed(), prototype.getFormatter()); + } } InternalDateRange() {} // for serialization - InternalDateRange(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - super(name, ranges, formatter, keyed, metaData); + InternalDateRange(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, + List reducers, Map metaData) { + super(name, ranges, formatter, keyed, reducers, metaData); } @Override @@ -142,7 +157,7 @@ public class InternalDateRange extends InternalRange { } @Override - protected InternalRange.Factory getFactory() { + public InternalRange.Factory getFactory() { return FACTORY; } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java index 713b94595f5..3eee3ceb704 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceParser.java @@ -35,6 +35,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.bucket.range.InternalRange; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator; import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.GeoPointParser; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -98,13 +99,15 @@ public class GeoDistanceParser implements Aggregator.Parser { } else if ("distance_type".equals(currentFieldName) || "distanceType".equals(currentFieldName)) { distanceType = GeoDistance.fromString(parser.text()); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_BOOLEAN) { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_ARRAY) { if ("ranges".equals(currentFieldName)) { @@ -138,20 +141,24 @@ public class GeoDistanceParser implements Aggregator.Parser { ranges.add(new RangeAggregator.Range(key(key, from, to), from, fromAsStr, to, toAsStr)); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (ranges == null) { - throw new SearchParseException(context, "Missing [ranges] in geo_distance aggregator [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [ranges] in geo_distance aggregator [" + aggregationName + "]", + parser.getTokenLocation()); } GeoPoint origin = geoPointParser.geoPoint(); if (origin == null) { - throw new SearchParseException(context, "Missing [origin] in geo_distance aggregator [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [origin] in geo_distance aggregator [" + aggregationName + "]", + parser.getTokenLocation()); } return new GeoDistanceFactory(aggregationName, vsParser.config(), InternalGeoDistance.FACTORY, origin, unit, distanceType, ranges, keyed); @@ -179,14 +186,18 @@ public class GeoDistanceParser implements Aggregator.Parser { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new Unmapped(name, ranges, keyed, null, aggregationContext, parent, rangeFactory, reducers, metaData); } @Override - protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, + Aggregator parent, boolean collectsFromSingleBucket, List reducers, Map metaData) + throws IOException { DistanceSource distanceSource = new DistanceSource(valuesSource, distanceType, origin, unit); - return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, metaData); + return new RangeAggregator(name, factories, distanceSource, null, rangeFactory, ranges, keyed, aggregationContext, parent, + reducers, metaData); } private static class DistanceSource extends ValuesSource.Numeric { diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistance.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistance.java index da2c41d5233..b271c3336e0 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistance.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/InternalGeoDistance.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; import org.elasticsearch.search.aggregations.bucket.range.InternalRange; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; @@ -35,7 +36,7 @@ import java.util.Map; /** * */ -public class InternalGeoDistance extends InternalRange { +public class InternalGeoDistance extends InternalRange { public static final Type TYPE = new Type("geo_distance", "gdist"); @@ -100,7 +101,7 @@ public class InternalGeoDistance extends InternalRange { + public static class Factory extends InternalRange.Factory { @Override public String type() { @@ -108,20 +109,34 @@ public class InternalGeoDistance extends InternalRange ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - return new InternalGeoDistance(name, ranges, formatter, keyed, metaData); + public InternalGeoDistance create(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, + List reducers, Map metaData) { + return new InternalGeoDistance(name, ranges, formatter, keyed, reducers, metaData); + } + + @Override + public InternalGeoDistance create(List ranges, InternalGeoDistance prototype) { + return new InternalGeoDistance(prototype.name, ranges, prototype.formatter, prototype.keyed, prototype.reducers(), + prototype.metaData); } @Override public Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) { return new Bucket(key, from, to, docCount, aggregations, keyed, formatter); } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.getKey(), ((Number) prototype.getFrom()).doubleValue(), ((Number) prototype.getTo()).doubleValue(), + prototype.getDocCount(), aggregations, prototype.getKeyed(), prototype.getFormatter()); + } } InternalGeoDistance() {} // for serialization - public InternalGeoDistance(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - super(name, ranges, formatter, keyed, metaData); + public InternalGeoDistance(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, List reducers, + Map metaData) { + super(name, ranges, formatter, keyed, reducers, metaData); } @Override @@ -130,7 +145,7 @@ public class InternalGeoDistance extends InternalRange getFactory() { + public InternalRange.Factory getFactory() { return FACTORY; } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/InternalIPv4Range.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/InternalIPv4Range.java index 9b608aa42d4..96668e67c69 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/InternalIPv4Range.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/InternalIPv4Range.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; import org.elasticsearch.search.aggregations.bucket.range.InternalRange; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; @@ -35,7 +36,7 @@ import java.util.Map; /** * */ -public class InternalIPv4Range extends InternalRange { +public class InternalIPv4Range extends InternalRange { public static final long MAX_IP = 4294967296l; @@ -109,7 +110,7 @@ public class InternalIPv4Range extends InternalRange { } } - private static class Factory extends InternalRange.Factory { + public static class Factory extends InternalRange.Factory { @Override public String type() { @@ -117,20 +118,33 @@ public class InternalIPv4Range extends InternalRange { } @Override - public InternalIPv4Range create(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, Map metaData) { - return new InternalIPv4Range(name, ranges, keyed, metaData); + public InternalIPv4Range create(String name, List ranges, @Nullable ValueFormatter formatter, boolean keyed, + List reducers, Map metaData) { + return new InternalIPv4Range(name, ranges, keyed, reducers, metaData); + } + + @Override + public InternalIPv4Range create(List ranges, InternalIPv4Range prototype) { + return new InternalIPv4Range(prototype.name, ranges, prototype.keyed, prototype.reducers(), prototype.metaData); } @Override public Bucket createBucket(String key, double from, double to, long docCount, InternalAggregations aggregations, boolean keyed, @Nullable ValueFormatter formatter) { return new Bucket(key, from, to, docCount, aggregations, keyed); } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.getKey(), ((Number) prototype.getFrom()).doubleValue(), ((Number) prototype.getTo()).doubleValue(), + prototype.getDocCount(), aggregations, prototype.getKeyed()); + } } public InternalIPv4Range() {} // for serialization - public InternalIPv4Range(String name, List ranges, boolean keyed, Map metaData) { - super(name, ranges, ValueFormatter.IPv4, keyed, metaData); + public InternalIPv4Range(String name, List ranges, boolean keyed, List reducers, + Map metaData) { + super(name, ranges, ValueFormatter.IPv4, keyed, reducers, metaData); } @Override @@ -139,7 +153,7 @@ public class InternalIPv4Range extends InternalRange { } @Override - protected InternalRange.Factory getFactory() { + public InternalRange.Factory getFactory() { return FACTORY; } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/IpRangeParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/IpRangeParser.java index 49c9c90b16e..37891f6f239 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/IpRangeParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ipv4/IpRangeParser.java @@ -99,21 +99,25 @@ public class IpRangeParser implements Aggregator.Parser { ranges.add(range); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_BOOLEAN) { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (ranges == null) { - throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]"); + throw new SearchParseException(context, "Missing [ranges] in ranges aggregator [" + aggregationName + "]", + parser.getTokenLocation()); } return new RangeAggregator.Factory(aggregationName, vsParser.config(), InternalIPv4Range.FACTORY, ranges, keyed); @@ -122,7 +126,8 @@ public class IpRangeParser implements Aggregator.Parser { private static void parseMaskRange(String cidr, RangeAggregator.Range range, String aggregationName, SearchContext ctx) { long[] fromTo = IPv4RangeBuilder.cidrMaskToMinMax(cidr); if (fromTo == null) { - throw new SearchParseException(ctx, "invalid CIDR mask [" + cidr + "] in aggregation [" + aggregationName + "]"); + throw new SearchParseException(ctx, "invalid CIDR mask [" + cidr + "] in aggregation [" + aggregationName + "]", + null); } range.from = fromTo[0] < 0 ? Double.NEGATIVE_INFINITY : fromTo[0]; range.to = fromTo[1] < 0 ? Double.POSITIVE_INFINITY : fromTo[1]; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedBytesHashSamplerAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedBytesHashSamplerAggregator.java index be81fc33719..5487e200fac 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedBytesHashSamplerAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedBytesHashSamplerAggregator.java @@ -31,10 +31,12 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -47,9 +49,10 @@ public class DiversifiedBytesHashSamplerAggregator extends SamplerAggregator { private int maxDocsPerValue; public DiversifiedBytesHashSamplerAggregator(String name, int shardSize, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData, ValuesSource valuesSource, + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData, + ValuesSource valuesSource, int maxDocsPerValue) throws IOException { - super(name, shardSize, factories, aggregationContext, parent, metaData); + super(name, shardSize, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.maxDocsPerValue = maxDocsPerValue; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java index 9774dddd391..bb34ea1c1a8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java @@ -33,10 +33,12 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; +import java.util.List; import java.util.Map; public class DiversifiedMapSamplerAggregator extends SamplerAggregator { @@ -46,9 +48,9 @@ public class DiversifiedMapSamplerAggregator extends SamplerAggregator { private BytesRefHash bucketOrds; public DiversifiedMapSamplerAggregator(String name, int shardSize, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData, ValuesSource valuesSource, - int maxDocsPerValue) throws IOException { - super(name, shardSize, factories, aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData, + ValuesSource valuesSource, int maxDocsPerValue) throws IOException { + super(name, shardSize, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.maxDocsPerValue = maxDocsPerValue; bucketOrds = new BytesRefHash(shardSize, aggregationContext.bigArrays()); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedNumericSamplerAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedNumericSamplerAggregator.java index c813a4ec084..4947dff39e5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedNumericSamplerAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedNumericSamplerAggregator.java @@ -30,10 +30,12 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; +import java.util.List; import java.util.Map; public class DiversifiedNumericSamplerAggregator extends SamplerAggregator { @@ -42,9 +44,9 @@ public class DiversifiedNumericSamplerAggregator extends SamplerAggregator { private int maxDocsPerValue; public DiversifiedNumericSamplerAggregator(String name, int shardSize, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData, ValuesSource.Numeric valuesSource, - int maxDocsPerValue) throws IOException { - super(name, shardSize, factories, aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData, + ValuesSource.Numeric valuesSource, int maxDocsPerValue) throws IOException { + super(name, shardSize, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.maxDocsPerValue = maxDocsPerValue; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedOrdinalsSamplerAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedOrdinalsSamplerAggregator.java index 407ba512f7f..87f1d5d8ab2 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedOrdinalsSamplerAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedOrdinalsSamplerAggregator.java @@ -31,10 +31,12 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; +import java.util.List; import java.util.Map; public class DiversifiedOrdinalsSamplerAggregator extends SamplerAggregator { @@ -43,9 +45,9 @@ public class DiversifiedOrdinalsSamplerAggregator extends SamplerAggregator { private int maxDocsPerValue; public DiversifiedOrdinalsSamplerAggregator(String name, int shardSize, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData, + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, int maxDocsPerValue) throws IOException { - super(name, shardSize, factories, aggregationContext, parent, metaData); + super(name, shardSize, factories, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.maxDocsPerValue = maxDocsPerValue; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSampler.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSampler.java index 509c89e3ccc..ce24c60a3e4 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSampler.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/InternalSampler.java @@ -22,8 +22,10 @@ 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.InternalSingleBucketAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,8 @@ public class InternalSampler extends InternalSingleBucketAggregation implements InternalSampler() { } // for serialization - InternalSampler(String name, long docCount, InternalAggregations subAggregations, Map metaData) { - super(name, docCount, subAggregations, metaData); + InternalSampler(String name, long docCount, InternalAggregations subAggregations, List reducers, Map metaData) { + super(name, docCount, subAggregations, reducers, metaData); } @Override @@ -59,7 +61,8 @@ public class InternalSampler extends InternalSingleBucketAggregation implements } @Override - protected InternalSingleBucketAggregation newAggregation(String name, long docCount, InternalAggregations subAggregations) { - return new InternalSampler(name, docCount, subAggregations, metaData); + protected InternalSingleBucketAggregation newAggregation(String name, long docCount, + InternalAggregations subAggregations) { + return new InternalSampler(name, docCount, subAggregations, reducers(), metaData); } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java index d4a078c2882..a3b7137975a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregator.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.BestDocsDeferringCollector; import org.elasticsearch.search.aggregations.bucket.DeferringBucketCollector; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -37,6 +38,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -58,9 +60,9 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { + AggregationContext context, Aggregator parent, List reducers, Map metaData) throws IOException { - return new DiversifiedMapSamplerAggregator(name, shardSize, factories, context, parent, metaData, valuesSource, + return new DiversifiedMapSamplerAggregator(name, shardSize, factories, context, parent, reducers, metaData, valuesSource, maxDocsPerValue); } @@ -74,9 +76,10 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { + AggregationContext context, Aggregator parent, List reducers, Map metaData) throws IOException { - return new DiversifiedBytesHashSamplerAggregator(name, shardSize, factories, context, parent, metaData, valuesSource, + return new DiversifiedBytesHashSamplerAggregator(name, shardSize, factories, context, parent, reducers, metaData, + valuesSource, maxDocsPerValue); } @@ -90,8 +93,8 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - return new DiversifiedOrdinalsSamplerAggregator(name, shardSize, factories, context, parent, metaData, + AggregationContext context, Aggregator parent, List reducers, Map metaData) throws IOException { + return new DiversifiedOrdinalsSamplerAggregator(name, shardSize, factories, context, parent, reducers, metaData, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, maxDocsPerValue); } @@ -118,7 +121,8 @@ public class SamplerAggregator extends SingleBucketAggregator { } abstract Aggregator create(String name, AggregatorFactories factories, int shardSize, int maxDocsPerValue, ValuesSource valuesSource, - AggregationContext context, Aggregator parent, Map metaData) throws IOException; + AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException; abstract boolean needsGlobalOrdinals(); @@ -132,9 +136,9 @@ public class SamplerAggregator extends SingleBucketAggregator { protected final int shardSize; protected BestDocsDeferringCollector bdd; - public SamplerAggregator(String name, int shardSize, AggregatorFactories factories, - AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, metaData); + public SamplerAggregator(String name, int shardSize, AggregatorFactories factories, AggregationContext aggregationContext, + Aggregator parent, List reducers, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, reducers, metaData); this.shardSize = shardSize; } @@ -159,12 +163,13 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOException { runDeferredCollections(owningBucketOrdinal); - return new InternalSampler(name, bdd == null ? 0 : bdd.getDocCount(), bucketAggregations(owningBucketOrdinal), metaData()); + return new InternalSampler(name, bdd == null ? 0 : bdd.getDocCount(), bucketAggregations(owningBucketOrdinal), reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalSampler(name, 0, buildEmptySubAggregations(), metaData()); + return new InternalSampler(name, 0, buildEmptySubAggregations(), reducers(), metaData()); } public static class Factory extends AggregatorFactory { @@ -178,12 +183,12 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, - Map metaData) throws IOException { + List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, context, parent); } - return new SamplerAggregator(name, shardSize, factories, context, parent, metaData); + return new SamplerAggregator(name, shardSize, factories, context, parent, reducers, metaData); } } @@ -203,7 +208,7 @@ public class SamplerAggregator extends SingleBucketAggregator { @Override protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent, - boolean collectsFromSingleBucket, Map metaData) throws IOException { + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, context, parent); @@ -211,7 +216,7 @@ public class SamplerAggregator extends SingleBucketAggregator { if (valuesSource instanceof ValuesSource.Numeric) { - return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, metaData, + return new DiversifiedNumericSamplerAggregator(name, shardSize, factories, context, parent, reducers, metaData, (Numeric) valuesSource, maxDocsPerValue); } @@ -229,7 +234,7 @@ public class SamplerAggregator extends SingleBucketAggregator { if ((execution.needsGlobalOrdinals()) && (!(valuesSource instanceof ValuesSource.Bytes.WithOrdinals))) { execution = ExecutionMode.MAP; } - return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, metaData); + return execution.create(name, factories, shardSize, maxDocsPerValue, valuesSource, context, parent, reducers, metaData); } throw new AggregationExecutionException("Sampler aggregation cannot be applied to field [" + config.fieldContext().field() + @@ -237,11 +242,11 @@ public class SamplerAggregator extends SingleBucketAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) - throws IOException { - final UnmappedSampler aggregation = new UnmappedSampler(name, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + final UnmappedSampler aggregation = new UnmappedSampler(name, reducers, metaData); - return new NonCollectingAggregator(name, aggregationContext, parent, factories, metaData) { + return new NonCollectingAggregator(name, aggregationContext, parent, factories, reducers, metaData) { @Override public InternalAggregation buildEmptyAggregation() { return aggregation; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java index 35a2963187e..d82dd2c6721 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerParser.java @@ -73,17 +73,18 @@ public class SamplerParser implements Aggregator.Parser { maxDocsPerValue = parser.intValue(); } else { throw new SearchParseException(context, "Unsupported property \"" + currentFieldName + "\" for aggregation \"" - + aggregationName); + + aggregationName, parser.getTokenLocation()); } } else if (!vsParser.token(currentFieldName, token, parser)) { if (EXECUTION_HINT_FIELD.match(currentFieldName)) { executionHint = parser.text(); } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } else { throw new SearchParseException(context, "Unsupported property \"" + currentFieldName + "\" for aggregation \"" - + aggregationName); + + aggregationName, parser.getTokenLocation()); } } @@ -93,7 +94,8 @@ public class SamplerParser implements Aggregator.Parser { } else { if (diversityChoiceMade) { throw new SearchParseException(context, "Sampler aggregation has " + MAX_DOCS_PER_VALUE_FIELD.getPreferredName() - + " setting but no \"field\" or \"script\" setting to provide values for aggregation \"" + aggregationName + "\""); + + " setting but no \"field\" or \"script\" setting to provide values for aggregation \"" + aggregationName + "\"", + parser.getTokenLocation()); } return new SamplerAggregator.Factory(aggregationName, shardSize); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/UnmappedSampler.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/UnmappedSampler.java index 95f8c7bfe78..5b019ee8bfb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/UnmappedSampler.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/UnmappedSampler.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.List; @@ -52,8 +53,8 @@ public class UnmappedSampler extends InternalSampler { UnmappedSampler() { } - public UnmappedSampler(String name, Map metaData) { - super(name, 0, InternalAggregations.EMPTY, metaData); + public UnmappedSampler(String name, List reducers, Map metaData) { + super(name, 0, InternalAggregations.EMPTY, reducers, metaData); } @Override @@ -62,7 +63,7 @@ public class UnmappedSampler extends InternalSampler { } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { for (InternalAggregation agg : aggregations) { if (!(agg instanceof UnmappedSampler)) { return agg.reduce(aggregations, reduceContext); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java index 49a7e56eefb..492167f1735 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/GlobalOrdinalsSignificantTermsAggregator.java @@ -25,10 +25,11 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.util.LongHash; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.ContextIndexSearcher; @@ -36,6 +37,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -46,12 +48,13 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri protected long numCollectedDocs; protected final SignificantTermsAggregatorFactory termsAggFactory; - public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, - BucketCountThresholds bucketCountThresholds, - IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, - SignificantTermsAggregatorFactory termsAggFactory, Map metaData) throws IOException { + public GlobalOrdinalsSignificantTermsAggregator(String name, AggregatorFactories factories, + ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, + IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, + SignificantTermsAggregatorFactory termsAggFactory, List reducers, Map metaData) throws IOException { - super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData); + super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, + SubAggCollectionMode.DEPTH_FIRST, false, reducers, metaData); this.termsAggFactory = termsAggFactory; } @@ -124,7 +127,9 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri list[i] = bucket; } - return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData()); + return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), reducers(), + metaData()); } @Override @@ -133,7 +138,9 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); + return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), + Collections. emptyList(), reducers(), metaData()); } @Override @@ -145,8 +152,8 @@ public class GlobalOrdinalsSignificantTermsAggregator extends GlobalOrdinalsStri private final LongHash bucketOrds; - public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, Map metaData) throws IOException { - super(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, metaData); + public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, List reducers, Map metaData) throws IOException { + super(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, termsAggFactory, reducers, metaData); bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java index 6ea57b606a9..825bb637fef 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/InternalSignificantTerms.java @@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.util.ArrayList; import java.util.Arrays; @@ -38,12 +39,13 @@ import java.util.Map; /** * */ -public abstract class InternalSignificantTerms extends InternalMultiBucketAggregation implements SignificantTerms, ToXContent, Streamable { +public abstract class InternalSignificantTerms extends + InternalMultiBucketAggregation implements SignificantTerms, ToXContent, Streamable { protected SignificanceHeuristic significanceHeuristic; protected int requiredSize; protected long minDocCount; - protected List buckets; + protected List buckets; protected Map bucketMap; protected long subsetSize; protected long supersetSize; @@ -122,8 +124,10 @@ public abstract class InternalSignificantTerms extends InternalMultiBucketAggreg } } - protected InternalSignificantTerms(long subsetSize, long supersetSize, String name, int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List buckets, Map metaData) { - super(name, metaData); + protected InternalSignificantTerms(long subsetSize, long supersetSize, String name, int requiredSize, long minDocCount, + SignificanceHeuristic significanceHeuristic, List buckets, List reducers, + Map metaData) { + super(name, reducers, metaData); this.requiredSize = requiredSize; this.minDocCount = minDocCount; this.buckets = buckets; @@ -156,20 +160,20 @@ public abstract class InternalSignificantTerms extends InternalMultiBucketAggreg } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { long globalSubsetSize = 0; long globalSupersetSize = 0; // Compute the overall result set size and the corpus size using the // top-level Aggregations from each shard for (InternalAggregation aggregation : aggregations) { - InternalSignificantTerms terms = (InternalSignificantTerms) aggregation; + InternalSignificantTerms terms = (InternalSignificantTerms) aggregation; globalSubsetSize += terms.subsetSize; globalSupersetSize += terms.supersetSize; } Map> buckets = new HashMap<>(); for (InternalAggregation aggregation : aggregations) { - InternalSignificantTerms terms = (InternalSignificantTerms) aggregation; + InternalSignificantTerms terms = (InternalSignificantTerms) aggregation; for (Bucket bucket : terms.buckets) { List existingBuckets = buckets.get(bucket.getKey()); if (existingBuckets == null) { @@ -197,9 +201,10 @@ public abstract class InternalSignificantTerms extends InternalMultiBucketAggreg for (int i = ordered.size() - 1; i >= 0; i--) { list[i] = (Bucket) ordered.pop(); } - return newAggregation(globalSubsetSize, globalSupersetSize, Arrays.asList(list)); + return create(globalSubsetSize, globalSupersetSize, Arrays.asList(list), this); } - abstract InternalSignificantTerms newAggregation(long subsetSize, long supersetSize, List buckets); + protected abstract A create(long subsetSize, long supersetSize, List buckets, + InternalSignificantTerms prototype); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTerms.java index fa3b7587533..a450f9d0933 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTerms.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -41,7 +42,7 @@ import java.util.Map; /** * */ -public class SignificantLongTerms extends InternalSignificantTerms { +public class SignificantLongTerms extends InternalSignificantTerms { public static final Type TYPE = new Type("significant_terms", "siglterms"); @@ -161,16 +162,16 @@ public class SignificantLongTerms extends InternalSignificantTerms { return builder; } } - private ValueFormatter formatter; SignificantLongTerms() { } // for serialization - public SignificantLongTerms(long subsetSize, long supersetSize, String name, @Nullable ValueFormatter formatter, - int requiredSize, long minDocCount, SignificanceHeuristic significanceHeuristic, List buckets, Map metaData) { + public SignificantLongTerms(long subsetSize, long supersetSize, String name, @Nullable ValueFormatter formatter, int requiredSize, + long minDocCount, SignificanceHeuristic significanceHeuristic, List buckets, + List reducers, Map metaData) { - super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, metaData); + super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, reducers, metaData); this.formatter = formatter; } @@ -180,9 +181,24 @@ public class SignificantLongTerms extends InternalSignificantTerms { } @Override - InternalSignificantTerms newAggregation(long subsetSize, long supersetSize, - List buckets) { - return new SignificantLongTerms(subsetSize, supersetSize, getName(), formatter, requiredSize, minDocCount, significanceHeuristic, buckets, getMetaData()); + public SignificantLongTerms create(List buckets) { + return new SignificantLongTerms(this.subsetSize, this.supersetSize, this.name, this.formatter, this.requiredSize, this.minDocCount, + this.significanceHeuristic, buckets, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, SignificantLongTerms.Bucket prototype) { + return new Bucket(prototype.subsetDf, prototype.subsetSize, prototype.supersetDf, prototype.supersetSize, prototype.term, + aggregations, prototype.formatter); + } + + @Override + protected SignificantLongTerms create(long subsetSize, long supersetSize, + List buckets, + InternalSignificantTerms prototype) { + return new SignificantLongTerms(subsetSize, supersetSize, prototype.getName(), ((SignificantLongTerms) prototype).formatter, + prototype.requiredSize, prototype.minDocCount, prototype.significanceHeuristic, buckets, prototype.reducers(), + prototype.getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java index 0b8d5813721..329f5f566f5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantLongTermsAggregator.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.terms.LongTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.format.ValueFormat; @@ -36,6 +37,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -44,10 +46,12 @@ import java.util.Map; public class SignificantLongTermsAggregator extends LongTermsAggregator { public SignificantLongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, - BucketCountThresholds bucketCountThresholds, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, Map metaData) throws IOException { + BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, + Aggregator parent, SignificantTermsAggregatorFactory termsAggFactory, IncludeExclude.LongFilter includeExclude, + List reducers, Map metaData) throws IOException { - super(name, factories, valuesSource, format, null, bucketCountThresholds, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, metaData); + super(name, factories, valuesSource, format, null, bucketCountThresholds, aggregationContext, parent, + SubAggCollectionMode.DEPTH_FIRST, false, includeExclude, reducers, metaData); this.termsAggFactory = termsAggFactory; } @@ -102,7 +106,9 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator { bucket.aggregations = bucketAggregations(bucket.bucketOrd); list[i] = bucket; } - return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData()); + return new SignificantLongTerms(subsetSize, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), reducers(), + metaData()); } @Override @@ -111,7 +117,9 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator { ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); + return new SignificantLongTerms(0, supersetSize, name, formatter, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), + Collections. emptyList(), reducers(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java index 374f3a09ce5..9fbaa6cc375 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTerms.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -40,7 +41,7 @@ import java.util.Map; /** * */ -public class SignificantStringTerms extends InternalSignificantTerms { +public class SignificantStringTerms extends InternalSignificantTerms { public static final InternalAggregation.Type TYPE = new Type("significant_terms", "sigsterms"); @@ -159,9 +160,10 @@ public class SignificantStringTerms extends InternalSignificantTerms { SignificantStringTerms() {} // for serialization - public SignificantStringTerms(long subsetSize, long supersetSize, String name, int requiredSize, - long minDocCount, SignificanceHeuristic significanceHeuristic, List buckets, Map metaData) { - super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, metaData); + public SignificantStringTerms(long subsetSize, long supersetSize, String name, int requiredSize, long minDocCount, + SignificanceHeuristic significanceHeuristic, List buckets, List reducers, + Map metaData) { + super(subsetSize, supersetSize, name, requiredSize, minDocCount, significanceHeuristic, buckets, reducers, metaData); } @Override @@ -170,9 +172,22 @@ public class SignificantStringTerms extends InternalSignificantTerms { } @Override - InternalSignificantTerms newAggregation(long subsetSize, long supersetSize, - List buckets) { - return new SignificantStringTerms(subsetSize, supersetSize, getName(), requiredSize, minDocCount, significanceHeuristic, buckets, getMetaData()); + public SignificantStringTerms create(List buckets) { + return new SignificantStringTerms(this.subsetSize, this.supersetSize, this.name, this.requiredSize, this.minDocCount, + this.significanceHeuristic, buckets, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, SignificantStringTerms.Bucket prototype) { + return new Bucket(prototype.termBytes, prototype.subsetDf, prototype.subsetSize, prototype.supersetDf, prototype.supersetSize, + aggregations); + } + + @Override + protected SignificantStringTerms create(long subsetSize, long supersetSize, List buckets, + InternalSignificantTerms prototype) { + return new SignificantStringTerms(subsetSize, supersetSize, prototype.getName(), prototype.requiredSize, prototype.minDocCount, + prototype.significanceHeuristic, buckets, prototype.reducers(), prototype.getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java index 532a71efae7..a49f18734ee 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantStringTermsAggregator.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.terms.StringTermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.ContextIndexSearcher; @@ -35,6 +36,7 @@ import org.elasticsearch.search.internal.ContextIndexSearcher; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -48,9 +50,11 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { public SignificantStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, BucketCountThresholds bucketCountThresholds, IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, - SignificantTermsAggregatorFactory termsAggFactory, Map metaData) throws IOException { + SignificantTermsAggregatorFactory termsAggFactory, List reducers, Map metaData) + throws IOException { - super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, SubAggCollectionMode.DEPTH_FIRST, false, metaData); + super(name, factories, valuesSource, null, bucketCountThresholds, includeExclude, aggregationContext, parent, + SubAggCollectionMode.DEPTH_FIRST, false, reducers, metaData); this.termsAggFactory = termsAggFactory; } @@ -107,7 +111,9 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { list[i] = bucket; } - return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), metaData()); + return new SignificantStringTerms(subsetSize, supersetSize, name, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Arrays.asList(list), reducers(), + metaData()); } @Override @@ -116,7 +122,9 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator { ContextIndexSearcher searcher = context.searchContext().searcher(); IndexReader topReader = searcher.getIndexReader(); int supersetSize = topReader.numDocs(); - return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), Collections.emptyList(), metaData()); + return new SignificantStringTerms(0, supersetSize, name, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), termsAggFactory.getSignificanceHeuristic(), + Collections. emptyList(), reducers(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java index 8f84be12770..ea4085f7cf2 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregatorFactory.java @@ -37,6 +37,7 @@ import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -44,6 +45,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -61,10 +63,12 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac @Override Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, - TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { + TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, + List reducers, Map metaData) throws IOException { final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter(); - return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, filter, aggregationContext, parent, termsAggregatorFactory, metaData); + return new SignificantStringTermsAggregator(name, factories, valuesSource, bucketCountThresholds, filter, + aggregationContext, parent, termsAggregatorFactory, reducers, metaData); } }, @@ -73,11 +77,12 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac @Override Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, + List reducers, Map metaData) throws IOException { ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource; IndexSearcher indexSearcher = aggregationContext.searchContext().searcher(); final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); - return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent, termsAggregatorFactory, metaData); + return new GlobalOrdinalsSignificantTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent, termsAggregatorFactory, reducers, metaData); } }, @@ -86,9 +91,12 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac @Override Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, + List reducers, Map metaData) throws IOException { final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); - return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, aggregationContext, parent, termsAggregatorFactory, metaData); + return new GlobalOrdinalsSignificantTermsAggregator.WithHash(name, factories, + (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, bucketCountThresholds, filter, + aggregationContext, parent, termsAggregatorFactory, reducers, metaData); } }; @@ -109,7 +117,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac abstract Aggregator create(String name, AggregatorFactories factories, ValuesSource valuesSource, TermsAggregator.BucketCountThresholds bucketCountThresholds, IncludeExclude includeExclude, - AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, Map metaData) throws IOException; + AggregationContext aggregationContext, Aggregator parent, SignificantTermsAggregatorFactory termsAggregatorFactory, + List reducers, Map metaData) throws IOException; @Override public String toString() { @@ -146,9 +155,11 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getMinDocCount(), metaData); - return new NonCollectingAggregator(name, aggregationContext, parent, metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + final InternalAggregation aggregation = new UnmappedSignificantTerms(name, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), reducers, metaData); + return new NonCollectingAggregator(name, aggregationContext, parent, reducers, metaData) { @Override public InternalAggregation buildEmptyAggregation() { return aggregation; @@ -157,7 +168,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } @Override - protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, aggregationContext, parent); } @@ -180,7 +192,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac } } assert execution != null; - return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this, metaData); + return execution.create(name, factories, valuesSource, bucketCountThresholds, includeExclude, aggregationContext, parent, this, + reducers, metaData); } @@ -198,7 +211,8 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac if (includeExclude != null) { longFilter = includeExclude.convertToLongFilter(); } - return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), bucketCountThresholds, aggregationContext, parent, this, longFilter, metaData); + return new SignificantLongTermsAggregator(name, factories, (ValuesSource.Numeric) valuesSource, config.format(), + bucketCountThresholds, aggregationContext, parent, this, longFilter, reducers, metaData); } throw new AggregationExecutionException("sigfnificant_terms aggregation cannot be applied to field [" + config.fieldContext().field() + diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParametersParser.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParametersParser.java index e9288528363..87a60d43967 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParametersParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsParametersParser.java @@ -68,10 +68,12 @@ public class SignificantTermsParametersParser extends AbstractTermsParametersPar } else if (BACKGROUND_FILTER.match(currentFieldName)) { filter = context.queryParserService().parseInnerFilter(parser).filter(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + + "].", parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/UnmappedSignificantTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/UnmappedSignificantTerms.java index bf29dd630a3..cfd765dbf6c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/UnmappedSignificantTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/UnmappedSignificantTerms.java @@ -23,7 +23,9 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.JLHScore; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.Collections; @@ -33,7 +35,7 @@ import java.util.Map; /** * */ -public class UnmappedSignificantTerms extends InternalSignificantTerms { +public class UnmappedSignificantTerms extends InternalSignificantTerms { public static final Type TYPE = new Type("significant_terms", "umsigterms"); @@ -55,10 +57,10 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms { UnmappedSignificantTerms() {} // for serialization - public UnmappedSignificantTerms(String name, int requiredSize, long minDocCount, Map metaData) { + public UnmappedSignificantTerms(String name, int requiredSize, long minDocCount, List reducers, Map metaData) { //We pass zero for index/subset sizes because for the purpose of significant term analysis // we assume an unmapped index's size is irrelevant to the proceedings. - super(0, 0, name, requiredSize, minDocCount, JLHScore.INSTANCE, BUCKETS, metaData); + super(0, 0, name, requiredSize, minDocCount, JLHScore.INSTANCE, BUCKETS, reducers, metaData); } @Override @@ -67,7 +69,22 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms { } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public UnmappedSignificantTerms create(List buckets) { + return new UnmappedSignificantTerms(this.name, this.requiredSize, this.minDocCount, this.reducers(), this.metaData); + } + + @Override + public InternalSignificantTerms.Bucket createBucket(InternalAggregations aggregations, InternalSignificantTerms.Bucket prototype) { + throw new UnsupportedOperationException("not supported for UnmappedSignificantTerms"); + } + + @Override + protected UnmappedSignificantTerms create(long subsetSize, long supersetSize, List buckets, InternalSignificantTerms prototype) { + throw new UnsupportedOperationException("not supported for UnmappedSignificantTerms"); + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { for (InternalAggregation aggregation : aggregations) { if (!(aggregation instanceof UnmappedSignificantTerms)) { return aggregation.reduce(aggregations, reduceContext); @@ -76,11 +93,6 @@ public class UnmappedSignificantTerms extends InternalSignificantTerms { return this; } - @Override - InternalSignificantTerms newAggregation(long subsetSize, long supersetSize, List buckets) { - throw new UnsupportedOperationException("How did you get there?"); - } - @Override protected void doReadFrom(StreamInput in) throws IOException { this.requiredSize = readSize(in); diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java index e87821e4e38..363895c5a39 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/AbstractStringTermsAggregator.java @@ -22,27 +22,30 @@ package org.elasticsearch.search.aggregations.bucket.terms; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Map; abstract class AbstractStringTermsAggregator extends TermsAggregator { protected final boolean showTermDocCountError; - public AbstractStringTermsAggregator(String name, AggregatorFactories factories, - AggregationContext context, Aggregator parent, - Terms.Order order, BucketCountThresholds bucketCountThresholds, - SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, Map metaData) throws IOException { - super(name, factories, context, parent, bucketCountThresholds, order, subAggCollectMode, metaData); + public AbstractStringTermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, + Terms.Order order, BucketCountThresholds bucketCountThresholds, SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, List reducers, Map metaData) throws IOException { + super(name, factories, context, parent, bucketCountThresholds, order, subAggCollectMode, reducers, metaData); this.showTermDocCountError = showTermDocCountError; } @Override public InternalAggregation buildEmptyAggregation() { - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, metaData()); + return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), + bucketCountThresholds.getMinDocCount(), Collections. emptyList(), showTermDocCountError, 0, 0, + reducers(), metaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java index c004f6e1e90..dbb8061db09 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java @@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -39,7 +40,7 @@ import java.util.Map; /** * */ -public class DoubleTerms extends InternalTerms { +public class DoubleTerms extends InternalTerms { public static final Type TYPE = new Type("terms", "dterms"); @@ -84,7 +85,8 @@ public class DoubleTerms extends InternalTerms { super(formatter, showDocCountError); } - public Bucket(double term, long docCount, InternalAggregations aggregations, boolean showDocCountError, long docCountError, @Nullable ValueFormatter formatter) { + public Bucket(double term, long docCount, InternalAggregations aggregations, boolean showDocCountError, long docCountError, + @Nullable ValueFormatter formatter) { super(docCount, aggregations, showDocCountError, docCountError, formatter); this.term = term; } @@ -152,12 +154,17 @@ public class DoubleTerms extends InternalTerms { } } - private @Nullable ValueFormatter formatter; + private @Nullable + ValueFormatter formatter; - DoubleTerms() {} // for serialization + DoubleTerms() { + } // for serialization - public DoubleTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public DoubleTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, + long minDocCount, List buckets, boolean showTermDocCountError, long docCountError, + long otherDocCount, List reducers, Map metaData) { + super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, reducers, + metaData); this.formatter = formatter; } @@ -167,8 +174,23 @@ public class DoubleTerms extends InternalTerms { } @Override - protected InternalTerms newAggregation(String name, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - return new DoubleTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public DoubleTerms create(List buckets) { + return new DoubleTerms(this.name, this.order, this.formatter, this.requiredSize, this.shardSize, this.minDocCount, buckets, + this.showTermDocCountError, this.docCountError, this.otherDocCount, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.term, prototype.docCount, aggregations, prototype.showDocCountError, prototype.docCountError, + prototype.formatter); + } + + @Override + protected DoubleTerms create(String name, List buckets, + long docCountError, long otherDocCount, InternalTerms prototype) { + return new DoubleTerms(name, prototype.order, ((DoubleTerms) prototype).formatter, prototype.requiredSize, prototype.shardSize, + prototype.minDocCount, buckets, prototype.showTermDocCountError, docCountError, otherDocCount, prototype.reducers(), + prototype.getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java index e71be14dc5b..9250495524e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsAggregator.java @@ -26,6 +26,7 @@ import org.elasticsearch.index.fielddata.FieldData; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -33,6 +34,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormat; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -41,8 +43,11 @@ import java.util.Map; public class DoubleTermsAggregator extends LongTermsAggregator { public DoubleTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, - Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map metaData) throws IOException { - super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode, showTermDocCountError, longFilter, metaData); + Terms.Order order, BucketCountThresholds bucketCountThresholds, + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, + IncludeExclude.LongFilter longFilter, List reducers, Map metaData) throws IOException { + super(name, factories, valuesSource, format, order, bucketCountThresholds, aggregationContext, parent, collectionMode, + showTermDocCountError, longFilter, reducers, metaData); } @Override @@ -73,7 +78,9 @@ public class DoubleTermsAggregator extends LongTermsAggregator { for (int i = 0; i < buckets.length; ++i) { buckets[i] = convertToDouble(buckets[i]); } - return new DoubleTerms(terms.getName(), terms.order, terms.formatter, terms.requiredSize, terms.shardSize, terms.minDocCount, Arrays.asList(buckets), terms.showTermDocCountError, terms.docCountError, terms.otherDocCount, terms.getMetaData()); + return new DoubleTerms(terms.getName(), terms.order, terms.formatter, terms.requiredSize, terms.shardSize, terms.minDocCount, + Arrays.asList(buckets), terms.showTermDocCountError, terms.docCountError, terms.otherDocCount, terms.reducers(), + terms.getMetaData()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java index 767f2d50926..67766c79c19 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java @@ -37,18 +37,20 @@ import org.elasticsearch.index.fielddata.AbstractRandomAccessOrds; import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalMapping; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; -import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms.Bucket; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -71,8 +73,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr public GlobalOrdinalsStringTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, Terms.Order order, BucketCountThresholds bucketCountThresholds, - IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); + IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, reducers, + metaData); this.valuesSource = valuesSource; this.includeExclude = includeExclude; } @@ -196,7 +199,9 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr bucket.docCountError = 0; } - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData()); + return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), + bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, reducers(), + metaData()); } /** @@ -261,8 +266,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr public WithHash(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, Terms.Order order, BucketCountThresholds bucketCountThresholds, IncludeExclude.OrdinalsFilter includeExclude, AggregationContext aggregationContext, - Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { - super(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); + Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { + super(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, collectionMode, showTermDocCountError, reducers, metaData); bucketOrds = new LongHash(1, aggregationContext.bigArrays()); } @@ -330,8 +335,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr private RandomAccessOrds segmentOrds; public LowCardinality(String name, AggregatorFactories factories, ValuesSource.Bytes.WithOrdinals.FieldData valuesSource, - Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { - super(name, factories, valuesSource, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, metaData); + Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { + super(name, factories, valuesSource, order, bucketCountThresholds, null, aggregationContext, parent, collectionMode, showTermDocCountError, reducers, metaData); assert factories == null || factories.count() == 0; this.segmentDocCounts = context.bigArrays().newIntArray(1, true); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java index b1e67ca87c2..c5fedd659f0 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.util.ArrayList; @@ -41,7 +42,8 @@ import java.util.Map; /** * */ -public abstract class InternalTerms extends InternalMultiBucketAggregation implements Terms, ToXContent, Streamable { +public abstract class InternalTerms extends InternalMultiBucketAggregation + implements Terms, ToXContent, Streamable { protected static final String DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME = "doc_count_error_upper_bound"; protected static final String SUM_OF_OTHER_DOC_COUNTS = "sum_other_doc_count"; @@ -113,7 +115,7 @@ public abstract class InternalTerms extends InternalMultiBucketAggregation imple protected int requiredSize; protected int shardSize; protected long minDocCount; - protected List buckets; + protected List buckets; protected Map bucketMap; protected long docCountError; protected boolean showTermDocCountError; @@ -121,8 +123,10 @@ public abstract class InternalTerms extends InternalMultiBucketAggregation imple protected InternalTerms() {} // for serialization - protected InternalTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - super(name, metaData); + protected InternalTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, + List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, List reducers, + Map metaData) { + super(name, reducers, metaData); this.order = order; this.requiredSize = requiredSize; this.shardSize = shardSize; @@ -161,13 +165,13 @@ public abstract class InternalTerms extends InternalMultiBucketAggregation imple } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { Multimap buckets = ArrayListMultimap.create(); long sumDocCountError = 0; long otherDocCount = 0; for (InternalAggregation aggregation : aggregations) { - InternalTerms terms = (InternalTerms) aggregation; + InternalTerms terms = (InternalTerms) aggregation; otherDocCount += terms.getSumOfOtherDocCounts(); final long thisAggDocCountError; if (terms.buckets.size() < this.shardSize || this.order == InternalOrder.TERM_ASC || this.order == InternalOrder.TERM_DESC) { @@ -220,9 +224,10 @@ public abstract class InternalTerms extends InternalMultiBucketAggregation imple } else { docCountError = aggregations.size() == 1 ? 0 : sumDocCountError; } - return newAggregation(name, Arrays.asList(list), showTermDocCountError, docCountError, otherDocCount, getMetaData()); + return create(name, Arrays.asList(list), docCountError, otherDocCount, this); } - protected abstract InternalTerms newAggregation(String name, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData); + protected abstract A create(String name, List buckets, long docCountError, long otherDocCount, + InternalTerms prototype); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java index 1a7c2b4d0ee..eee9e6bfc4b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -38,7 +39,7 @@ import java.util.Map; /** * */ -public class LongTerms extends InternalTerms { +public class LongTerms extends InternalTerms { public static final Type TYPE = new Type("terms", "lterms"); @@ -155,8 +156,11 @@ public class LongTerms extends InternalTerms { LongTerms() {} // for serialization - public LongTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public LongTerms(String name, Terms.Order order, @Nullable ValueFormatter formatter, int requiredSize, int shardSize, long minDocCount, + List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, + List reducers, Map metaData) { + super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, reducers, + metaData); this.formatter = formatter; } @@ -166,8 +170,23 @@ public class LongTerms extends InternalTerms { } @Override - protected InternalTerms newAggregation(String name, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - return new LongTerms(name, order, formatter, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public LongTerms create(List buckets) { + return new LongTerms(this.name, this.order, this.formatter, this.requiredSize, this.shardSize, this.minDocCount, buckets, + this.showTermDocCountError, this.docCountError, this.otherDocCount, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.term, prototype.docCount, aggregations, prototype.showDocCountError, prototype.docCountError, + prototype.formatter); + } + + @Override + protected LongTerms create(String name, List buckets, + long docCountError, long otherDocCount, InternalTerms prototype) { + return new LongTerms(name, prototype.order, ((LongTerms) prototype).formatter, prototype.requiredSize, prototype.shardSize, + prototype.minDocCount, buckets, prototype.showTermDocCountError, docCountError, otherDocCount, prototype.reducers(), + prototype.getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java index a570b06360f..ea32e388fe6 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsAggregator.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude.LongFilter; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.format.ValueFormat; @@ -39,6 +40,7 @@ import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; /** @@ -53,8 +55,10 @@ public class LongTermsAggregator extends TermsAggregator { private LongFilter longFilter; public LongTermsAggregator(String name, AggregatorFactories factories, ValuesSource.Numeric valuesSource, @Nullable ValueFormat format, - Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, metaData); + Terms.Order order, BucketCountThresholds bucketCountThresholds, AggregationContext aggregationContext, Aggregator parent, + SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, IncludeExclude.LongFilter longFilter, + List reducers, Map metaData) throws IOException { + super(name, factories, aggregationContext, parent, bucketCountThresholds, order, subAggCollectMode, reducers, metaData); this.valuesSource = valuesSource; this.showTermDocCountError = showTermDocCountError; this.formatter = format != null ? format.formatter() : null; @@ -157,13 +161,16 @@ public class LongTermsAggregator extends TermsAggregator { list[i].docCountError = 0; } - return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData()); + return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), + bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, reducers(), + metaData()); } - @Override public InternalAggregation buildEmptyAggregation() { - return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Collections.emptyList(), showTermDocCountError, 0, 0, metaData()); + return new LongTerms(name, order, formatter, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), + bucketCountThresholds.getMinDocCount(), Collections. emptyList(), showTermDocCountError, 0, 0, + reducers(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java index 7caec199df3..ee458acdf13 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java @@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.bucket.BucketStreamContext; import org.elasticsearch.search.aggregations.bucket.BucketStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -37,7 +38,7 @@ import java.util.Map; /** * */ -public class StringTerms extends InternalTerms { +public class StringTerms extends InternalTerms { public static final InternalAggregation.Type TYPE = new Type("terms", "sterms"); @@ -73,7 +74,6 @@ public class StringTerms extends InternalTerms { BucketStreams.registerStream(BUCKET_STREAM, TYPE.stream()); } - public static class Bucket extends InternalTerms.Bucket { BytesRef termBytes; @@ -148,10 +148,14 @@ public class StringTerms extends InternalTerms { } } - StringTerms() {} // for serialization + StringTerms() { + } // for serialization - public StringTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public StringTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, + List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, + List reducers, Map metaData) { + super(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, reducers, + metaData); } @Override @@ -160,8 +164,21 @@ public class StringTerms extends InternalTerms { } @Override - protected InternalTerms newAggregation(String name, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - return new StringTerms(name, order, requiredSize, shardSize, minDocCount, buckets, showTermDocCountError, docCountError, otherDocCount, metaData); + public StringTerms create(List buckets) { + return new StringTerms(this.name, this.order, this.requiredSize, this.shardSize, this.minDocCount, buckets, + this.showTermDocCountError, this.docCountError, this.otherDocCount, this.reducers(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.termBytes, prototype.docCount, aggregations, prototype.showDocCountError, prototype.docCountError); + } + + @Override + protected StringTerms create(String name, List buckets, + long docCountError, long otherDocCount, InternalTerms prototype) { + return new StringTerms(name, prototype.order, prototype.requiredSize, prototype.shardSize, prototype.minDocCount, buckets, + prototype.showTermDocCountError, docCountError, otherDocCount, prototype.reducers(), prototype.getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java index d625e3b9954..6f80142da27 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsAggregator.java @@ -31,11 +31,13 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.terms.support.BucketPriorityQueue; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Map; /** @@ -49,9 +51,12 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator { public StringTermsAggregator(String name, AggregatorFactories factories, ValuesSource valuesSource, Terms.Order order, BucketCountThresholds bucketCountThresholds, - IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, Map metaData) throws IOException { + IncludeExclude.StringFilter includeExclude, AggregationContext aggregationContext, + Aggregator parent, SubAggCollectionMode collectionMode, boolean showTermDocCountError, List reducers, + Map metaData) throws IOException { - super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, metaData); + super(name, factories, aggregationContext, parent, order, bucketCountThresholds, collectionMode, showTermDocCountError, reducers, + metaData); this.valuesSource = valuesSource; this.includeExclude = includeExclude; bucketOrds = new BytesRefHash(1, aggregationContext.bigArrays()); @@ -158,7 +163,9 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator { bucket.docCountError = 0; } - return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, metaData()); + return new StringTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), + bucketCountThresholds.getMinDocCount(), Arrays.asList(list), showTermDocCountError, 0, otherDocCount, reducers(), + metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java index 4cfe549a452..165675d0ddf 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregator.java @@ -28,11 +28,13 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.Aggregation; import org.elasticsearch.search.aggregations.bucket.terms.InternalOrder.CompoundOrder; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.AggregationPath; import java.io.IOException; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -135,8 +137,8 @@ public abstract class TermsAggregator extends BucketsAggregator { protected final Set aggsUsedForSorting = new HashSet<>(); protected final SubAggCollectionMode collectMode; - public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode collectMode, Map metaData) throws IOException { - super(name, factories, context, parent, metaData); + public TermsAggregator(String name, AggregatorFactories factories, AggregationContext context, Aggregator parent, BucketCountThresholds bucketCountThresholds, Terms.Order order, SubAggCollectionMode collectMode, List reducers, Map metaData) throws IOException { + super(name, factories, context, parent, reducers, metaData); this.bucketCountThresholds = bucketCountThresholds; this.order = InternalOrder.validate(order, this); this.collectMode = collectMode; diff --git a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java index 7207c67e742..4056409517e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregatorFactory.java @@ -27,12 +27,14 @@ import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.NonCollectingAggregator; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -47,9 +49,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, List reducers, Map metaData) throws IOException { final IncludeExclude.StringFilter filter = includeExclude == null ? null : includeExclude.convertToStringFilter(); - return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + return new StringTermsAggregator(name, factories, valuesSource, order, bucketCountThresholds, filter, + aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override @@ -63,9 +67,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); - return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + return new GlobalOrdinalsStringTermsAggregator(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override @@ -79,9 +83,9 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException { final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); - return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + return new GlobalOrdinalsStringTermsAggregator.WithHash(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, filter, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override @@ -94,11 +98,12 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { + AggregationContext aggregationContext, Aggregator parent, SubAggCollectionMode subAggCollectMode, + boolean showTermDocCountError, List reducers, Map metaData) throws IOException { if (includeExclude != null || factories.count() > 0) { - return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + return GLOBAL_ORDINALS.create(name, factories, valuesSource, order, bucketCountThresholds, includeExclude, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } - return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, metaData); + return new GlobalOrdinalsStringTermsAggregator.LowCardinality(name, factories, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, order, bucketCountThresholds, aggregationContext, parent, subAggCollectMode, showTermDocCountError, reducers, metaData); } @Override @@ -125,7 +130,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException; + SubAggCollectionMode subAggCollectMode, boolean showTermDocCountError, List reducers, Map metaData) throws IOException; abstract boolean needsGlobalOrdinals(); @@ -153,9 +158,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { - final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), metaData); - return new NonCollectingAggregator(name, aggregationContext, parent, factories, metaData) { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + final InternalAggregation aggregation = new UnmappedTerms(name, order, bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getShardSize(), bucketCountThresholds.getMinDocCount(), reducers, metaData); + return new NonCollectingAggregator(name, aggregationContext, parent, factories, reducers, metaData) { { // even in the case of an unmapped aggregator, validate the order InternalOrder.validate(order, this); @@ -168,7 +175,8 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, aggregationContext, parent); } @@ -218,7 +226,7 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory { public static final Type TYPE = new Type("terms", "umterms"); @@ -54,8 +56,9 @@ public class UnmappedTerms extends InternalTerms { UnmappedTerms() {} // for serialization - public UnmappedTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, Map metaData) { - super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0, 0, metaData); + public UnmappedTerms(String name, Terms.Order order, int requiredSize, int shardSize, long minDocCount, List reducers, + Map metaData) { + super(name, order, requiredSize, shardSize, minDocCount, BUCKETS, false, 0, 0, reducers, metaData); } @Override @@ -63,6 +66,21 @@ public class UnmappedTerms extends InternalTerms { return TYPE; } + @Override + public UnmappedTerms create(List buckets) { + return new UnmappedTerms(this.name, this.order, this.requiredSize, this.shardSize, this.minDocCount, this.reducers(), this.metaData); + } + + @Override + public InternalTerms.Bucket createBucket(InternalAggregations aggregations, InternalTerms.Bucket prototype) { + throw new UnsupportedOperationException("not supported for UnmappedTerms"); + } + + @Override + protected UnmappedTerms create(String name, List buckets, long docCountError, long otherDocCount, InternalTerms prototype) { + throw new UnsupportedOperationException("not supported for UnmappedTerms"); + } + @Override protected void doReadFrom(StreamInput in) throws IOException { this.docCountError = 0; @@ -81,7 +99,7 @@ public class UnmappedTerms extends InternalTerms { } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { for (InternalAggregation agg : aggregations) { if (!(agg instanceof UnmappedTerms)) { return agg.reduce(aggregations, reduceContext); @@ -90,11 +108,6 @@ public class UnmappedTerms extends InternalTerms { return this; } - @Override - protected InternalTerms newAggregation(String name, List buckets, boolean showTermDocCountError, long docCountError, long otherDocCount, Map metaData) { - throw new UnsupportedOperationException("How did you get there?"); - } - @Override public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { builder.field(InternalTerms.DOC_COUNT_ERROR_UPPER_BOUND_FIELD_NAME, docCountError); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMetricsAggregation.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMetricsAggregation.java index e3a9476e56a..8facf4c1ae5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMetricsAggregation.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalMetricsAggregation.java @@ -20,14 +20,16 @@ package org.elasticsearch.search.aggregations.metrics; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import java.util.List; import java.util.Map; public abstract class InternalMetricsAggregation extends InternalAggregation { protected InternalMetricsAggregation() {} // for serialization - protected InternalMetricsAggregation(String name, Map metaData) { - super(name, metaData); + protected InternalMetricsAggregation(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java index 5fac71fc6f7..d9d0aeeb1a8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalNumericMetricsAggregation.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.search.aggregations.metrics; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.util.List; @@ -34,8 +35,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA protected SingleValue() {} - protected SingleValue(String name, Map metaData) { - super(name, metaData); + protected SingleValue(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } @Override @@ -64,8 +65,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA protected MultiValue() {} - protected MultiValue(String name, Map metaData) { - super(name, metaData); + protected MultiValue(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } public abstract double value(String name); @@ -92,8 +93,8 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA private InternalNumericMetricsAggregation() {} // for serialization - private InternalNumericMetricsAggregation(String name, Map metaData) { - super(name, metaData); + private InternalNumericMetricsAggregation(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java index f29e063d61a..f3160cf464c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/MetricsAggregator.java @@ -22,14 +22,17 @@ package org.elasticsearch.search.aggregations.metrics; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorBase; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; public abstract class MetricsAggregator extends AggregatorBase { - protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, AggregatorFactories.EMPTY, context, parent, metaData); + protected MetricsAggregator(String name, AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, AggregatorFactories.EMPTY, context, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java index 66adf3ed74e..6342df383ed 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericMetricsAggregator.java @@ -19,9 +19,11 @@ package org.elasticsearch.search.aggregations.metrics; import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -29,14 +31,16 @@ import java.util.Map; */ public abstract class NumericMetricsAggregator extends MetricsAggregator { - private NumericMetricsAggregator(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + private NumericMetricsAggregator(String name, AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); } public static abstract class SingleValue extends NumericMetricsAggregator { - protected SingleValue(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + protected SingleValue(String name, AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); } public abstract double metric(long owningBucketOrd); @@ -44,8 +48,9 @@ public abstract class NumericMetricsAggregator extends MetricsAggregator { public static abstract class MultiValue extends NumericMetricsAggregator { - protected MultiValue(String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + protected MultiValue(String name, AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); } public abstract boolean hasMetric(String name); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericValuesSourceMetricsAggregatorParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericValuesSourceMetricsAggregatorParser.java index ae9e6844e2f..6847a9a5b3d 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericValuesSourceMetricsAggregatorParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/NumericValuesSourceMetricsAggregatorParser.java @@ -58,7 +58,8 @@ public abstract class NumericValuesSourceMetricsAggregatorParser metaData) throws IOException { - super(name,context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { @@ -103,12 +105,12 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue { if (valuesSource == null || bucket >= sums.size()) { return buildEmptyAggregation(); } - return new InternalAvg(name, sums.get(bucket), counts.get(bucket), formatter, metaData()); + return new InternalAvg(name, sums.get(bucket), counts.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalAvg(name, 0.0, 0l, formatter, metaData()); + return new InternalAvg(name, 0.0, 0l, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -118,13 +120,15 @@ public class AvgAggregator extends NumericMetricsAggregator.SingleValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new AvgAggregator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new AvgAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvg.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvg.java index 15980b45710..3f4af6cdf5c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvg.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/InternalAvg.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -57,8 +58,9 @@ public class InternalAvg extends InternalNumericMetricsAggregation.SingleValue i InternalAvg() {} // for serialization - public InternalAvg(String name, double sum, long count, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + public InternalAvg(String name, double sum, long count, @Nullable ValueFormatter formatter, List reducers, + Map metaData) { + super(name, reducers, metaData); this.sum = sum; this.count = count; this.valueFormatter = formatter; @@ -80,14 +82,14 @@ public class InternalAvg extends InternalNumericMetricsAggregation.SingleValue i } @Override - public InternalAvg reduce(List aggregations, ReduceContext reduceContext) { + public InternalAvg doReduce(List aggregations, ReduceContext reduceContext) { long count = 0; double sum = 0; for (InternalAggregation aggregation : aggregations) { count += ((InternalAvg) aggregation).count; sum += ((InternalAvg) aggregation).sum; } - return new InternalAvg(getName(), sum, count, valueFormatter, getMetaData()); + return new InternalAvg(getName(), sum, count, valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java index c241d690765..e00782687c9 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregator.java @@ -42,11 +42,13 @@ import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -66,8 +68,8 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue private ValueFormatter formatter; public CardinalityAggregator(String name, ValuesSource valuesSource, boolean rehash, int precision, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.rehash = rehash; this.precision = precision; @@ -156,12 +158,12 @@ public class CardinalityAggregator extends NumericMetricsAggregator.SingleValue // this Aggregator (and its HLL++ counters) is released. HyperLogLogPlusPlus copy = new HyperLogLogPlusPlus(precision, BigArrays.NON_RECYCLING_INSTANCE, 1); copy.merge(0, counts, owningBucketOrdinal); - return new InternalCardinality(name, copy, formatter, metaData()); + return new InternalCardinality(name, copy, formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalCardinality(name, null, formatter, metaData()); + return new InternalCardinality(name, null, formatter, reducers(), metaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java index 2d063dd5bd9..d2341bb2647 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregatorFactory.java @@ -22,12 +22,14 @@ package org.elasticsearch.search.aggregations.metrics.cardinality; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.bucket.SingleBucketAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import java.io.IOException; +import java.util.List; import java.util.Map; final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory { @@ -46,16 +48,19 @@ final class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory metaData) throws IOException { - return new CardinalityAggregator(name, null, true, precision(parent), config.formatter(), context, parent, metaData); + protected Aggregator createUnmapped(AggregationContext context, Aggregator parent, List reducers, Map metaData) + throws IOException { + return new CardinalityAggregator(name, null, true, precision(parent), config.formatter(), context, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource valuesSource, AggregationContext context, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { if (!(valuesSource instanceof ValuesSource.Numeric) && !rehash) { throw new AggregationExecutionException("Turning off rehashing for cardinality aggregation [" + name + "] on non-numeric values in not allowed"); } - return new CardinalityAggregator(name, valuesSource, rehash, precision(parent), config.formatter(), context, parent, metaData); + return new CardinalityAggregator(name, valuesSource, rehash, precision(parent), config.formatter(), context, parent, reducers, + metaData); } /* diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java index 57582476dc6..ee516e64313 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityParser.java @@ -62,10 +62,11 @@ public class CardinalityParser implements Aggregator.Parser { } else if (PRECISION_THRESHOLD.match(currentFieldName)) { precisionThreshold = parser.longValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + name + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + name + "]: [" + currentFieldName + + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + name + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + name + "].", parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinality.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinality.java index 9998d3eae10..5517702142c 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinality.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/InternalCardinality.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -53,8 +54,9 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation private HyperLogLogPlusPlus counts; - InternalCardinality(String name, HyperLogLogPlusPlus counts, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + InternalCardinality(String name, HyperLogLogPlusPlus counts, @Nullable ValueFormatter formatter, List reducers, + Map metaData) { + super(name, reducers, metaData); this.counts = counts; this.valueFormatter = formatter; } @@ -99,14 +101,14 @@ public final class InternalCardinality extends InternalNumericMetricsAggregation } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { InternalCardinality reduced = null; for (InternalAggregation aggregation : aggregations) { final InternalCardinality cardinality = (InternalCardinality) aggregation; if (cardinality.counts != null) { if (reduced == null) { reduced = new InternalCardinality(name, new HyperLogLogPlusPlus(cardinality.counts.precision(), - BigArrays.NON_RECYCLING_INSTANCE, 1), this.valueFormatter, getMetaData()); + BigArrays.NON_RECYCLING_INSTANCE, 1), this.valueFormatter, reducers(), getMetaData()); } reduced.merge(cardinality); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregator.java index 44e7fd195c0..464d0a339a8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregator.java @@ -30,12 +30,14 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import java.io.IOException; +import java.util.List; import java.util.Map; public final class GeoBoundsAggregator extends MetricsAggregator { @@ -49,9 +51,10 @@ public final class GeoBoundsAggregator extends MetricsAggregator { DoubleArray negLefts; DoubleArray negRights; - protected GeoBoundsAggregator(String name, AggregationContext aggregationContext, - Aggregator parent, ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, Map metaData) throws IOException { - super(name, aggregationContext, parent, metaData); + protected GeoBoundsAggregator(String name, AggregationContext aggregationContext, Aggregator parent, + ValuesSource.GeoPoint valuesSource, boolean wrapLongitude, List reducers, + Map metaData) throws IOException { + super(name, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.wrapLongitude = wrapLongitude; if (valuesSource != null) { @@ -149,13 +152,13 @@ public final class GeoBoundsAggregator extends MetricsAggregator { double posRight = posRights.get(owningBucketOrdinal); double negLeft = negLefts.get(owningBucketOrdinal); double negRight = negRights.get(owningBucketOrdinal); - return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, metaData()); + return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { return new InternalGeoBounds(name, Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.POSITIVE_INFINITY, - Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, metaData()); + Double.NEGATIVE_INFINITY, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, wrapLongitude, reducers(), metaData()); } @Override @@ -173,14 +176,15 @@ public final class GeoBoundsAggregator extends MetricsAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new GeoBoundsAggregator(name, aggregationContext, parent, null, wrapLongitude, reducers, metaData); } @Override protected Aggregator doCreateInternal(ValuesSource.GeoPoint valuesSource, AggregationContext aggregationContext, - Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, metaData); + Aggregator parent, boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new GeoBoundsAggregator(name, aggregationContext, parent, valuesSource, wrapLongitude, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsParser.java index d0f7da083c5..de1fea2f504 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsParser.java @@ -56,10 +56,12 @@ public class GeoBoundsParser implements Aggregator.Parser { if ("wrap_longitude".equals(currentFieldName) || "wrapLongitude".equals(currentFieldName)) { wrapLongitude = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in aggregation [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } return new GeoBoundsAggregator.Factory(aggregationName, vsParser.config(), wrapLongitude); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBounds.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBounds.java index 1d32f7b52eb..0da4e6bfcc7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBounds.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/InternalGeoBounds.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.List; @@ -55,8 +56,9 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo } InternalGeoBounds(String name, double top, double bottom, double posLeft, double posRight, - double negLeft, double negRight, boolean wrapLongitude, Map metaData) { - super(name, metaData); + double negLeft, double negRight, boolean wrapLongitude, + List reducers, Map metaData) { + super(name, reducers, metaData); this.top = top; this.bottom = bottom; this.posLeft = posLeft; @@ -72,7 +74,7 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { double top = Double.NEGATIVE_INFINITY; double bottom = Double.POSITIVE_INFINITY; double posLeft = Double.POSITIVE_INFINITY; @@ -102,7 +104,7 @@ public class InternalGeoBounds extends InternalMetricsAggregation implements Geo negRight = bounds.negRight; } } - return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, getMetaData()); + return new InternalGeoBounds(name, top, bottom, posLeft, posRight, negLeft, negRight, wrapLongitude, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/InternalMax.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/InternalMax.java index a3b938cf363..b2f103fc488 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/InternalMax.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/InternalMax.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -56,8 +57,8 @@ public class InternalMax extends InternalNumericMetricsAggregation.SingleValue i InternalMax() {} // for serialization - public InternalMax(String name, double max, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + public InternalMax(String name, double max, @Nullable ValueFormatter formatter, List reducers, Map metaData) { + super(name, reducers, metaData); this.valueFormatter = formatter; this.max = max; } @@ -78,12 +79,12 @@ public class InternalMax extends InternalNumericMetricsAggregation.SingleValue i } @Override - public InternalMax reduce(List aggregations, ReduceContext reduceContext) { + public InternalMax doReduce(List aggregations, ReduceContext reduceContext) { double max = Double.NEGATIVE_INFINITY; for (InternalAggregation aggregation : aggregations) { max = Math.max(max, ((InternalMax) aggregation).max); } - return new InternalMax(name, max, valueFormatter, getMetaData()); + return new InternalMax(name, max, valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java index 88edddc286c..7ade492660e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregator.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -38,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -51,8 +53,9 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { DoubleArray maxes; public MaxAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { @@ -71,7 +74,7 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { final LeafBucketCollector sub) throws IOException { if (valuesSource == null) { return LeafBucketCollector.NO_OP_COLLECTOR; - } + } final BigArrays bigArrays = context.bigArrays(); final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx); final NumericDoubleValues values = MultiValueMode.MAX.select(allValues, Double.NEGATIVE_INFINITY); @@ -103,12 +106,12 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { if (valuesSource == null || bucket >= maxes.size()) { return buildEmptyAggregation(); } - return new InternalMax(name, maxes.get(bucket), formatter, metaData()); + return new InternalMax(name, maxes.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, metaData()); + return new InternalMax(name, Double.NEGATIVE_INFINITY, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -118,13 +121,15 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new MaxAggregator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new MaxAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/InternalMin.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/InternalMin.java index 7beb9c5a76a..d6269602169 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/InternalMin.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/InternalMin.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -57,8 +58,8 @@ public class InternalMin extends InternalNumericMetricsAggregation.SingleValue i InternalMin() {} // for serialization - public InternalMin(String name, double min, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + public InternalMin(String name, double min, @Nullable ValueFormatter formatter, List reducers, Map metaData) { + super(name, reducers, metaData); this.min = min; this.valueFormatter = formatter; } @@ -79,12 +80,12 @@ public class InternalMin extends InternalNumericMetricsAggregation.SingleValue i } @Override - public InternalMin reduce(List aggregations, ReduceContext reduceContext) { + public InternalMin doReduce(List aggregations, ReduceContext reduceContext) { double min = Double.POSITIVE_INFINITY; for (InternalAggregation aggregation : aggregations) { min = Math.min(min, ((InternalMin) aggregation).min); } - return new InternalMin(getName(), min, this.valueFormatter, getMetaData()); + return new InternalMin(getName(), min, this.valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java index 438272e2bc1..cf832cabe1f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregator.java @@ -31,6 +31,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -38,6 +39,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -51,8 +53,9 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue { DoubleArray mins; public MinAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; if (valuesSource != null) { mins = context.bigArrays().newDoubleArray(1, false); @@ -103,12 +106,12 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue { if (valuesSource == null || bucket >= mins.size()) { return buildEmptyAggregation(); } - return new InternalMin(name, mins.get(bucket), formatter, metaData()); + return new InternalMin(name, mins.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, metaData()); + return new InternalMin(name, Double.POSITIVE_INFINITY, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -118,13 +121,15 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new MinAggregator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new MinAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractInternalPercentiles.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractInternalPercentiles.java index 87f12155e30..081e5e07fda 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractInternalPercentiles.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractInternalPercentiles.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -43,8 +44,9 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega AbstractInternalPercentiles() {} // for serialization public AbstractInternalPercentiles(String name, double[] keys, TDigestState state, boolean keyed, @Nullable ValueFormatter formatter, + List reducers, Map metaData) { - super(name, metaData); + super(name, reducers, metaData); this.keys = keys; this.state = state; this.keyed = keyed; @@ -59,7 +61,7 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega public abstract double value(double key); @Override - public AbstractInternalPercentiles reduce(List aggregations, ReduceContext reduceContext) { + public AbstractInternalPercentiles doReduce(List aggregations, ReduceContext reduceContext) { TDigestState merged = null; for (InternalAggregation aggregation : aggregations) { final AbstractInternalPercentiles percentiles = (AbstractInternalPercentiles) aggregation; @@ -68,10 +70,11 @@ abstract class AbstractInternalPercentiles extends InternalNumericMetricsAggrega } merged.add(percentiles.state); } - return createReduced(getName(), keys, merged, keyed, getMetaData()); + return createReduced(getName(), keys, merged, keyed, reducers(), getMetaData()); } - protected abstract AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map metaData); + protected abstract AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, + List reducers, Map metaData); @Override protected void doReadFrom(StreamInput in) throws IOException { diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java index 31a097f0b47..a73639a3d7f 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesAggregator.java @@ -31,11 +31,13 @@ import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; public abstract class AbstractPercentilesAggregator extends NumericMetricsAggregator.MultiValue { @@ -53,8 +55,9 @@ public abstract class AbstractPercentilesAggregator extends NumericMetricsAggreg public AbstractPercentilesAggregator(String name, ValuesSource.Numeric valuesSource, AggregationContext context, Aggregator parent, double[] keys, double compression, boolean keyed, - @Nullable ValueFormatter formatter, Map metaData) throws IOException { - super(name, context, parent, metaData); + @Nullable ValueFormatter formatter, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.keyed = keyed; this.formatter = formatter; diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesParser.java index 7fee395966b..8d12c9bf1a7 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/AbstractPercentilesParser.java @@ -69,22 +69,26 @@ public abstract class AbstractPercentilesParser implements Aggregator.Parser { keys = values.toArray(); Arrays.sort(keys); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_BOOLEAN) { if ("keyed".equals(currentFieldName)) { keyed = parser.booleanValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.VALUE_NUMBER) { if ("compression".equals(currentFieldName)) { compression = parser.doubleValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } return buildFactory(context, aggregationName, vsParser.config(), keys, compression, keyed); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentileRanks.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentileRanks.java index b096d479243..291db0451b0 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentileRanks.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentileRanks.java @@ -24,10 +24,12 @@ import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; import java.util.Iterator; +import java.util.List; import java.util.Map; /** @@ -53,8 +55,9 @@ public class InternalPercentileRanks extends AbstractInternalPercentiles impleme InternalPercentileRanks() {} // for serialization public InternalPercentileRanks(String name, double[] cdfValues, TDigestState state, boolean keyed, @Nullable ValueFormatter formatter, + List reducers, Map metaData) { - super(name, cdfValues, state, keyed, formatter, metaData); + super(name, cdfValues, state, keyed, formatter, reducers, metaData); } @Override @@ -78,8 +81,9 @@ public class InternalPercentileRanks extends AbstractInternalPercentiles impleme } @Override - protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map metaData) { - return new InternalPercentileRanks(name, keys, merged, keyed, valueFormatter, metaData); + protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, + List reducers, Map metaData) { + return new InternalPercentileRanks(name, keys, merged, keyed, valueFormatter, reducers, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentiles.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentiles.java index 94beb90a911..71360dad8fb 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentiles.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/InternalPercentiles.java @@ -24,10 +24,12 @@ import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; import java.util.Iterator; +import java.util.List; import java.util.Map; /** @@ -53,8 +55,9 @@ public class InternalPercentiles extends AbstractInternalPercentiles implements InternalPercentiles() {} // for serialization public InternalPercentiles(String name, double[] percents, TDigestState state, boolean keyed, @Nullable ValueFormatter formatter, + List reducers, Map metaData) { - super(name, percents, state, keyed, formatter, metaData); + super(name, percents, state, keyed, formatter, reducers, metaData); } @Override @@ -78,8 +81,9 @@ public class InternalPercentiles extends AbstractInternalPercentiles implements } @Override - protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, Map metaData) { - return new InternalPercentiles(name, keys, merged, keyed, valueFormatter, metaData); + protected AbstractInternalPercentiles createReduced(String name, double[] keys, TDigestState merged, boolean keyed, + List reducers, Map metaData) { + return new InternalPercentiles(name, keys, merged, keyed, valueFormatter, reducers, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java index 0383e33e8a7..380482b8ab3 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregator.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -30,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -37,10 +39,10 @@ import java.util.Map; */ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { - public PercentileRanksAggregator(String name, Numeric valuesSource, AggregationContext context, - Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter, - Map metaData) throws IOException { - super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); + public PercentileRanksAggregator(String name, Numeric valuesSource, AggregationContext context, Aggregator parent, double[] percents, + double compression, boolean keyed, @Nullable ValueFormatter formatter, List reducers, Map metaData) + throws IOException { + super(name, valuesSource, context, parent, percents, compression, keyed, formatter, reducers, metaData); } @Override @@ -49,13 +51,13 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { if (state == null) { return buildEmptyAggregation(); } else { - return new InternalPercentileRanks(name, keys, state, keyed, formatter, metaData()); + return new InternalPercentileRanks(name, keys, state, keyed, formatter, reducers(), metaData()); } } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed, formatter, metaData()); + return new InternalPercentileRanks(name, keys, new TDigestState(compression), keyed, formatter, reducers(), metaData()); } @Override @@ -83,15 +85,18 @@ public class PercentileRanksAggregator extends AbstractPercentilesAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { return new PercentileRanksAggregator(name, null, aggregationContext, parent, values, compression, keyed, config.formatter(), + reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { return new PercentileRanksAggregator(name, valuesSource, aggregationContext, parent, values, compression, - keyed, config.formatter(), metaData); + keyed, config.formatter(), reducers, metaData); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksParser.java index d0dd632314e..7d27c5556da 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksParser.java @@ -46,7 +46,7 @@ public class PercentileRanksParser extends AbstractPercentilesParser { @Override protected AggregatorFactory buildFactory(SearchContext context, String aggregationName, ValuesSourceConfig valuesSourceConfig, double[] keys, double compression, boolean keyed) { if (keys == null) { - throw new SearchParseException(context, "Missing token values in [" + aggregationName + "]."); + throw new SearchParseException(context, "Missing token values in [" + aggregationName + "].", null); } return new PercentileRanksAggregator.Factory(aggregationName, valuesSourceConfig, keys, compression, keyed); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java index 4dd99b73cd9..2a42dc94620 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregator.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.inject.internal.Nullable; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.percentiles.tdigest.TDigestState; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; @@ -30,6 +31,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -38,9 +40,10 @@ import java.util.Map; public class PercentilesAggregator extends AbstractPercentilesAggregator { public PercentilesAggregator(String name, Numeric valuesSource, AggregationContext context, - Aggregator parent, double[] percents, double compression, boolean keyed, @Nullable ValueFormatter formatter, + Aggregator parent, double[] percents, + double compression, boolean keyed, @Nullable ValueFormatter formatter, List reducers, Map metaData) throws IOException { - super(name, valuesSource, context, parent, percents, compression, keyed, formatter, metaData); + super(name, valuesSource, context, parent, percents, compression, keyed, formatter, reducers, metaData); } @Override @@ -49,7 +52,7 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator { if (state == null) { return buildEmptyAggregation(); } else { - return new InternalPercentiles(name, keys, state, keyed, formatter, metaData()); + return new InternalPercentiles(name, keys, state, keyed, formatter, reducers(), metaData()); } } @@ -65,7 +68,7 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator { @Override public InternalAggregation buildEmptyAggregation() { - return new InternalPercentiles(name, keys, new TDigestState(compression), keyed, formatter, metaData()); + return new InternalPercentiles(name, keys, new TDigestState(compression), keyed, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -83,15 +86,18 @@ public class PercentilesAggregator extends AbstractPercentilesAggregator { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { return new PercentilesAggregator(name, null, aggregationContext, parent, percents, compression, keyed, config.formatter(), + reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { return new PercentilesAggregator(name, valuesSource, aggregationContext, parent, percents, compression, - keyed, config.formatter(), metaData); + keyed, config.formatter(), reducers, metaData); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetric.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetric.java index 6a076678d4f..4f80befc440 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetric.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/InternalScriptedMetric.java @@ -29,6 +29,7 @@ import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import java.io.IOException; import java.util.ArrayList; @@ -62,13 +63,13 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement private InternalScriptedMetric() { } - private InternalScriptedMetric(String name, Map metaData) { - super(name, metaData); + private InternalScriptedMetric(String name, List reducers, Map metaData) { + super(name, reducers, metaData); } public InternalScriptedMetric(String name, Object aggregation, String scriptLang, ScriptType scriptType, String reduceScript, - Map reduceParams, Map metaData) { - this(name, metaData); + Map reduceParams, List reducers, Map metaData) { + this(name, reducers, metaData); this.aggregation = aggregation; this.scriptType = scriptType; this.reduceScript = reduceScript; @@ -82,7 +83,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { List aggregationObjects = new ArrayList<>(); for (InternalAggregation aggregation : aggregations) { InternalScriptedMetric mapReduceAggregation = (InternalScriptedMetric) aggregation; @@ -105,7 +106,7 @@ public class InternalScriptedMetric extends InternalMetricsAggregation implement aggregation = aggregationObjects; } return new InternalScriptedMetric(firstAggregation.getName(), aggregation, firstAggregation.scriptLang, firstAggregation.scriptType, - firstAggregation.reduceScript, firstAggregation.reduceParams, getMetaData()); + firstAggregation.reduceScript, firstAggregation.reduceParams, reducers(), getMetaData()); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java index 9d52242b7bf..6ac07a6fedc 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregator.java @@ -22,16 +22,19 @@ package org.elasticsearch.search.aggregations.metrics.scripted; import org.apache.lucene.index.LeafReaderContext; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.LeafSearchScript; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.script.*; import org.elasticsearch.script.ScriptService.ScriptType; +import org.elasticsearch.script.SearchScript; import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactory; -import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; @@ -57,8 +60,9 @@ public class ScriptedMetricAggregator extends MetricsAggregator { protected ScriptedMetricAggregator(String name, String scriptLang, ScriptType initScriptType, String initScript, ScriptType mapScriptType, String mapScript, ScriptType combineScriptType, String combineScript, ScriptType reduceScriptType, - String reduceScript, Map params, Map reduceParams, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + String reduceScript, Map params, Map reduceParams, AggregationContext context, + Aggregator parent, List reducers, Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.scriptLang = scriptLang; this.reduceScriptType = reduceScriptType; if (params == null) { @@ -112,12 +116,13 @@ public class ScriptedMetricAggregator extends MetricsAggregator { } else { aggregation = params.get("_agg"); } - return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams, metaData()); + return new InternalScriptedMetric(name, aggregation, scriptLang, reduceScriptType, reduceScript, reduceParams, reducers(), + metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams, metaData()); + return new InternalScriptedMetric(name, null, scriptLang, reduceScriptType, reduceScript, reduceParams, reducers(), metaData()); } public static class Factory extends AggregatorFactory { @@ -151,7 +156,8 @@ public class ScriptedMetricAggregator extends MetricsAggregator { } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { if (collectsFromSingleBucket == false) { return asMultiBucketAggregator(this, context, parent); } @@ -164,7 +170,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator { reduceParams = deepCopyParams(this.reduceParams, context.searchContext()); } return new ScriptedMetricAggregator(name, scriptLang, initScriptType, initScript, mapScriptType, mapScript, combineScriptType, - combineScript, reduceScriptType, reduceScript, params, reduceParams, context, parent, metaData); + combineScript, reduceScriptType, reduceScript, params, reduceParams, context, parent, reducers, metaData); } @SuppressWarnings({ "unchecked" }) @@ -190,7 +196,7 @@ public class ScriptedMetricAggregator extends MetricsAggregator { clone = original; } else { throw new SearchParseException(context, "Can only clone primitives, String, ArrayList, and HashMap. Found: " - + original.getClass().getCanonicalName()); + + original.getClass().getCanonicalName(), null); } return clone; } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricParser.java index 1b0b5aa3290..c37d0aaccf8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricParser.java @@ -72,14 +72,17 @@ public class ScriptedMetricParser implements Aggregator.Parser { } else if (REDUCE_PARAMS_FIELD.match(currentFieldName)) { reduceParams = parser.map(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token.isValue()) { if (!scriptParameterParser.token(currentFieldName, token, parser)) { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } @@ -114,7 +117,7 @@ public class ScriptedMetricParser implements Aggregator.Parser { scriptLang = scriptParameterParser.lang(); if (mapScript == null) { - throw new SearchParseException(context, "map_script field is required in [" + aggregationName + "]."); + throw new SearchParseException(context, "map_script field is required in [" + aggregationName + "].", parser.getTokenLocation()); } return new ScriptedMetricAggregator.Factory(aggregationName, scriptLang, initScriptType, initScript, mapScriptType, mapScript, combineScriptType, combineScript, reduceScriptType, reduceScript, params, reduceParams); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/InternalStats.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/InternalStats.java index 684dae5afdf..c7b68da9a88 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/InternalStats.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/InternalStats.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -70,8 +71,9 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue protected InternalStats() {} // for serialization public InternalStats(String name, long count, double sum, double min, double max, @Nullable ValueFormatter formatter, + List reducers, Map metaData) { - super(name, metaData); + super(name, reducers, metaData); this.count = count; this.sum = sum; this.min = min; @@ -149,7 +151,7 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue } @Override - public InternalStats reduce(List aggregations, ReduceContext reduceContext) { + public InternalStats doReduce(List aggregations, ReduceContext reduceContext) { long count = 0; double min = Double.POSITIVE_INFINITY; double max = Double.NEGATIVE_INFINITY; @@ -161,7 +163,7 @@ public class InternalStats extends InternalNumericMetricsAggregation.MultiValue max = Math.max(max, stats.getMax()); sum += stats.getSum(); } - return new InternalStats(name, count, sum, min, max, valueFormatter, getMetaData()); + return new InternalStats(name, count, sum, min, max, valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java index ca8168bc87c..cf9356fee1e 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggegator.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -37,6 +38,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -54,8 +56,9 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue { public StatsAggegator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; if (valuesSource != null) { final BigArrays bigArrays = context.bigArrays(); @@ -144,12 +147,12 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue { return buildEmptyAggregation(); } return new InternalStats(name, counts.get(bucket), sums.get(bucket), mins.get(bucket), - maxes.get(bucket), formatter, metaData()); + maxes.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, metaData()); + return new InternalStats(name, 0, 0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -159,13 +162,15 @@ public class StatsAggegator extends NumericMetricsAggregator.MultiValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new StatsAggegator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new StatsAggegator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new StatsAggegator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new StatsAggegator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java index b8cca425ab4..164fb86d00a 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregator.java @@ -30,6 +30,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -37,6 +38,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -54,10 +56,10 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue DoubleArray maxes; DoubleArray sumOfSqrs; - public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource, - @Nullable ValueFormatter formatter, AggregationContext context, - Aggregator parent, double sigma, Map metaData) throws IOException { - super(name, context, parent, metaData); + public ExtendedStatsAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, + AggregationContext context, Aggregator parent, double sigma, List reducers, Map metaData) + throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.formatter = formatter; this.sigma = sigma; @@ -166,16 +168,19 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue @Override public InternalAggregation buildAggregation(long owningBucketOrdinal) { if (valuesSource == null) { - return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, 0d, formatter, metaData()); + return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, 0d, formatter, + reducers(), metaData()); } assert owningBucketOrdinal < counts.size(); return new InternalExtendedStats(name, counts.get(owningBucketOrdinal), sums.get(owningBucketOrdinal), - mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal), sigma, formatter, metaData()); + mins.get(owningBucketOrdinal), maxes.get(owningBucketOrdinal), sumOfSqrs.get(owningBucketOrdinal), sigma, formatter, + reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, 0d, formatter, metaData()); + return new InternalExtendedStats(name, 0, 0d, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, 0d, 0d, formatter, reducers(), + metaData()); } @Override @@ -194,13 +199,16 @@ public class ExtendedStatsAggregator extends NumericMetricsAggregator.MultiValue } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new ExtendedStatsAggregator(name, null, config.formatter(), aggregationContext, parent, sigma, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new ExtendedStatsAggregator(name, null, config.formatter(), aggregationContext, parent, sigma, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new ExtendedStatsAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, sigma, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new ExtendedStatsAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, sigma, reducers, + metaData); } } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsParser.java index 18ca93495c3..ea48e4b11f8 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsParser.java @@ -65,15 +65,17 @@ public class ExtendedStatsParser implements Aggregator.Parser { if (SIGMA.match(currentFieldName)) { sigma = parser.doubleValue(); } else { - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } if (sigma < 0) { - throw new SearchParseException(context, "[sigma] must not be negative. Value provided was" + sigma ); + throw new SearchParseException(context, "[sigma] must not be negative. Value provided was" + sigma, parser.getTokenLocation()); } return createFactory(aggregationName, vsParser.config(), sigma); diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/InternalExtendedStats.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/InternalExtendedStats.java index 86051ba6320..7d1568156f5 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/InternalExtendedStats.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/InternalExtendedStats.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.stats.InternalStats; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; @@ -68,8 +69,8 @@ public class InternalExtendedStats extends InternalStats implements ExtendedStat InternalExtendedStats() {} // for serialization public InternalExtendedStats(String name, long count, double sum, double min, double max, double sumOfSqrs, - double sigma, @Nullable ValueFormatter formatter, Map metaData) { - super(name, count, sum, min, max, formatter, metaData); + double sigma, @Nullable ValueFormatter formatter, List reducers, Map metaData) { + super(name, count, sum, min, max, formatter, reducers, metaData); this.sumOfSqrs = sumOfSqrs; this.sigma = sigma; } @@ -144,14 +145,15 @@ public class InternalExtendedStats extends InternalStats implements ExtendedStat } @Override - public InternalExtendedStats reduce(List aggregations, ReduceContext reduceContext) { + public InternalExtendedStats doReduce(List aggregations, ReduceContext reduceContext) { double sumOfSqrs = 0; for (InternalAggregation aggregation : aggregations) { InternalExtendedStats stats = (InternalExtendedStats) aggregation; sumOfSqrs += stats.getSumOfSquares(); } - final InternalStats stats = super.reduce(aggregations, reduceContext); - return new InternalExtendedStats(name, stats.getCount(), stats.getSum(), stats.getMin(), stats.getMax(), sumOfSqrs, sigma, valueFormatter, getMetaData()); + final InternalStats stats = super.doReduce(aggregations, reduceContext); + return new InternalExtendedStats(name, stats.getCount(), stats.getSum(), stats.getMin(), stats.getMax(), sumOfSqrs, sigma, + valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSum.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSum.java index 7eb7e789710..00c8050e714 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSum.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/InternalSum.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; @@ -56,8 +57,8 @@ public class InternalSum extends InternalNumericMetricsAggregation.SingleValue i InternalSum() {} // for serialization - InternalSum(String name, double sum, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + InternalSum(String name, double sum, @Nullable ValueFormatter formatter, List reducers, Map metaData) { + super(name, reducers, metaData); this.sum = sum; this.valueFormatter = formatter; } @@ -78,12 +79,12 @@ public class InternalSum extends InternalNumericMetricsAggregation.SingleValue i } @Override - public InternalSum reduce(List aggregations, ReduceContext reduceContext) { + public InternalSum doReduce(List aggregations, ReduceContext reduceContext) { double sum = 0; for (InternalAggregation aggregation : aggregations) { sum += ((InternalSum) aggregation).sum; } - return new InternalSum(name, sum, valueFormatter, getMetaData()); + return new InternalSum(name, sum, valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java index ab6b565a62b..4c7981422f3 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregator.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -36,6 +37,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,8 +51,9 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue { DoubleArray sums; public SumAggregator(String name, ValuesSource.Numeric valuesSource, @Nullable ValueFormatter formatter, - AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + AggregationContext context, Aggregator parent, List reducers, + Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { @@ -96,12 +99,12 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue { if (valuesSource == null || bucket >= sums.size()) { return buildEmptyAggregation(); } - return new InternalSum(name, sums.get(bucket), formatter, metaData()); + return new InternalSum(name, sums.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalSum(name, 0.0, formatter, metaData()); + return new InternalSum(name, 0.0, formatter, reducers(), metaData()); } public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly { @@ -111,13 +114,15 @@ public class SumAggregator extends NumericMetricsAggregator.SingleValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new SumAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new SumAggregator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new SumAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator doCreateInternal(ValuesSource.Numeric valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new SumAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java index 562a6738667..2ae1d3d50c9 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/InternalTopHits.java @@ -31,11 +31,13 @@ import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.internal.InternalSearchHit; import org.elasticsearch.search.internal.InternalSearchHits; import java.io.IOException; import java.util.List; +import java.util.Map; /** */ @@ -64,15 +66,15 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi InternalTopHits() { } - public InternalTopHits(String name, int from, int size, TopDocs topDocs, InternalSearchHits searchHits) { - this.name = name; + public InternalTopHits(String name, int from, int size, TopDocs topDocs, InternalSearchHits searchHits, List reducers, + Map metaData) { + super(name, reducers, metaData); this.from = from; this.size = size; this.topDocs = topDocs; this.searchHits = searchHits; } - @Override public Type type() { return TYPE; @@ -84,7 +86,7 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { InternalSearchHits[] shardHits = new InternalSearchHits[aggregations.size()]; final TopDocs reducedTopDocs; @@ -120,7 +122,9 @@ public class InternalTopHits extends InternalMetricsAggregation implements TopHi } while (shardDocs[scoreDoc.shardIndex].scoreDocs[position] != scoreDoc); hits[i] = (InternalSearchHit) shardHits[scoreDoc.shardIndex].getAt(position); } - return new InternalTopHits(name, from, size, reducedTopDocs, new InternalSearchHits(hits, reducedTopDocs.totalHits, reducedTopDocs.getMaxScore())); + return new InternalTopHits(name, from, size, reducedTopDocs, new InternalSearchHits(hits, reducedTopDocs.totalHits, + reducedTopDocs.getMaxScore()), + reducers(), getMetaData()); } catch (IOException e) { throw ExceptionsHelper.convertToElastic(e); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java index 52b2c233cde..1f211471457 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregator.java @@ -37,10 +37,12 @@ import org.elasticsearch.search.aggregations.AggregationInitializationException; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.metrics.MetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -49,6 +51,7 @@ import org.elasticsearch.search.internal.InternalSearchHits; import org.elasticsearch.search.internal.SubSearchContext; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -69,8 +72,9 @@ public class TopHitsAggregator extends MetricsAggregator { final SubSearchContext subSearchContext; final LongObjectPagedHashMap topDocsCollectors; - public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, Aggregator parent, Map metaData) throws IOException { - super(name, context, parent, metaData); + public TopHitsAggregator(FetchPhase fetchPhase, SubSearchContext subSearchContext, String name, AggregationContext context, + Aggregator parent, List reducers, Map metaData) throws IOException { + super(name, context, parent, reducers, metaData); this.fetchPhase = fetchPhase; topDocsCollectors = new LongObjectPagedHashMap<>(1, context.bigArrays()); this.subSearchContext = subSearchContext; @@ -153,7 +157,8 @@ public class TopHitsAggregator extends MetricsAggregator { searchHitFields.sortValues(fieldDoc.fields); } } - topHits = new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, fetchResult.hits()); + topHits = new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, fetchResult.hits(), reducers(), + metaData()); } return topHits; } @@ -166,7 +171,7 @@ public class TopHitsAggregator extends MetricsAggregator { } else { topDocs = Lucene.EMPTY_TOP_DOCS; } - return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, InternalSearchHits.empty()); + return new InternalTopHits(name, subSearchContext.from(), subSearchContext.size(), topDocs, InternalSearchHits.empty(), reducers(), metaData()); } @Override @@ -186,8 +191,9 @@ public class TopHitsAggregator extends MetricsAggregator { } @Override - public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new TopHitsAggregator(fetchPhase, subSearchContext, name, aggregationContext, parent, metaData); + public Aggregator createInternal(AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { + return new TopHitsAggregator(fetchPhase, subSearchContext, name, aggregationContext, parent, reducers, metaData); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsParser.java index 6300374663b..206587ac6a4 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsParser.java @@ -94,7 +94,8 @@ public class TopHitsParser implements Aggregator.Parser { subSearchContext.explain(parser.booleanValue()); break; default: - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_OBJECT) { switch (currentFieldName) { @@ -106,7 +107,8 @@ public class TopHitsParser implements Aggregator.Parser { scriptFieldsParseElement.parse(parser, subSearchContext); break; default: - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else if (token == XContentParser.Token.START_ARRAY) { switch (currentFieldName) { @@ -115,10 +117,12 @@ public class TopHitsParser implements Aggregator.Parser { fieldDataFieldsParseElement.parse(parser, subSearchContext); break; default: - throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + currentFieldName + "]."); + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); } } else { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } } catch (Exception e) { diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCount.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCount.java index 8c6dbf9b038..9ebb7e534ea 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCount.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/InternalValueCount.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregationStreams; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; @@ -55,8 +56,9 @@ public class InternalValueCount extends InternalNumericMetricsAggregation.Single InternalValueCount() {} // for serialization - public InternalValueCount(String name, long value, @Nullable ValueFormatter formatter, Map metaData) { - super(name, metaData); + public InternalValueCount(String name, long value, @Nullable ValueFormatter formatter, List reducers, + Map metaData) { + super(name, reducers, metaData); this.value = value; this.valueFormatter = formatter; } @@ -77,12 +79,12 @@ public class InternalValueCount extends InternalNumericMetricsAggregation.Single } @Override - public InternalAggregation reduce(List aggregations, ReduceContext reduceContext) { + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { long valueCount = 0; for (InternalAggregation aggregation : aggregations) { valueCount += ((InternalValueCount) aggregation).value; } - return new InternalValueCount(name, valueCount, valueFormatter, getMetaData()); + return new InternalValueCount(name, valueCount, valueFormatter, reducers(), getMetaData()); } @Override diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java index a74ec061b8e..fedd7e09a2b 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregator.java @@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; @@ -36,6 +37,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.format.ValueFormatter; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -53,8 +55,9 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { LongArray counts; public ValueCountAggregator(String name, ValuesSource valuesSource, @Nullable ValueFormatter formatter, - AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - super(name, aggregationContext, parent, metaData); + AggregationContext aggregationContext, Aggregator parent, List reducers, Map metaData) + throws IOException { + super(name, aggregationContext, parent, reducers, metaData); this.valuesSource = valuesSource; this.formatter = formatter; if (valuesSource != null) { @@ -92,12 +95,12 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { if (valuesSource == null || bucket >= counts.size()) { return buildEmptyAggregation(); } - return new InternalValueCount(name, counts.get(bucket), formatter, metaData()); + return new InternalValueCount(name, counts.get(bucket), formatter, reducers(), metaData()); } @Override public InternalAggregation buildEmptyAggregation() { - return new InternalValueCount(name, 0l, formatter, metaData()); + return new InternalValueCount(name, 0l, formatter, reducers(), metaData()); } @Override @@ -112,13 +115,15 @@ public class ValueCountAggregator extends NumericMetricsAggregator.SingleValue { } @Override - protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException { - return new ValueCountAggregator(name, null, config.formatter(), aggregationContext, parent, metaData); + protected Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException { + return new ValueCountAggregator(name, null, config.formatter(), aggregationContext, parent, reducers, metaData); } @Override - protected Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { - return new ValueCountAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, + protected Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException { + return new ValueCountAggregator(name, valuesSource, config.formatter(), aggregationContext, parent, reducers, metaData); } diff --git a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java index fb1d31f41f0..764f6ce9384 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountParser.java @@ -49,7 +49,8 @@ public class ValueCountParser implements Aggregator.Parser { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (!vsParser.token(currentFieldName, token, parser)) { - throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "]."); + throw new SearchParseException(context, "Unexpected token " + token + " in [" + aggregationName + "].", + parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/BucketHelpers.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/BucketHelpers.java new file mode 100644 index 00000000000..9eb70d860e5 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/BucketHelpers.java @@ -0,0 +1,187 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentLocation; +import org.elasticsearch.search.SearchParseException; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.InvalidAggregationPathException; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeParser; +import org.elasticsearch.search.aggregations.support.AggregationPath; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * A set of static helpers to simplify working with aggregation buckets, in particular + * providing utilities that help reducers. + */ +public class BucketHelpers { + + /** + * A gap policy determines how "holes" in a set of buckets should be handled. For example, + * a date_histogram might have empty buckets due to no data existing for that time interval. + * This can cause problems for operations like a derivative, which relies on a continuous + * function. + * + * "insert_zeros": empty buckets will be filled with zeros for all metrics + * "ignore": empty buckets will simply be ignored + */ + public static enum GapPolicy { + INSERT_ZEROS((byte) 0, "insert_zeros"), SKIP((byte) 1, "skip"); + + /** + * Parse a string GapPolicy into the byte enum + * + * @param context SearchContext this is taking place in + * @param text GapPolicy in string format (e.g. "ignore") + * @return GapPolicy enum + */ + public static GapPolicy parse(SearchContext context, String text, XContentLocation tokenLocation) { + GapPolicy result = null; + for (GapPolicy policy : values()) { + if (policy.parseField.match(text)) { + if (result == null) { + result = policy; + } else { + throw new IllegalStateException("Text can be parsed to 2 different gap policies: text=[" + text + + "], " + "policies=" + Arrays.asList(result, policy)); + } + } + } + if (result == null) { + final List validNames = new ArrayList<>(); + for (GapPolicy policy : values()) { + validNames.add(policy.getName()); + } + throw new SearchParseException(context, "Invalid gap policy: [" + text + "], accepted values: " + validNames, tokenLocation); + } + return result; + } + + private final byte id; + private final ParseField parseField; + + private GapPolicy(byte id, String name) { + this.id = id; + this.parseField = new ParseField(name); + } + + /** + * Serialize the GapPolicy to the output stream + * + * @param out + * @throws IOException + */ + public void writeTo(StreamOutput out) throws IOException { + out.writeByte(id); + } + + /** + * Deserialize the GapPolicy from the input stream + * + * @param in + * @return GapPolicy Enum + * @throws IOException + */ + public static GapPolicy readFrom(StreamInput in) throws IOException { + byte id = in.readByte(); + for (GapPolicy gapPolicy : values()) { + if (id == gapPolicy.id) { + return gapPolicy; + } + } + throw new IllegalStateException("Unknown GapPolicy with id [" + id + "]"); + } + + /** + * Return the english-formatted name of the GapPolicy + * + * @return English representation of GapPolicy + */ + public String getName() { + return parseField.getPreferredName(); + } + } + + /** + * Given a path and a set of buckets, this method will return the value inside the agg at + * that path. This is used to extract values for use by reducers (e.g. a derivative might need + * the price for each bucket). If the bucket is empty, the configured GapPolicy is invoked to + * resolve the missing bucket + * + * @param histo A series of agg buckets in the form of a histogram + * @param bucket A specific bucket that a value needs to be extracted from. This bucket should be present + * in the histo parameter + * @param aggPath The path to a particular value that needs to be extracted. This path should point to a metric + * inside the bucket + * @param gapPolicy The gap policy to apply if empty buckets are found + * @return The value extracted from bucket found at aggPath + */ + public static Double resolveBucketValue(InternalMultiBucketAggregation agg, + InternalMultiBucketAggregation.Bucket bucket, String aggPath, GapPolicy gapPolicy) { + List aggPathsList = AggregationPath.parse(aggPath).getPathElementsAsStringList(); + return resolveBucketValue(agg, bucket, aggPathsList, gapPolicy); + } + + public static Double resolveBucketValue(InternalMultiBucketAggregation agg, + InternalMultiBucketAggregation.Bucket bucket, List aggPathAsList, GapPolicy gapPolicy) { + try { + Object propertyValue = bucket.getProperty(agg.getName(), aggPathAsList); + if (propertyValue == null) { + throw new AggregationExecutionException(DerivativeParser.BUCKETS_PATH.getPreferredName() + + " must reference either a number value or a single value numeric metric aggregation"); + } else { + double value; + if (propertyValue instanceof Number) { + value = ((Number) propertyValue).doubleValue(); + } else if (propertyValue instanceof InternalNumericMetricsAggregation.SingleValue) { + value = ((InternalNumericMetricsAggregation.SingleValue) propertyValue).value(); + } else { + throw new AggregationExecutionException(DerivativeParser.BUCKETS_PATH.getPreferredName() + + " must reference either a number value or a single value numeric metric aggregation"); + } + // doc count never has missing values so gap policy doesn't apply here + boolean isDocCountProperty = aggPathAsList.size() == 1 && "_count".equals(aggPathAsList.get(0)); + if (Double.isInfinite(value) || Double.isNaN(value) || (bucket.getDocCount() == 0 && !isDocCountProperty)) { + switch (gapPolicy) { + case INSERT_ZEROS: + return 0.0; + case SKIP: + default: + return Double.NaN; + } + } else { + return value; + } + } + } catch (InvalidAggregationPathException e) { + return null; + } + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/InternalSimpleValue.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/InternalSimpleValue.java new file mode 100644 index 00000000000..2106f3247e1 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/InternalSimpleValue.java @@ -0,0 +1,104 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.common.inject.internal.Nullable; +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.AggregationStreams; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.metrics.max.InternalMax; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class InternalSimpleValue extends InternalNumericMetricsAggregation.SingleValue implements SimpleValue { + + public final static Type TYPE = new Type("simple_value"); + + public final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() { + @Override + public InternalSimpleValue readResult(StreamInput in) throws IOException { + InternalSimpleValue result = new InternalSimpleValue(); + result.readFrom(in); + return result; + } + }; + + public static void registerStreams() { + AggregationStreams.registerStream(STREAM, TYPE.stream()); + } + + private double value; + + InternalSimpleValue() {} // for serialization + + public InternalSimpleValue(String name, double value, @Nullable ValueFormatter formatter, List reducers, Map metaData) { + super(name, reducers, metaData); + this.valueFormatter = formatter; + this.value = value; + } + + @Override + public double value() { + return value; + } + + public double getValue() { + return value; + } + + @Override + public Type type() { + return TYPE; + } + + @Override + public InternalMax doReduce(List aggregations, ReduceContext reduceContext) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + protected void doReadFrom(StreamInput in) throws IOException { + valueFormatter = ValueFormatterStreams.readOptional(in); + value = in.readDouble(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + ValueFormatterStreams.writeOptional(valueFormatter, out); + out.writeDouble(value); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + boolean hasValue = !(Double.isInfinite(value) || Double.isNaN(value)); + builder.field(CommonFields.VALUE, hasValue ? value : null); + if (hasValue && valueFormatter != null) { + builder.field(CommonFields.VALUE_AS_STRING, valueFormatter.format(value)); + } + return builder; + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/Reducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/Reducer.java new file mode 100644 index 00000000000..8daa4d6180a --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/Reducer.java @@ -0,0 +1,130 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import com.google.common.base.Function; + +import org.elasticsearch.common.ParseField; +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.XContentParser; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Map; + +public abstract class Reducer implements Streamable { + + /** + * Parses the reducer request and creates the appropriate reducer factory + * for it. + * + * @see {@link ReducerFactory} + */ + public static interface Parser { + + public static final ParseField BUCKETS_PATH = new ParseField("buckets_path"); + + public static final ParseField FORMAT = new ParseField("format"); + public static final ParseField GAP_POLICY = new ParseField("gap_policy"); + + /** + * @return The reducer type this parser is associated with. + */ + String type(); + + /** + * Returns the reducer factory with which this parser is associated. + * + * @param reducerName + * The name of the reducer + * @param parser + * The xcontent parser + * @param context + * The search context + * @return The resolved reducer factory + * @throws java.io.IOException + * When parsing fails + */ + ReducerFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException; + + } + + public static final Function AGGREGATION_TRANFORM_FUNCTION = new Function() { + @Override + public InternalAggregation apply(Aggregation input) { + return (InternalAggregation) input; + } + }; + + private String name; + private String[] bucketsPaths; + private Map metaData; + + protected Reducer() { // for Serialisation + } + + protected Reducer(String name, String[] bucketsPaths, Map metaData) { + this.name = name; + this.bucketsPaths = bucketsPaths; + this.metaData = metaData; + } + + public String name() { + return name; + } + + public String[] bucketsPaths() { + return bucketsPaths; + } + + public Map metaData() { + return metaData; + } + + public abstract Type type(); + + public abstract InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext); + + @Override + public final void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeStringArray(bucketsPaths); + out.writeMap(metaData); + doWriteTo(out); + } + + protected abstract void doWriteTo(StreamOutput out) throws IOException; + + @Override + public final void readFrom(StreamInput in) throws IOException { + name = in.readString(); + bucketsPaths = in.readStringArray(); + metaData = in.readMap(); + doReadFrom(in); + } + + protected abstract void doReadFrom(StreamInput in) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilder.java new file mode 100644 index 00000000000..4dee8ea96a2 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilder.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; + +import java.io.IOException; +import java.util.Map; + +/** + * A base class for all reducer builders. + */ +public abstract class ReducerBuilder> extends AbstractAggregationBuilder { + + private String[] bucketsPaths; + private Map metaData; + + /** + * Sole constructor, typically used by sub-classes. + */ + protected ReducerBuilder(String name, String type) { + super(name, type); + } + + /** + * Sets the paths to the buckets to use for this reducer + */ + public B setBucketsPaths(String... bucketsPaths) { + this.bucketsPaths = bucketsPaths; + return (B) this; + } + + /** + * Sets the meta data to be included in the reducer's response + */ + public B setMetaData(Map metaData) { + this.metaData = metaData; + return (B)this; + } + + @Override + public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(getName()); + + if (this.metaData != null) { + builder.field("meta", this.metaData); + } + builder.startObject(type); + + if (bucketsPaths != null) { + builder.startArray(Reducer.Parser.BUCKETS_PATH.getPreferredName()); + for (String path : bucketsPaths) { + builder.value(path); + } + builder.endArray(); + } + + internalXContent(builder, params); + + builder.endObject(); + + return builder.endObject(); + } + + protected abstract XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException; +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java new file mode 100644 index 00000000000..ba6d3ebe7c2 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerBuilders.java @@ -0,0 +1,42 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.MaxBucketBuilder; +import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeBuilder; +import org.elasticsearch.search.aggregations.reducers.movavg.MovAvgBuilder; + +public final class ReducerBuilders { + + private ReducerBuilders() { + } + + public static final DerivativeBuilder derivative(String name) { + return new DerivativeBuilder(name); + } + + public static final MaxBucketBuilder maxBucket(String name) { + return new MaxBucketBuilder(name); + } + + public static final MovAvgBuilder movingAvg(String name) { + return new MovAvgBuilder(name); + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerFactory.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerFactory.java new file mode 100644 index 00000000000..46ac844808c --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerFactory.java @@ -0,0 +1,102 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * A factory that knows how to create an {@link Aggregator} of a specific type. + */ +public abstract class ReducerFactory { + + protected String name; + protected String type; + protected String[] bucketsPaths; + protected Map metaData; + + /** + * Constructs a new reducer factory. + * + * @param name + * The aggregation name + * @param type + * The aggregation type + */ + public ReducerFactory(String name, String type, String[] bucketsPaths) { + this.name = name; + this.type = type; + this.bucketsPaths = bucketsPaths; + } + + /** + * Validates the state of this factory (makes sure the factory is properly + * configured) + * + * @param reducerFactories + * @param factories + * @param parent + */ + public final void validate(AggregatorFactory parent, AggregatorFactory[] factories, List reducerFactories) { + doValidate(parent, factories, reducerFactories); + } + + protected abstract Reducer createInternal(Map metaData) throws IOException; + + /** + * Creates the reducer + * + * @param context + * The aggregation context + * @param parent + * The parent aggregator (if this is a top level factory, the + * parent will be {@code null}) + * @param collectsFromSingleBucket + * If true then the created aggregator will only be collected + * with 0 as a bucket ordinal. Some factories can take + * advantage of this in order to return more optimized + * implementations. + * + * @return The created aggregator + */ + public final Reducer create() throws IOException { + Reducer aggregator = createInternal(this.metaData); + return aggregator; + } + + public void doValidate(AggregatorFactory parent, AggregatorFactory[] factories, List reducerFactories) { + } + + public void setMetaData(Map metaData) { + this.metaData = metaData; + } + + public String getName() { + return name; + } + + public String[] getBucketsPaths() { + return bucketsPaths; + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerStreams.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerStreams.java new file mode 100644 index 00000000000..7a4319e0a2b --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/ReducerStreams.java @@ -0,0 +1,68 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.search.aggregations.reducers; + +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 ReducerStreams { + + private static ImmutableMap streams = ImmutableMap.of(); + + /** + * A stream that knows how to read an aggregation from the input. + */ + public static interface Stream { + Reducer 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 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); + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/SiblingReducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/SiblingReducer.java new file mode 100644 index 00000000000..b0be9634ddc --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/SiblingReducer.java @@ -0,0 +1,65 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import com.google.common.collect.Lists; + +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public abstract class SiblingReducer extends Reducer { + + protected SiblingReducer() { // for Serialisation + super(); + } + + protected SiblingReducer(String name, String[] bucketsPaths, Map metaData) { + super(name, bucketsPaths, metaData); + } + + @SuppressWarnings("unchecked") + @Override + public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { + @SuppressWarnings("rawtypes") + InternalMultiBucketAggregation multiBucketsAgg = (InternalMultiBucketAggregation) aggregation; + List buckets = multiBucketsAgg.getBuckets(); + List newBuckets = new ArrayList<>(); + for (int i = 0; i < buckets.size(); i++) { + InternalMultiBucketAggregation.InternalBucket bucket = (InternalMultiBucketAggregation.InternalBucket) buckets.get(i); + InternalAggregation aggToAdd = doReduce(bucket.getAggregations(), reduceContext); + List aggs = new ArrayList<>(Lists.transform(bucket.getAggregations().asList(), AGGREGATION_TRANFORM_FUNCTION)); + aggs.add(aggToAdd); + InternalMultiBucketAggregation.InternalBucket newBucket = multiBucketsAgg.createBucket(new InternalAggregations(aggs), bucket); + newBuckets.add(newBucket); + } + + return multiBucketsAgg.create(newBuckets); + } + + public abstract InternalAggregation doReduce(Aggregations aggregations, ReduceContext context); +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/SimpleValue.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/SimpleValue.java new file mode 100644 index 00000000000..e1c510e1a29 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/SimpleValue.java @@ -0,0 +1,26 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; + +public interface SimpleValue extends NumericMetricsAggregation.SingleValue { + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/InternalBucketMetricValue.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/InternalBucketMetricValue.java new file mode 100644 index 00000000000..2229dd7baf5 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/InternalBucketMetricValue.java @@ -0,0 +1,131 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.bucketmetrics; + +import org.elasticsearch.common.Nullable; +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.AggregationStreams; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class InternalBucketMetricValue extends InternalNumericMetricsAggregation.SingleValue { + + public final static Type TYPE = new Type("bucket_metric_value"); + + public final static AggregationStreams.Stream STREAM = new AggregationStreams.Stream() { + @Override + public InternalBucketMetricValue readResult(StreamInput in) throws IOException { + InternalBucketMetricValue result = new InternalBucketMetricValue(); + result.readFrom(in); + return result; + } + }; + + public static void registerStreams() { + AggregationStreams.registerStream(STREAM, TYPE.stream()); + } + + private double value; + + private String[] keys; + + protected InternalBucketMetricValue() { + super(); + } + + public InternalBucketMetricValue(String name, String[] keys, double value, @Nullable ValueFormatter formatter, + List reducers, Map metaData) { + super(name, reducers, metaData); + this.keys = keys; + this.value = value; + this.valueFormatter = formatter; + } + + @Override + public Type type() { + return TYPE; + } + + @Override + public double value() { + return value; + } + + public String[] keys() { + return keys; + } + + @Override + public InternalAggregation doReduce(List aggregations, ReduceContext reduceContext) { + throw new UnsupportedOperationException("Not supported"); + } + + @Override + public Object getProperty(List path) { + if (path.isEmpty()) { + return this; + } else if (path.size() == 1 && "value".equals(path.get(0))) { + return value(); + } else if (path.size() == 1 && "keys".equals(path.get(0))) { + return keys(); + } else { + throw new IllegalArgumentException("path not supported for [" + getName() + "]: " + path); + } + } + + @Override + protected void doReadFrom(StreamInput in) throws IOException { + valueFormatter = ValueFormatterStreams.readOptional(in); + value = in.readDouble(); + keys = in.readStringArray(); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + ValueFormatterStreams.writeOptional(valueFormatter, out); + out.writeDouble(value); + out.writeStringArray(keys); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + boolean hasValue = !Double.isInfinite(value); + builder.field(CommonFields.VALUE, hasValue ? value : null); + if (hasValue && valueFormatter != null) { + builder.field(CommonFields.VALUE_AS_STRING, valueFormatter.format(value)); + } + builder.startArray("keys"); + for (String key : keys) { + builder.value(key); + } + builder.endArray(); + return builder; + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketBuilder.java new file mode 100644 index 00000000000..7fbcd54f789 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketBuilder.java @@ -0,0 +1,59 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.bucketmetrics; + +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; +import org.elasticsearch.search.aggregations.reducers.derivative.DerivativeParser; + +import java.io.IOException; + +public class MaxBucketBuilder extends ReducerBuilder { + + private String format; + private GapPolicy gapPolicy; + + public MaxBucketBuilder(String name) { + super(name, MaxBucketReducer.TYPE.name()); + } + + public MaxBucketBuilder format(String format) { + this.format = format; + return this; + } + + public MaxBucketBuilder gapPolicy(GapPolicy gapPolicy) { + this.gapPolicy = gapPolicy; + return this; + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { + if (format != null) { + builder.field(MaxBucketParser.FORMAT.getPreferredName(), format); + } + if (gapPolicy != null) { + builder.field(DerivativeParser.GAP_POLICY.getPreferredName(), gapPolicy.getName()); + } + return builder; + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketParser.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketParser.java new file mode 100644 index 00000000000..28fe0110238 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketParser.java @@ -0,0 +1,97 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.bucketmetrics; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchParseException; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.support.format.ValueFormat; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MaxBucketParser implements Reducer.Parser { + public static final ParseField FORMAT = new ParseField("format"); + + @Override + public String type() { + return MaxBucketReducer.TYPE.name(); + } + + @Override + public ReducerFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException { + XContentParser.Token token; + String currentFieldName = null; + String[] bucketsPaths = null; + String format = null; + GapPolicy gapPolicy = GapPolicy.SKIP; + + 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 (FORMAT.match(currentFieldName)) { + format = parser.text(); + } else if (BUCKETS_PATH.match(currentFieldName)) { + bucketsPaths = new String[] { parser.text() }; + } else if (GAP_POLICY.match(currentFieldName)) { + gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation()); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (BUCKETS_PATH.match(currentFieldName)) { + List paths = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String path = parser.text(); + paths.add(path); + } + bucketsPaths = paths.toArray(new String[paths.size()]); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else { + throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].", + parser.getTokenLocation()); + } + } + + if (bucketsPaths == null) { + throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName() + + "] for derivative aggregation [" + reducerName + "]", parser.getTokenLocation()); + } + + ValueFormatter formatter = null; + if (format != null) { + formatter = ValueFormat.Patternable.Number.format(format).formatter(); + } + + return new MaxBucketReducer.Factory(reducerName, bucketsPaths, gapPolicy, formatter); + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketReducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketReducer.java new file mode 100644 index 00000000000..22bc30fd730 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/bucketmetrics/MaxBucketReducer.java @@ -0,0 +1,150 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.bucketmetrics; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.reducers.ReducerStreams; +import org.elasticsearch.search.aggregations.reducers.SiblingReducer; +import org.elasticsearch.search.aggregations.support.AggregationPath; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class MaxBucketReducer extends SiblingReducer { + + public final static Type TYPE = new Type("max_bucket"); + + public final static ReducerStreams.Stream STREAM = new ReducerStreams.Stream() { + @Override + public MaxBucketReducer readResult(StreamInput in) throws IOException { + MaxBucketReducer result = new MaxBucketReducer(); + result.readFrom(in); + return result; + } + }; + + private ValueFormatter formatter; + private GapPolicy gapPolicy; + + public static void registerStreams() { + ReducerStreams.registerStream(STREAM, TYPE.stream()); + } + + private MaxBucketReducer() { + } + + protected MaxBucketReducer(String name, String[] bucketsPaths, GapPolicy gapPolicy, @Nullable ValueFormatter formatter, + Map metaData) { + super(name, bucketsPaths, metaData); + this.gapPolicy = gapPolicy; + this.formatter = formatter; + } + + @Override + public Type type() { + return TYPE; + } + + public InternalAggregation doReduce(Aggregations aggregations, ReduceContext context) { + List maxBucketKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + List bucketsPath = AggregationPath.parse(bucketsPaths()[0]).getPathElementsAsStringList(); + for (Aggregation aggregation : aggregations) { + if (aggregation.getName().equals(bucketsPath.get(0))) { + bucketsPath = bucketsPath.subList(1, bucketsPath.size()); + InternalMultiBucketAggregation multiBucketsAgg = (InternalMultiBucketAggregation) aggregation; + List buckets = multiBucketsAgg.getBuckets(); + for (int i = 0; i < buckets.size(); i++) { + Bucket bucket = buckets.get(i); + Double bucketValue = BucketHelpers.resolveBucketValue(multiBucketsAgg, bucket, bucketsPath, gapPolicy); + if (bucketValue != null) { + if (bucketValue > maxValue) { + maxBucketKeys.clear(); + maxBucketKeys.add(bucket.getKeyAsString()); + maxValue = bucketValue; + } else if (bucketValue.equals(maxValue)) { + maxBucketKeys.add(bucket.getKeyAsString()); + } + } + } + } + } + String[] keys = maxBucketKeys.toArray(new String[maxBucketKeys.size()]); + return new InternalBucketMetricValue(name(), keys, maxValue, formatter, Collections.EMPTY_LIST, metaData()); + } + + @Override + public void doReadFrom(StreamInput in) throws IOException { + formatter = ValueFormatterStreams.readOptional(in); + gapPolicy = GapPolicy.readFrom(in); + } + + @Override + public void doWriteTo(StreamOutput out) throws IOException { + ValueFormatterStreams.writeOptional(formatter, out); + gapPolicy.writeTo(out); + } + + public static class Factory extends ReducerFactory { + + private final ValueFormatter formatter; + private final GapPolicy gapPolicy; + + public Factory(String name, String[] bucketsPaths, GapPolicy gapPolicy, @Nullable ValueFormatter formatter) { + super(name, TYPE.name(), bucketsPaths); + this.gapPolicy = gapPolicy; + this.formatter = formatter; + } + + @Override + protected Reducer createInternal(Map metaData) throws IOException { + return new MaxBucketReducer(name, bucketsPaths, gapPolicy, formatter, metaData); + } + + @Override + public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFactories, List reducerFactories) { + if (bucketsPaths.length != 1) { + throw new IllegalStateException(Reducer.Parser.BUCKETS_PATH.getPreferredName() + + " must contain a single entry for reducer [" + name + "]"); + } + } + + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeBuilder.java new file mode 100644 index 00000000000..210d56d4a6f --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeBuilder.java @@ -0,0 +1,59 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.derivative; + +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; + +import java.io.IOException; + +import static org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; + +public class DerivativeBuilder extends ReducerBuilder { + + private String format; + private GapPolicy gapPolicy; + + public DerivativeBuilder(String name) { + super(name, DerivativeReducer.TYPE.name()); + } + + public DerivativeBuilder format(String format) { + this.format = format; + return this; + } + + public DerivativeBuilder gapPolicy(GapPolicy gapPolicy) { + this.gapPolicy = gapPolicy; + return this; + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { + if (format != null) { + builder.field(DerivativeParser.FORMAT.getPreferredName(), format); + } + if (gapPolicy != null) { + builder.field(DerivativeParser.GAP_POLICY.getPreferredName(), gapPolicy.getName()); + } + return builder; + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeParser.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeParser.java new file mode 100644 index 00000000000..a049a285b0e --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeParser.java @@ -0,0 +1,95 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.derivative; + +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchParseException; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.support.format.ValueFormat; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class DerivativeParser implements Reducer.Parser { + + @Override + public String type() { + return DerivativeReducer.TYPE.name(); + } + + @Override + public ReducerFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException { + XContentParser.Token token; + String currentFieldName = null; + String[] bucketsPaths = null; + String format = null; + GapPolicy gapPolicy = GapPolicy.SKIP; + + 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 (FORMAT.match(currentFieldName)) { + format = parser.text(); + } else if (BUCKETS_PATH.match(currentFieldName)) { + bucketsPaths = new String[] { parser.text() }; + } else if (GAP_POLICY.match(currentFieldName)) { + gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation()); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (BUCKETS_PATH.match(currentFieldName)) { + List paths = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String path = parser.text(); + paths.add(path); + } + bucketsPaths = paths.toArray(new String[paths.size()]); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else { + throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].", + parser.getTokenLocation()); + } + } + + if (bucketsPaths == null) { + throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName() + + "] for derivative aggregation [" + reducerName + "]", parser.getTokenLocation()); + } + + ValueFormatter formatter = null; + if (format != null) { + formatter = ValueFormat.Patternable.Number.format(format).formatter(); + } + + return new DerivativeReducer.Factory(reducerName, bucketsPaths, formatter, gapPolicy); + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeReducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeReducer.java new file mode 100644 index 00000000000..9d3397e8746 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/derivative/DerivativeReducer.java @@ -0,0 +1,157 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.derivative; + +import com.google.common.collect.Lists; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.InternalSimpleValue; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.reducers.ReducerStreams; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.search.aggregations.reducers.BucketHelpers.resolveBucketValue; + +public class DerivativeReducer extends Reducer { + + public final static Type TYPE = new Type("derivative"); + + public final static ReducerStreams.Stream STREAM = new ReducerStreams.Stream() { + @Override + public DerivativeReducer readResult(StreamInput in) throws IOException { + DerivativeReducer result = new DerivativeReducer(); + result.readFrom(in); + return result; + } + }; + + public static void registerStreams() { + ReducerStreams.registerStream(STREAM, TYPE.stream()); + } + + private ValueFormatter formatter; + private GapPolicy gapPolicy; + + public DerivativeReducer() { + } + + public DerivativeReducer(String name, String[] bucketsPaths, @Nullable ValueFormatter formatter, GapPolicy gapPolicy, + Map metadata) { + super(name, bucketsPaths, metadata); + this.formatter = formatter; + this.gapPolicy = gapPolicy; + } + + @Override + public Type type() { + return TYPE; + } + + @Override + public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { + InternalHistogram histo = (InternalHistogram) aggregation; + List buckets = histo.getBuckets(); + InternalHistogram.Factory factory = histo.getFactory(); + + List newBuckets = new ArrayList<>(); + Double lastBucketValue = null; + for (InternalHistogram.Bucket bucket : buckets) { + Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy); + if (lastBucketValue != null) { + double diff = thisBucketValue - lastBucketValue; + + List aggs = new ArrayList<>(Lists.transform(bucket.getAggregations().asList(), AGGREGATION_TRANFORM_FUNCTION)); + aggs.add(new InternalSimpleValue(name(), diff, formatter, new ArrayList(), metaData())); + InternalHistogram.Bucket newBucket = factory.createBucket(bucket.getKey(), bucket.getDocCount(), new InternalAggregations( + aggs), bucket.getKeyed(), bucket.getFormatter()); + newBuckets.add(newBucket); + } else { + newBuckets.add(bucket); + } + lastBucketValue = thisBucketValue; + } + return factory.create(newBuckets, histo); + } + + @Override + public void doReadFrom(StreamInput in) throws IOException { + formatter = ValueFormatterStreams.readOptional(in); + gapPolicy = GapPolicy.readFrom(in); + } + + @Override + public void doWriteTo(StreamOutput out) throws IOException { + ValueFormatterStreams.writeOptional(formatter, out); + gapPolicy.writeTo(out); + } + + public static class Factory extends ReducerFactory { + + private final ValueFormatter formatter; + private GapPolicy gapPolicy; + + public Factory(String name, String[] bucketsPaths, @Nullable ValueFormatter formatter, GapPolicy gapPolicy) { + super(name, TYPE.name(), bucketsPaths); + this.formatter = formatter; + this.gapPolicy = gapPolicy; + } + + @Override + protected Reducer createInternal(Map metaData) throws IOException { + return new DerivativeReducer(name, bucketsPaths, formatter, gapPolicy, metaData); + } + + @Override + public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFactories, List reducerFactories) { + if (bucketsPaths.length != 1) { + throw new IllegalStateException(Reducer.Parser.BUCKETS_PATH.getPreferredName() + + " must contain a single entry for reducer [" + name + "]"); + } + if (!(parent instanceof HistogramAggregator.Factory)) { + throw new IllegalStateException("derivative reducer [" + name + + "] must have a histogram or date_histogram as parent"); + } else { + HistogramAggregator.Factory histoParent = (HistogramAggregator.Factory) parent; + if (histoParent.minDocCount() != 0) { + throw new IllegalStateException("parent histogram of derivative reducer [" + name + + "] must have min_doc_count of 0"); + } + } + } + + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgBuilder.java new file mode 100644 index 00000000000..5fba23957e9 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgBuilder.java @@ -0,0 +1,119 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg; + +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelBuilder; + +import java.io.IOException; + +import static org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; + +/** + * A builder to create MovingAvg reducer aggregations + */ +public class MovAvgBuilder extends ReducerBuilder { + + private String format; + private GapPolicy gapPolicy; + private MovAvgModelBuilder modelBuilder; + private Integer window; + private Integer predict; + + public MovAvgBuilder(String name) { + super(name, MovAvgReducer.TYPE.name()); + } + + public MovAvgBuilder format(String format) { + this.format = format; + return this; + } + + /** + * Defines what should be done when a gap in the series is discovered + * + * @param gapPolicy A GapPolicy enum defining the selected policy + * @return Returns the builder to continue chaining + */ + public MovAvgBuilder gapPolicy(GapPolicy gapPolicy) { + this.gapPolicy = gapPolicy; + return this; + } + + /** + * Sets a MovAvgModelBuilder for the Moving Average. The model builder is used to + * define what type of moving average you want to use on the series + * + * @param modelBuilder A MovAvgModelBuilder which has been prepopulated with settings + * @return Returns the builder to continue chaining + */ + public MovAvgBuilder modelBuilder(MovAvgModelBuilder modelBuilder) { + this.modelBuilder = modelBuilder; + return this; + } + + /** + * Sets the window size for the moving average. This window will "slide" across the + * series, and the values inside that window will be used to calculate the moving avg value + * + * @param window Size of window + * @return Returns the builder to continue chaining + */ + public MovAvgBuilder window(int window) { + this.window = window; + return this; + } + + /** + * Sets the number of predictions that should be returned. Each prediction will be spaced at + * the intervals specified in the histogram. E.g "predict: 2" will return two new buckets at the + * end of the histogram with the predicted values. + * + * @param numPredictions Number of predictions to make + * @return Returns the builder to continue chaining + */ + public MovAvgBuilder predict(int numPredictions) { + this.predict = numPredictions; + return this; + } + + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { + if (format != null) { + builder.field(MovAvgParser.FORMAT.getPreferredName(), format); + } + if (gapPolicy != null) { + builder.field(MovAvgParser.GAP_POLICY.getPreferredName(), gapPolicy.getName()); + } + if (modelBuilder != null) { + modelBuilder.toXContent(builder, params); + } + if (window != null) { + builder.field(MovAvgParser.WINDOW.getPreferredName(), window); + } + if (predict != null) { + builder.field(MovAvgParser.PREDICT.getPreferredName(), predict); + } + return builder; + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgParser.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgParser.java new file mode 100644 index 00000000000..4ebf6e10d32 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgParser.java @@ -0,0 +1,155 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg; + +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.SearchParseException; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModel; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelParser; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelParserMapper; +import org.elasticsearch.search.aggregations.support.format.ValueFormat; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class MovAvgParser implements Reducer.Parser { + + public static final ParseField MODEL = new ParseField("model"); + public static final ParseField WINDOW = new ParseField("window"); + public static final ParseField SETTINGS = new ParseField("settings"); + public static final ParseField PREDICT = new ParseField("predict"); + + private final MovAvgModelParserMapper movAvgModelParserMapper; + + @Inject + public MovAvgParser(MovAvgModelParserMapper movAvgModelParserMapper) { + this.movAvgModelParserMapper = movAvgModelParserMapper; + } + + @Override + public String type() { + return MovAvgReducer.TYPE.name(); + } + + @Override + public ReducerFactory parse(String reducerName, XContentParser parser, SearchContext context) throws IOException { + XContentParser.Token token; + String currentFieldName = null; + String[] bucketsPaths = null; + String format = null; + + GapPolicy gapPolicy = GapPolicy.SKIP; + int window = 5; + Map settings = null; + String model = "simple"; + int predict = 0; + + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.VALUE_NUMBER) { + if (WINDOW.match(currentFieldName)) { + window = parser.intValue(); + if (window <= 0) { + throw new SearchParseException(context, "[" + currentFieldName + "] value must be a positive, " + + "non-zero integer. Value supplied was [" + predict + "] in [" + reducerName + "].", + parser.getTokenLocation()); + } + } else if (PREDICT.match(currentFieldName)) { + predict = parser.intValue(); + if (predict <= 0) { + throw new SearchParseException(context, "[" + currentFieldName + "] value must be a positive, " + + "non-zero integer. Value supplied was [" + predict + "] in [" + reducerName + "].", + parser.getTokenLocation()); + } + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.VALUE_STRING) { + if (FORMAT.match(currentFieldName)) { + format = parser.text(); + } else if (BUCKETS_PATH.match(currentFieldName)) { + bucketsPaths = new String[] { parser.text() }; + } else if (GAP_POLICY.match(currentFieldName)) { + gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation()); + } else if (MODEL.match(currentFieldName)) { + model = parser.text(); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (BUCKETS_PATH.match(currentFieldName)) { + List paths = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String path = parser.text(); + paths.add(path); + } + bucketsPaths = paths.toArray(new String[paths.size()]); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else if (token == XContentParser.Token.START_OBJECT) { + if (SETTINGS.match(currentFieldName)) { + settings = parser.map(); + } else { + throw new SearchParseException(context, "Unknown key for a " + token + " in [" + reducerName + "]: [" + + currentFieldName + "].", parser.getTokenLocation()); + } + } else { + throw new SearchParseException(context, "Unexpected token " + token + " in [" + reducerName + "].", + parser.getTokenLocation()); + } + } + + if (bucketsPaths == null) { + throw new SearchParseException(context, "Missing required field [" + BUCKETS_PATH.getPreferredName() + + "] for movingAvg aggregation [" + reducerName + "]", parser.getTokenLocation()); + } + + ValueFormatter formatter = null; + if (format != null) { + formatter = ValueFormat.Patternable.Number.format(format).formatter(); + } + + MovAvgModelParser modelParser = movAvgModelParserMapper.get(model); + if (modelParser == null) { + throw new SearchParseException(context, "Unknown model [" + model + "] specified. Valid options are:" + + movAvgModelParserMapper.getAllNames().toString(), parser.getTokenLocation()); + } + MovAvgModel movAvgModel = modelParser.parse(settings); + + + return new MovAvgReducer.Factory(reducerName, bucketsPaths, formatter, gapPolicy, window, predict, movAvgModel); + } + + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgReducer.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgReducer.java new file mode 100644 index 00000000000..d9ff1b4173e --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/MovAvgReducer.java @@ -0,0 +1,233 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg; + +import com.google.common.base.Function; +import com.google.common.collect.EvictingQueue; +import com.google.common.collect.Lists; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregator; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.InternalSimpleValue; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerFactory; +import org.elasticsearch.search.aggregations.reducers.ReducerStreams; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModel; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelStreams; +import org.elasticsearch.search.aggregations.support.format.ValueFormatter; +import org.elasticsearch.search.aggregations.support.format.ValueFormatterStreams; +import org.joda.time.DateTime; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.search.aggregations.reducers.BucketHelpers.resolveBucketValue; + +public class MovAvgReducer extends Reducer { + + public final static Type TYPE = new Type("moving_avg"); + + public final static ReducerStreams.Stream STREAM = new ReducerStreams.Stream() { + @Override + public MovAvgReducer readResult(StreamInput in) throws IOException { + MovAvgReducer result = new MovAvgReducer(); + result.readFrom(in); + return result; + } + }; + + public static void registerStreams() { + ReducerStreams.registerStream(STREAM, TYPE.stream()); + } + + private static final Function FUNCTION = new Function() { + @Override + public InternalAggregation apply(Aggregation input) { + return (InternalAggregation) input; + } + }; + + private ValueFormatter formatter; + private GapPolicy gapPolicy; + private int window; + private MovAvgModel model; + private int predict; + + public MovAvgReducer() { + } + + public MovAvgReducer(String name, String[] bucketsPaths, @Nullable ValueFormatter formatter, GapPolicy gapPolicy, + int window, int predict, MovAvgModel model, Map metadata) { + super(name, bucketsPaths, metadata); + this.formatter = formatter; + this.gapPolicy = gapPolicy; + this.window = window; + this.model = model; + this.predict = predict; + } + + @Override + public Type type() { + return TYPE; + } + + @Override + public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { + InternalHistogram histo = (InternalHistogram) aggregation; + List buckets = histo.getBuckets(); + InternalHistogram.Factory factory = histo.getFactory(); + + List newBuckets = new ArrayList<>(); + EvictingQueue values = EvictingQueue.create(this.window); + + long lastKey = 0; + long interval = Long.MAX_VALUE; + Object currentKey; + + for (InternalHistogram.Bucket bucket : buckets) { + Double thisBucketValue = resolveBucketValue(histo, bucket, bucketsPaths()[0], gapPolicy); + currentKey = bucket.getKey(); + + if (!(thisBucketValue == null || thisBucketValue.equals(Double.NaN))) { + values.offer(thisBucketValue); + + double movavg = model.next(values); + + List aggs = new ArrayList<>(Lists.transform(bucket.getAggregations().asList(), FUNCTION)); + aggs.add(new InternalSimpleValue(name(), movavg, formatter, new ArrayList(), metaData())); + InternalHistogram.Bucket newBucket = factory.createBucket(currentKey, bucket.getDocCount(), new InternalAggregations( + aggs), bucket.getKeyed(), bucket.getFormatter()); + newBuckets.add(newBucket); + + } else { + newBuckets.add(bucket); + } + + if (predict > 0) { + if (currentKey instanceof Number) { + interval = Math.min(interval, ((Number) bucket.getKey()).longValue() - lastKey); + lastKey = ((Number) bucket.getKey()).longValue(); + } else if (currentKey instanceof DateTime) { + interval = Math.min(interval, ((DateTime) bucket.getKey()).getMillis() - lastKey); + lastKey = ((DateTime) bucket.getKey()).getMillis(); + } else { + throw new AggregationExecutionException("Expected key of type Number or DateTime but got [" + currentKey + "]"); + } + } + + } + + + if (buckets.size() > 0 && predict > 0) { + + boolean keyed; + ValueFormatter formatter; + keyed = buckets.get(0).getKeyed(); + formatter = buckets.get(0).getFormatter(); + + double[] predictions = model.predict(values, predict); + for (int i = 0; i < predictions.length; i++) { + List aggs = new ArrayList<>(); + aggs.add(new InternalSimpleValue(name(), predictions[i], formatter, new ArrayList(), metaData())); + InternalHistogram.Bucket newBucket = factory.createBucket(lastKey + (interval * (i + 1)), 0, new InternalAggregations( + aggs), keyed, formatter); + newBuckets.add(newBucket); + } + } + + return factory.create(newBuckets, histo); + } + + @Override + public void doReadFrom(StreamInput in) throws IOException { + formatter = ValueFormatterStreams.readOptional(in); + gapPolicy = GapPolicy.readFrom(in); + window = in.readVInt(); + predict = in.readVInt(); + model = MovAvgModelStreams.read(in); + + } + + @Override + public void doWriteTo(StreamOutput out) throws IOException { + ValueFormatterStreams.writeOptional(formatter, out); + gapPolicy.writeTo(out); + out.writeVInt(window); + out.writeVInt(predict); + model.writeTo(out); + + } + + public static class Factory extends ReducerFactory { + + private final ValueFormatter formatter; + private GapPolicy gapPolicy; + private int window; + private MovAvgModel model; + private int predict; + + public Factory(String name, String[] bucketsPaths, @Nullable ValueFormatter formatter, GapPolicy gapPolicy, + int window, int predict, MovAvgModel model) { + super(name, TYPE.name(), bucketsPaths); + this.formatter = formatter; + this.gapPolicy = gapPolicy; + this.window = window; + this.model = model; + this.predict = predict; + } + + @Override + protected Reducer createInternal(Map metaData) throws IOException { + return new MovAvgReducer(name, bucketsPaths, formatter, gapPolicy, window, predict, model, metaData); + } + + @Override + public void doValidate(AggregatorFactory parent, AggregatorFactory[] aggFactories, List reducerFactories) { + if (bucketsPaths.length != 1) { + throw new IllegalStateException(Reducer.Parser.BUCKETS_PATH.getPreferredName() + + " must contain a single entry for reducer [" + name + "]"); + } + if (!(parent instanceof HistogramAggregator.Factory)) { + throw new IllegalStateException("derivative reducer [" + name + + "] must have a histogram or date_histogram as parent"); + } else { + HistogramAggregator.Factory histoParent = (HistogramAggregator.Factory) parent; + if (histoParent.minDocCount() != 0) { + throw new IllegalStateException("parent histogram of derivative reducer [" + name + + "] must have min_doc_count of 0"); + } + } + } + + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/DoubleExpModel.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/DoubleExpModel.java new file mode 100644 index 00000000000..7d32989cda1 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/DoubleExpModel.java @@ -0,0 +1,214 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +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.reducers.movavg.MovAvgParser; + +import java.io.IOException; +import java.util.*; + +/** + * Calculate a doubly exponential weighted moving average + */ +public class DoubleExpModel extends MovAvgModel { + + protected static final ParseField NAME_FIELD = new ParseField("double_exp"); + + /** + * Controls smoothing of data. Alpha = 1 retains no memory of past values + * (e.g. random walk), while alpha = 0 retains infinite memory of past values (e.g. + * mean of the series). Useful values are somewhere in between + */ + private double alpha; + + /** + * Equivalent to alpha, but controls the smoothing of the trend instead of the data + */ + private double beta; + + public DoubleExpModel(double alpha, double beta) { + this.alpha = alpha; + this.beta = beta; + } + + /** + * Predicts the next `n` values in the series, using the smoothing model to generate new values. + * Unlike the other moving averages, double-exp has forecasting/prediction built into the algorithm. + * Prediction is more than simply adding the next prediction to the window and repeating. Double-exp + * will extrapolate into the future by applying the trend information to the smoothed data. + * + * @param values Collection of numerics to movingAvg, usually windowed + * @param numPredictions Number of newly generated predictions to return + * @param Type of numeric + * @return Returns an array of doubles, since most smoothing methods operate on floating points + */ + @Override + public double[] predict(Collection values, int numPredictions) { + return next(values, numPredictions); + } + + @Override + public double next(Collection values) { + return next(values, 1)[0]; + } + + /** + * Calculate a doubly exponential weighted moving average + * + * @param values Collection of values to calculate avg for + * @param numForecasts number of forecasts into the future to return + * + * @param Type T extending Number + * @return Returns a Double containing the moving avg for the window + */ + public double[] next(Collection values, int numForecasts) { + + if (values.size() == 0) { + return emptyPredictions(numForecasts); + } + + // Smoothed value + double s = 0; + double last_s = 0; + + // Trend value + double b = 0; + double last_b = 0; + + int counter = 0; + + //TODO bail if too few values + + T last; + for (T v : values) { + last = v; + if (counter == 1) { + s = v.doubleValue(); + b = v.doubleValue() - last.doubleValue(); + } else { + s = alpha * v.doubleValue() + (1.0d - alpha) * (last_s + last_b); + b = beta * (s - last_s) + (1 - beta) * last_b; + } + + counter += 1; + last_s = s; + last_b = b; + } + + double[] forecastValues = new double[numForecasts]; + for (int i = 0; i < numForecasts; i++) { + forecastValues[i] = s + (i * b); + } + + return forecastValues; + } + + public static final MovAvgModelStreams.Stream STREAM = new MovAvgModelStreams.Stream() { + @Override + public MovAvgModel readResult(StreamInput in) throws IOException { + return new DoubleExpModel(in.readDouble(), in.readDouble()); + } + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + }; + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(STREAM.getName()); + out.writeDouble(alpha); + out.writeDouble(beta); + } + + public static class DoubleExpModelParser implements MovAvgModelParser { + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + + @Override + public MovAvgModel parse(@Nullable Map settings) { + + Double alpha; + Double beta; + + if (settings == null || (alpha = (Double)settings.get("alpha")) == null) { + alpha = 0.5; + } + + if (settings == null || (beta = (Double)settings.get("beta")) == null) { + beta = 0.5; + } + + return new DoubleExpModel(alpha, beta); + } + } + + public static class DoubleExpModelBuilder implements MovAvgModelBuilder { + + private double alpha = 0.5; + private double beta = 0.5; + + /** + * Alpha controls the smoothing of the data. Alpha = 1 retains no memory of past values + * (e.g. a random walk), while alpha = 0 retains infinite memory of past values (e.g. + * the series mean). Useful values are somewhere in between. Defaults to 0.5. + * + * @param alpha A double between 0-1 inclusive, controls data smoothing + * + * @return The builder to continue chaining + */ + public DoubleExpModelBuilder alpha(double alpha) { + this.alpha = alpha; + return this; + } + + /** + * Equivalent to alpha, but controls the smoothing of the trend instead of the data + * + * @param beta a double between 0-1 inclusive, controls trend smoothing + * + * @return The builder to continue chaining + */ + public DoubleExpModelBuilder beta(double beta) { + this.beta = beta; + return this; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(MovAvgParser.MODEL.getPreferredName(), NAME_FIELD.getPreferredName()); + builder.startObject(MovAvgParser.SETTINGS.getPreferredName()); + builder.field("alpha", alpha); + builder.field("beta", beta); + builder.endObject(); + return builder; + } + } +} + diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/LinearModel.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/LinearModel.java new file mode 100644 index 00000000000..6c269590d33 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/LinearModel.java @@ -0,0 +1,93 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +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.reducers.movavg.MovAvgParser; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + +/** + * Calculate a linearly weighted moving average, such that older values are + * linearly less important. "Time" is determined by position in collection + */ +public class LinearModel extends MovAvgModel { + + protected static final ParseField NAME_FIELD = new ParseField("linear"); + + @Override + public double next(Collection values) { + double avg = 0; + long totalWeight = 1; + long current = 1; + + for (T v : values) { + avg += v.doubleValue() * current; + totalWeight += current; + current += 1; + } + return avg / totalWeight; + } + + public static final MovAvgModelStreams.Stream STREAM = new MovAvgModelStreams.Stream() { + @Override + public MovAvgModel readResult(StreamInput in) throws IOException { + return new LinearModel(); + } + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + }; + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(STREAM.getName()); + } + + public static class LinearModelParser implements MovAvgModelParser { + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + + @Override + public MovAvgModel parse(@Nullable Map settings) { + return new LinearModel(); + } + } + + public static class LinearModelBuilder implements MovAvgModelBuilder { + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(MovAvgParser.MODEL.getPreferredName(), NAME_FIELD.getPreferredName()); + return builder; + } + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModel.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModel.java new file mode 100644 index 00000000000..8b8e80cff2f --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModel.java @@ -0,0 +1,97 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import com.google.common.collect.EvictingQueue; + +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; + +public abstract class MovAvgModel { + + /** + * Returns the next value in the series, according to the underlying smoothing model + * + * @param values Collection of numerics to movingAvg, usually windowed + * @param Type of numeric + * @return Returns a double, since most smoothing methods operate on floating points + */ + public abstract double next(Collection values); + + /** + * Predicts the next `n` values in the series, using the smoothing model to generate new values. + * Default prediction mode is to simply continuing calling next() and adding the + * predicted value back into the windowed buffer. + * + * @param values Collection of numerics to movingAvg, usually windowed + * @param numPredictions Number of newly generated predictions to return + * @param Type of numeric + * @return Returns an array of doubles, since most smoothing methods operate on floating points + */ + public double[] predict(Collection values, int numPredictions) { + double[] predictions = new double[numPredictions]; + + // If there are no values, we can't do anything. Return an array of NaNs. + if (values.size() == 0) { + return emptyPredictions(numPredictions); + } + + // special case for one prediction, avoids allocation + if (numPredictions < 1) { + throw new IllegalArgumentException("numPredictions may not be less than 1."); + } else if (numPredictions == 1){ + predictions[0] = next(values); + return predictions; + } + + Collection predictionBuffer = EvictingQueue.create(values.size()); + predictionBuffer.addAll(values); + + for (int i = 0; i < numPredictions; i++) { + predictions[i] = next(predictionBuffer); + + // Add the last value to the buffer, so we can keep predicting + predictionBuffer.add(predictions[i]); + } + + return predictions; + } + + protected double[] emptyPredictions(int numPredictions) { + double[] predictions = new double[numPredictions]; + Arrays.fill(predictions, Double.NaN); + return predictions; + } + + /** + * Write the model to the output stream + * + * @param out Output stream + * @throws IOException + */ + public abstract void writeTo(StreamOutput out) throws IOException; +} + + + + diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelBuilder.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelBuilder.java new file mode 100644 index 00000000000..a8f40d474ac --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelBuilder.java @@ -0,0 +1,32 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; + +import java.io.IOException; + +/** + * Represents the common interface that all moving average models share. Moving + * average models are used by the MovAvg reducer + */ +public interface MovAvgModelBuilder extends ToXContent { +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelModule.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelModule.java new file mode 100644 index 00000000000..71ccbcb31b0 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelModule.java @@ -0,0 +1,55 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import com.google.common.collect.Lists; +import org.elasticsearch.common.inject.AbstractModule; +import org.elasticsearch.common.inject.multibindings.Multibinder; + +import java.util.List; + +/** + * Register the various model parsers + */ +public class MovAvgModelModule extends AbstractModule { + + private List> parsers = Lists.newArrayList(); + + public MovAvgModelModule() { + registerParser(SimpleModel.SimpleModelParser.class); + registerParser(LinearModel.LinearModelParser.class); + registerParser(SingleExpModel.SingleExpModelParser.class); + registerParser(DoubleExpModel.DoubleExpModelParser.class); + } + + public void registerParser(Class parser) { + parsers.add(parser); + } + + @Override + protected void configure() { + Multibinder parserMapBinder = Multibinder.newSetBinder(binder(), MovAvgModelParser.class); + for (Class clazz : parsers) { + parserMapBinder.addBinding().to(clazz); + } + bind(MovAvgModelParserMapper.class); + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParser.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParser.java new file mode 100644 index 00000000000..d27e447baa4 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParser.java @@ -0,0 +1,34 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + + +import org.elasticsearch.common.Nullable; + +import java.util.Map; + +/** + * Common interface for parsers used by the various Moving Average models + */ +public interface MovAvgModelParser { + public MovAvgModel parse(@Nullable Map settings); + + public String getName(); +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParserMapper.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParserMapper.java new file mode 100644 index 00000000000..459729d8960 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelParserMapper.java @@ -0,0 +1,54 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.MapBuilder; +import org.elasticsearch.common.inject.Inject; + +import java.util.Set; + +/** + * Contains a map of all concrete model parsers which can be used to build Models + */ +public class MovAvgModelParserMapper { + + protected ImmutableMap movAvgParsers; + + @Inject + public MovAvgModelParserMapper(Set parsers) { + MapBuilder builder = MapBuilder.newMapBuilder(); + for (MovAvgModelParser parser : parsers) { + builder.put(parser.getName(), parser); + } + movAvgParsers = builder.immutableMap(); + } + + public @Nullable + MovAvgModelParser get(String parserName) { + return movAvgParsers.get(parserName); + } + + public ImmutableSet getAllNames() { + return movAvgParsers.keySet(); + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelStreams.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelStreams.java new file mode 100644 index 00000000000..b11a3687021 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/MovAvgModelStreams.java @@ -0,0 +1,74 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.common.collect.MapBuilder; +import org.elasticsearch.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * A registry for all moving average models. This is needed for reading them from a stream without knowing which + * one it is. + */ +public class MovAvgModelStreams { + + private static ImmutableMap STREAMS = ImmutableMap.of(); + + public static MovAvgModel read(StreamInput in) throws IOException { + return stream(in.readString()).readResult(in); + } + + /** + * A stream that knows how to read an heuristic from the input. + */ + public static interface Stream { + + MovAvgModel readResult(StreamInput in) throws IOException; + + String getName(); + } + + /** + * Registers the given stream and associate it with the given types. + * + * @param stream The stream to register + * @param names The names associated with the streams + */ + public static synchronized void registerStream(Stream stream, String... names) { + MapBuilder uStreams = MapBuilder.newMapBuilder(STREAMS); + for (String name : names) { + uStreams.put(name, stream); + } + STREAMS = uStreams.immutableMap(); + } + + /** + * Returns the stream that is registered for the given name + * + * @param name The given name + * @return The associated stream + */ + public static Stream stream(String name) { + return STREAMS.get(name); + } + +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SimpleModel.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SimpleModel.java new file mode 100644 index 00000000000..243b022af2c --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SimpleModel.java @@ -0,0 +1,86 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +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.reducers.movavg.MovAvgParser; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + +/** + * Calculate a simple unweighted (arithmetic) moving average + */ +public class SimpleModel extends MovAvgModel { + + protected static final ParseField NAME_FIELD = new ParseField("simple"); + + @Override + public double next(Collection values) { + double avg = 0; + for (T v : values) { + avg += v.doubleValue(); + } + return avg / values.size(); + } + + public static final MovAvgModelStreams.Stream STREAM = new MovAvgModelStreams.Stream() { + @Override + public MovAvgModel readResult(StreamInput in) throws IOException { + return new SimpleModel(); + } + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + }; + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(STREAM.getName()); + } + + public static class SimpleModelParser implements MovAvgModelParser { + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + + @Override + public MovAvgModel parse(@Nullable Map settings) { + return new SimpleModel(); + } + } + + public static class SimpleModelBuilder implements MovAvgModelBuilder { + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(MovAvgParser.MODEL.getPreferredName(), NAME_FIELD.getPreferredName()); + return builder; + } + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SingleExpModel.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SingleExpModel.java new file mode 100644 index 00000000000..f17ba68f498 --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/SingleExpModel.java @@ -0,0 +1,133 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseField; +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.reducers.movavg.MovAvgParser; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + +/** + * Calculate a exponentially weighted moving average + */ +public class SingleExpModel extends MovAvgModel { + + protected static final ParseField NAME_FIELD = new ParseField("single_exp"); + + /** + * Controls smoothing of data. Alpha = 1 retains no memory of past values + * (e.g. random walk), while alpha = 0 retains infinite memory of past values (e.g. + * mean of the series). Useful values are somewhere in between + */ + private double alpha; + + public SingleExpModel(double alpha) { + this.alpha = alpha; + } + + + @Override + public double next(Collection values) { + double avg = 0; + boolean first = true; + + for (T v : values) { + if (first) { + avg = v.doubleValue(); + first = false; + } else { + avg = (v.doubleValue() * alpha) + (avg * (1 - alpha)); + } + } + return avg; + } + + public static final MovAvgModelStreams.Stream STREAM = new MovAvgModelStreams.Stream() { + @Override + public MovAvgModel readResult(StreamInput in) throws IOException { + return new SingleExpModel(in.readDouble()); + } + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + }; + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(STREAM.getName()); + out.writeDouble(alpha); + } + + public static class SingleExpModelParser implements MovAvgModelParser { + + @Override + public String getName() { + return NAME_FIELD.getPreferredName(); + } + + @Override + public MovAvgModel parse(@Nullable Map settings) { + + Double alpha; + if (settings == null || (alpha = (Double)settings.get("alpha")) == null) { + alpha = 0.5; + } + + return new SingleExpModel(alpha); + } + } + + public static class SingleExpModelBuilder implements MovAvgModelBuilder { + + private double alpha = 0.5; + + /** + * Alpha controls the smoothing of the data. Alpha = 1 retains no memory of past values + * (e.g. a random walk), while alpha = 0 retains infinite memory of past values (e.g. + * the series mean). Useful values are somewhere in between. Defaults to 0.5. + * + * @param alpha A double between 0-1 inclusive, controls data smoothing + * + * @return The builder to continue chaining + */ + public SingleExpModelBuilder alpha(double alpha) { + this.alpha = alpha; + return this; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(MovAvgParser.MODEL.getPreferredName(), NAME_FIELD.getPreferredName()); + builder.startObject(MovAvgParser.SETTINGS.getPreferredName()); + builder.field("alpha", alpha); + builder.endObject(); + return builder; + } + } +} + diff --git a/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/TransportMovAvgModelModule.java b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/TransportMovAvgModelModule.java new file mode 100644 index 00000000000..bc085f6241a --- /dev/null +++ b/src/main/java/org/elasticsearch/search/aggregations/reducers/movavg/models/TransportMovAvgModelModule.java @@ -0,0 +1,51 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.movavg.models; + +import com.google.common.collect.Lists; +import org.elasticsearch.common.inject.AbstractModule; + +import java.util.List; + +/** + * Register the transport streams so that models can be serialized/deserialized from the stream + */ +public class TransportMovAvgModelModule extends AbstractModule { + + private List streams = Lists.newArrayList(); + + public TransportMovAvgModelModule() { + registerStream(SimpleModel.STREAM); + registerStream(LinearModel.STREAM); + registerStream(SingleExpModel.STREAM); + registerStream(DoubleExpModel.STREAM); + } + + public void registerStream(MovAvgModelStreams.Stream stream) { + streams.add(stream); + } + + @Override + protected void configure() { + for (MovAvgModelStreams.Stream stream : streams) { + MovAvgModelStreams.registerStream(stream, stream.getName()); + } + } +} diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/GeoPointParser.java b/src/main/java/org/elasticsearch/search/aggregations/support/GeoPointParser.java index 35c381ec2a1..b423dd2f755 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/GeoPointParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/GeoPointParser.java @@ -66,7 +66,8 @@ public class GeoPointParser { lat = parser.doubleValue(); } else { throw new SearchParseException(context, "malformed [" + currentFieldName + "] geo point array in [" + - aggName + "] " + aggType + " aggregation. a geo point array must be of the form [lon, lat]"); + aggName + "] " + aggType + " aggregation. a geo point array must be of the form [lon, lat]", + parser.getTokenLocation()); } } point = new GeoPoint(lat, lon); @@ -88,7 +89,7 @@ public class GeoPointParser { } if (Double.isNaN(lat) || Double.isNaN(lon)) { throw new SearchParseException(context, "malformed [" + currentFieldName + "] geo point object. either [lat] or [lon] (or both) are " + - "missing in [" + aggName + "] " + aggType + " aggregation"); + "missing in [" + aggName + "] " + aggType + " aggregation", parser.getTokenLocation()); } point = new GeoPoint(lat, lon); return true; diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java index d88f95642c3..dbefc2e2612 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceAggregatorFactory.java @@ -18,10 +18,16 @@ */ package org.elasticsearch.search.aggregations.support; -import org.elasticsearch.search.aggregations.*; +import org.elasticsearch.search.aggregations.AggregationExecutionException; +import org.elasticsearch.search.aggregations.AggregationInitializationException; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.aggregations.support.format.ValueFormat; import java.io.IOException; +import java.util.List; import java.util.Map; /** @@ -49,12 +55,13 @@ public abstract class ValuesSourceAggregatorFactory ext } @Override - public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException { + public Aggregator createInternal(AggregationContext context, Aggregator parent, boolean collectsFromSingleBucket, + List reducers, Map metaData) throws IOException { if (config.unmapped()) { - return createUnmapped(context, parent, metaData); + return createUnmapped(context, parent, reducers, metaData); } VS vs = context.valuesSource(config); - return doCreateInternal(vs, context, parent, collectsFromSingleBucket, metaData); + return doCreateInternal(vs, context, parent, collectsFromSingleBucket, reducers, metaData); } @Override @@ -64,9 +71,11 @@ public abstract class ValuesSourceAggregatorFactory ext } } - protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, Map metaData) throws IOException; + protected abstract Aggregator createUnmapped(AggregationContext aggregationContext, Aggregator parent, List reducers, + Map metaData) throws IOException; - protected abstract Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, boolean collectsFromSingleBucket, Map metaData) throws IOException; + protected abstract Aggregator doCreateInternal(VS valuesSource, AggregationContext aggregationContext, Aggregator parent, + boolean collectsFromSingleBucket, List reducers, Map metaData) throws IOException; private void resolveValuesSourceConfigFromAncestors(String aggName, AggregatorFactory parent, Class requiredValuesSourceType) { ValuesSourceConfig config; diff --git a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java index 37182685761..88c3f64b089 100644 --- a/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java +++ b/src/main/java/org/elasticsearch/search/aggregations/support/ValuesSourceParser.java @@ -101,7 +101,8 @@ public class ValuesSourceParser { if (targetValueType != null && input.valueType.isNotA(targetValueType)) { throw new SearchParseException(context, aggType.name() + " aggregation [" + aggName + "] was configured with an incompatible value type [" + input.valueType + "]. [" + aggType + - "] aggregation can only work on value of type [" + targetValueType + "]"); + "] aggregation can only work on value of type [" + targetValueType + "]", + parser.getTokenLocation()); } } else if (!scriptParameterParser.token(currentFieldName, token, parser)) { return false; diff --git a/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index 892cc2085ae..7ca72f5cef6 100644 --- a/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -23,6 +23,7 @@ import com.carrotsearch.hppc.ObjectFloatOpenHashMap; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; + import org.elasticsearch.ElasticsearchGenerationException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.client.Requests; @@ -38,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.FilterBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilder; import org.elasticsearch.search.fetch.innerhits.InnerHitsBuilder; import org.elasticsearch.search.fetch.source.FetchSourceContext; import org.elasticsearch.search.highlight.HighlightBuilder; @@ -55,8 +57,9 @@ import java.util.List; import java.util.Map; /** - * A search source builder allowing to easily build search source. Simple construction - * using {@link org.elasticsearch.search.builder.SearchSourceBuilder#searchSource()}. + * A search source builder allowing to easily build search source. Simple + * construction using + * {@link org.elasticsearch.search.builder.SearchSourceBuilder#searchSource()}. * * @see org.elasticsearch.action.search.SearchRequest#source(SearchSourceBuilder) */ @@ -109,7 +112,6 @@ public class SearchSourceBuilder implements ToXContent { private List aggregations; private BytesReference aggregationsBinary; - private HighlightBuilder highlightBuilder; private SuggestBuilder suggestBuilder; @@ -123,7 +125,6 @@ public class SearchSourceBuilder implements ToXContent { private String[] stats; - /** * Constructs a new search source builder. */ @@ -190,8 +191,9 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Sets a filter that will be executed after the query has been executed and only has affect on the search hits - * (not aggregations). This filter is always executed as last filtering mechanism. + * Sets a filter that will be executed after the query has been executed and + * only has affect on the search hits (not aggregations). This filter is + * always executed as last filtering mechanism. */ public SearchSourceBuilder postFilter(FilterBuilder postFilter) { this.postFilterBuilder = postFilter; @@ -276,8 +278,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Should each {@link org.elasticsearch.search.SearchHit} be returned with an - * explanation of the hit (ranking). + * Should each {@link org.elasticsearch.search.SearchHit} be returned with + * an explanation of the hit (ranking). */ public SearchSourceBuilder explain(Boolean explain) { this.explain = explain; @@ -285,8 +287,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Should each {@link org.elasticsearch.search.SearchHit} be returned with a version - * associated with it. + * Should each {@link org.elasticsearch.search.SearchHit} be returned with a + * version associated with it. */ public SearchSourceBuilder version(Boolean version) { this.version = version; @@ -310,8 +312,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * An optional terminate_after to terminate the search after - * collecting terminateAfter documents + * An optional terminate_after to terminate the search after collecting + * terminateAfter documents */ public SearchSourceBuilder terminateAfter(int terminateAfter) { if (terminateAfter <= 0) { @@ -320,11 +322,14 @@ public class SearchSourceBuilder implements ToXContent { this.terminateAfter = terminateAfter; return this; } + /** * Adds a sort against the given field name and the sort ordering. * - * @param name The name of the field - * @param order The sort ordering + * @param name + * The name of the field + * @param order + * The sort ordering */ public SearchSourceBuilder sort(String name, SortOrder order) { return sort(SortBuilders.fieldSort(name).order(order)); @@ -333,7 +338,8 @@ public class SearchSourceBuilder implements ToXContent { /** * Add a sort against the given field name. * - * @param name The name of the field to sort by + * @param name + * The name of the field to sort by */ public SearchSourceBuilder sort(String name) { return sort(SortBuilders.fieldSort(name)); @@ -351,8 +357,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Applies when sorting, and controls if scores will be tracked as well. Defaults to - * false. + * Applies when sorting, and controls if scores will be tracked as well. + * Defaults to false. */ public SearchSourceBuilder trackScores(boolean trackScores) { this.trackScores = trackScores; @@ -401,6 +407,7 @@ public class SearchSourceBuilder implements ToXContent { /** * Set the rescore window size for rescores that don't specify their window. + * * @param defaultRescoreWindowSize * @return */ @@ -465,7 +472,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Indicates whether the response should contain the stored _source for every hit + * Indicates whether the response should contain the stored _source for + * every hit * * @param fetch * @return @@ -480,22 +488,33 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Indicate that _source should be returned with every hit, with an "include" and/or "exclude" set which can include simple wildcard + * Indicate that _source should be returned with every hit, with an + * "include" and/or "exclude" set which can include simple wildcard * elements. * - * @param include An optional include (optionally wildcarded) pattern to filter the returned _source - * @param exclude An optional exclude (optionally wildcarded) pattern to filter the returned _source + * @param include + * An optional include (optionally wildcarded) pattern to filter + * the returned _source + * @param exclude + * An optional exclude (optionally wildcarded) pattern to filter + * the returned _source */ public SearchSourceBuilder fetchSource(@Nullable String include, @Nullable String exclude) { - return fetchSource(include == null ? Strings.EMPTY_ARRAY : new String[]{include}, exclude == null ? Strings.EMPTY_ARRAY : new String[]{exclude}); + return fetchSource(include == null ? Strings.EMPTY_ARRAY : new String[] { include }, exclude == null ? Strings.EMPTY_ARRAY + : new String[] { exclude }); } /** - * Indicate that _source should be returned with every hit, with an "include" and/or "exclude" set which can include simple wildcard + * Indicate that _source should be returned with every hit, with an + * "include" and/or "exclude" set which can include simple wildcard * elements. * - * @param includes An optional list of include (optionally wildcarded) pattern to filter the returned _source - * @param excludes An optional list of exclude (optionally wildcarded) pattern to filter the returned _source + * @param includes + * An optional list of include (optionally wildcarded) pattern to + * filter the returned _source + * @param excludes + * An optional list of exclude (optionally wildcarded) pattern to + * filter the returned _source */ public SearchSourceBuilder fetchSource(@Nullable String[] includes, @Nullable String[] excludes) { fetchSourceContext = new FetchSourceContext(includes, excludes); @@ -511,7 +530,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Sets no fields to be loaded, resulting in only id and type to be returned per field. + * Sets no fields to be loaded, resulting in only id and type to be returned + * per field. */ public SearchSourceBuilder noFields() { this.fieldNames = ImmutableList.of(); @@ -519,8 +539,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Sets the fields to load and return as part of the search request. If none are specified, - * the source of the document will be returned. + * Sets the fields to load and return as part of the search request. If none + * are specified, the source of the document will be returned. */ public SearchSourceBuilder fields(List fields) { this.fieldNames = fields; @@ -528,8 +548,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Adds the fields to load and return as part of the search request. If none are specified, - * the source of the document will be returned. + * Adds the fields to load and return as part of the search request. If none + * are specified, the source of the document will be returned. */ public SearchSourceBuilder fields(String... fields) { if (fieldNames == null) { @@ -542,8 +562,9 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Adds a field to load and return (note, it must be stored) as part of the search request. - * If none are specified, the source of the document will be return. + * Adds a field to load and return (note, it must be stored) as part of the + * search request. If none are specified, the source of the document will be + * return. */ public SearchSourceBuilder field(String name) { if (fieldNames == null) { @@ -554,7 +575,8 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Adds a field to load from the field data cache and return as part of the search request. + * Adds a field to load from the field data cache and return as part of the + * search request. */ public SearchSourceBuilder fieldDataField(String name) { if (fieldDataFields == null) { @@ -567,8 +589,10 @@ public class SearchSourceBuilder implements ToXContent { /** * Adds a script field under the given name with the provided script. * - * @param name The name of the field - * @param script The script + * @param name + * The name of the field + * @param script + * The script */ public SearchSourceBuilder scriptField(String name, String script) { return scriptField(name, null, script, null); @@ -577,9 +601,12 @@ public class SearchSourceBuilder implements ToXContent { /** * Adds a script field. * - * @param name The name of the field - * @param script The script to execute - * @param params The script parameters + * @param name + * The name of the field + * @param script + * The script to execute + * @param params + * The script parameters */ public SearchSourceBuilder scriptField(String name, String script, Map params) { return scriptField(name, null, script, params); @@ -588,10 +615,14 @@ public class SearchSourceBuilder implements ToXContent { /** * Adds a script field. * - * @param name The name of the field - * @param lang The language of the script - * @param script The script to execute - * @param params The script parameters (can be null) + * @param name + * The name of the field + * @param lang + * The language of the script + * @param script + * The script to execute + * @param params + * The script parameters (can be null) */ public SearchSourceBuilder scriptField(String name, String lang, String script, Map params) { if (scriptFields == null) { @@ -602,10 +633,13 @@ public class SearchSourceBuilder implements ToXContent { } /** - * Sets the boost a specific index will receive when the query is executeed against it. + * Sets the boost a specific index will receive when the query is executeed + * against it. * - * @param index The index to apply the boost against - * @param indexBoost The boost to apply to the index + * @param index + * The index to apply the boost against + * @param indexBoost + * The boost to apply to the index */ public SearchSourceBuilder indexBoost(String index, float indexBoost) { if (this.indexBoost == null) { @@ -648,7 +682,6 @@ public class SearchSourceBuilder implements ToXContent { } } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); @@ -657,7 +690,7 @@ public class SearchSourceBuilder implements ToXContent { return builder; } - public void innerToXContent(XContentBuilder builder, Params params) throws IOException{ + public void innerToXContent(XContentBuilder builder, Params params) throws IOException { if (from != -1) { builder.field("from", from); } @@ -885,34 +918,4 @@ public class SearchSourceBuilder implements ToXContent { return params; } } - - private static class PartialField { - private final String name; - private final String[] includes; - private final String[] excludes; - - private PartialField(String name, String[] includes, String[] excludes) { - this.name = name; - this.includes = includes; - this.excludes = excludes; - } - - private PartialField(String name, String include, String exclude) { - this.name = name; - this.includes = include == null ? null : new String[]{include}; - this.excludes = exclude == null ? null : new String[]{exclude}; - } - - public String name() { - return name; - } - - public String[] includes() { - return includes; - } - - public String[] excludes() { - return excludes; - } - } } diff --git a/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java b/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java index 1894e26277d..4489d98c2c8 100644 --- a/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java +++ b/src/main/java/org/elasticsearch/search/controller/SearchPhaseController.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.controller; import com.carrotsearch.hppc.IntArrayList; import com.carrotsearch.hppc.ObjectObjectOpenHashMap; +import com.google.common.collect.Lists; import org.apache.lucene.index.Term; import org.apache.lucene.search.CollectionStatistics; @@ -40,8 +41,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.SiblingReducer; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -403,6 +407,19 @@ public class SearchPhaseController extends AbstractComponent { } } + if (aggregations != null) { + List reducers = firstResult.reducers(); + if (reducers != null) { + List newAggs = new ArrayList<>(Lists.transform(aggregations.asList(), Reducer.AGGREGATION_TRANFORM_FUNCTION)); + for (SiblingReducer reducer : reducers) { + InternalAggregation newAgg = reducer.doReduce(new InternalAggregations(newAggs), new ReduceContext(bigArrays, + scriptService)); + newAggs.add(newAgg); + } + aggregations = new InternalAggregations(newAggs); + } + } + InternalSearchHits searchHits = new InternalSearchHits(hits.toArray(new InternalSearchHit[hits.size()]), totalHits, maxScore); return new InternalSearchResponse(searchHits, aggregations, suggest, timedOut, terminatedEarly); diff --git a/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java b/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java index c4d8aa80ef5..3613327c679 100644 --- a/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java +++ b/src/main/java/org/elasticsearch/search/highlight/HighlighterParseElement.java @@ -21,6 +21,7 @@ package org.elasticsearch.search.highlight; import com.google.common.collect.Lists; import com.google.common.collect.Sets; + import org.apache.lucene.search.vectorhighlight.SimpleBoundaryScanner; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.xcontent.XContentParser; @@ -70,7 +71,7 @@ public class HighlighterParseElement implements SearchParseElement { try { context.highlight(parse(parser, context.queryParserService())); } catch (IllegalArgumentException ex) { - throw new SearchParseException(context, "Error while trying to parse Highlighter element in request"); + throw new SearchParseException(context, "Error while trying to parse Highlighter element in request", parser.getTokenLocation()); } } diff --git a/src/main/java/org/elasticsearch/search/query/FromParseElement.java b/src/main/java/org/elasticsearch/search/query/FromParseElement.java index 13e58caa471..21063a93d35 100644 --- a/src/main/java/org/elasticsearch/search/query/FromParseElement.java +++ b/src/main/java/org/elasticsearch/search/query/FromParseElement.java @@ -35,7 +35,8 @@ public class FromParseElement implements SearchParseElement { if (token.isValue()) { int from = parser.intValue(); if (from < 0) { - throw new SearchParseException(context, "from is set to [" + from + "] and is expected to be higher or equal to 0"); + throw new SearchParseException(context, "from is set to [" + from + "] and is expected to be higher or equal to 0", + parser.getTokenLocation()); } context.from(from); } diff --git a/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 50167676cc7..e45006b2c32 100644 --- a/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -20,15 +20,20 @@ package org.elasticsearch.search.query; import org.apache.lucene.search.TopDocs; -import org.elasticsearch.Version; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.reducers.Reducer; +import org.elasticsearch.search.aggregations.reducers.ReducerStreams; +import org.elasticsearch.search.aggregations.reducers.SiblingReducer; import org.elasticsearch.search.suggest.Suggest; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import static org.elasticsearch.common.lucene.Lucene.readTopDocs; import static org.elasticsearch.common.lucene.Lucene.writeTopDocs; @@ -44,6 +49,7 @@ public class QuerySearchResult extends QuerySearchResultProvider { private int size; private TopDocs topDocs; private InternalAggregations aggregations; + private List reducers; private Suggest suggest; private boolean searchTimedOut; private Boolean terminatedEarly = null; @@ -114,6 +120,14 @@ public class QuerySearchResult extends QuerySearchResultProvider { this.aggregations = aggregations; } + public List reducers() { + return reducers; + } + + public void reducers(List reducers) { + this.reducers = reducers; + } + public Suggest suggest() { return suggest; } @@ -162,6 +176,16 @@ public class QuerySearchResult extends QuerySearchResultProvider { if (in.readBoolean()) { aggregations = InternalAggregations.readAggregations(in); } + if (in.readBoolean()) { + int size = in.readVInt(); + List reducers = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + BytesReference type = in.readBytesReference(); + Reducer reducer = ReducerStreams.stream(type).readResult(in); + reducers.add((SiblingReducer) reducer); + } + this.reducers = reducers; + } if (in.readBoolean()) { suggest = Suggest.readSuggest(Suggest.Fields.SUGGEST, in); } @@ -187,6 +211,16 @@ public class QuerySearchResult extends QuerySearchResultProvider { out.writeBoolean(true); aggregations.writeTo(out); } + if (reducers == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVInt(reducers.size()); + for (Reducer reducer : reducers) { + out.writeBytesReference(reducer.type().stream()); + reducer.writeTo(out); + } + } if (suggest == null) { out.writeBoolean(false); } else { diff --git a/src/main/java/org/elasticsearch/search/query/SizeParseElement.java b/src/main/java/org/elasticsearch/search/query/SizeParseElement.java index b729ea4cdb2..5560ec939c4 100644 --- a/src/main/java/org/elasticsearch/search/query/SizeParseElement.java +++ b/src/main/java/org/elasticsearch/search/query/SizeParseElement.java @@ -35,7 +35,8 @@ public class SizeParseElement implements SearchParseElement { if (token.isValue()) { int size = parser.intValue(); if (size < 0) { - throw new SearchParseException(context, "size is set to [" + size + "] and is expected to be higher or equal to 0"); + throw new SearchParseException(context, "size is set to [" + size + "] and is expected to be higher or equal to 0", + parser.getTokenLocation()); } context.size(size); } diff --git a/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java b/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java index d0bfebe81f4..7caf89e9c08 100644 --- a/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java +++ b/src/main/java/org/elasticsearch/search/sort/ScriptSortParser.java @@ -118,15 +118,15 @@ public class ScriptSortParser implements SortParser { } if (script == null) { - throw new SearchParseException(context, "_script sorting requires setting the script to sort by"); + throw new SearchParseException(context, "_script sorting requires setting the script to sort by", parser.getTokenLocation()); } if (type == null) { - throw new SearchParseException(context, "_script sorting requires setting the type of the script"); + throw new SearchParseException(context, "_script sorting requires setting the type of the script", parser.getTokenLocation()); } final SearchScript searchScript = context.scriptService().search(context.lookup(), new Script(scriptLang, script, scriptType, params), ScriptContext.Standard.SEARCH); if (STRING_SORT_TYPE.equals(type) && (sortMode == MultiValueMode.SUM || sortMode == MultiValueMode.AVG)) { - throw new SearchParseException(context, "type [string] doesn't support mode [" + sortMode + "]"); + throw new SearchParseException(context, "type [string] doesn't support mode [" + sortMode + "]", parser.getTokenLocation()); } if (sortMode == null) { @@ -196,7 +196,7 @@ public class ScriptSortParser implements SortParser { }; break; default: - throw new SearchParseException(context, "custom script sort type [" + type + "] not supported"); + throw new SearchParseException(context, "custom script sort type [" + type + "] not supported", parser.getTokenLocation()); } return new SortField("_script", fieldComparatorSource, reverse); diff --git a/src/main/java/org/elasticsearch/search/sort/SortParseElement.java b/src/main/java/org/elasticsearch/search/sort/SortParseElement.java index 4723f427dbb..aa2f1315960 100644 --- a/src/main/java/org/elasticsearch/search/sort/SortParseElement.java +++ b/src/main/java/org/elasticsearch/search/sort/SortParseElement.java @@ -212,12 +212,12 @@ public class SortParseElement implements SearchParseElement { if (unmappedType != null) { fieldMapper = context.mapperService().unmappedFieldMapper(unmappedType); } else { - throw new SearchParseException(context, "No mapping found for [" + fieldName + "] in order to sort on"); + throw new SearchParseException(context, "No mapping found for [" + fieldName + "] in order to sort on", null); } } if (!fieldMapper.isSortable()) { - throw new SearchParseException(context, "Sorting not supported for field[" + fieldName + "]"); + throw new SearchParseException(context, "Sorting not supported for field[" + fieldName + "]", null); } // Enable when we also know how to detect fields that do tokenize, but only emit one token diff --git a/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java b/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java index de56f823eac..ef1ef44ffb9 100644 --- a/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java +++ b/src/main/java/org/elasticsearch/search/warmer/IndexWarmersMetaData.java @@ -22,7 +22,9 @@ package org.elasticsearch.search.warmer; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.elasticsearch.Version; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -33,16 +35,33 @@ import org.elasticsearch.common.xcontent.*; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; /** */ -public class IndexWarmersMetaData implements IndexMetaData.Custom { +public class IndexWarmersMetaData extends AbstractDiffable implements IndexMetaData.Custom { public static final String TYPE = "warmers"; - public static final Factory FACTORY = new Factory(); + public static final IndexWarmersMetaData PROTO = new IndexWarmersMetaData(); + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + IndexWarmersMetaData that = (IndexWarmersMetaData) o; + + return entries.equals(that.entries); + + } + + @Override + public int hashCode() { + return entries.hashCode(); + } public static class Entry { private final String name; @@ -74,6 +93,29 @@ public class IndexWarmersMetaData implements IndexMetaData.Custom { public Boolean queryCache() { return this.queryCache; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Entry entry = (Entry) o; + + if (!name.equals(entry.name)) return false; + if (!Arrays.equals(types, entry.types)) return false; + if (!source.equals(entry.source)) return false; + return !(queryCache != null ? !queryCache.equals(entry.queryCache) : entry.queryCache != null); + + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + Arrays.hashCode(types); + result = 31 * result + source.hashCode(); + result = 31 * result + (queryCache != null ? queryCache.hashCode() : 0); + return result; + } } private final ImmutableList entries; @@ -92,149 +134,143 @@ public class IndexWarmersMetaData implements IndexMetaData.Custom { return TYPE; } - public static class Factory implements IndexMetaData.Custom.Factory { - - @Override - public String type() { - return TYPE; + @Override + public IndexWarmersMetaData readFrom(StreamInput in) throws IOException { + Entry[] entries = new Entry[in.readVInt()]; + for (int i = 0; i < entries.length; i++) { + String name = in.readString(); + String[] types = in.readStringArray(); + BytesReference source = null; + if (in.readBoolean()) { + source = in.readBytesReference(); + } + Boolean queryCache; + queryCache = in.readOptionalBoolean(); + entries[i] = new Entry(name, types, queryCache, source); } + return new IndexWarmersMetaData(entries); + } - @Override - public IndexWarmersMetaData readFrom(StreamInput in) throws IOException { - Entry[] entries = new Entry[in.readVInt()]; - for (int i = 0; i < entries.length; i++) { - String name = in.readString(); - String[] types = in.readStringArray(); - BytesReference source = null; - if (in.readBoolean()) { - source = in.readBytesReference(); - } - Boolean queryCache = null; - queryCache = in.readOptionalBoolean(); - entries[i] = new Entry(name, types, queryCache, source); - } - return new IndexWarmersMetaData(entries); - } - - @Override - public void writeTo(IndexWarmersMetaData warmers, StreamOutput out) throws IOException { - out.writeVInt(warmers.entries().size()); - for (Entry entry : warmers.entries()) { - out.writeString(entry.name()); - out.writeStringArray(entry.types()); - if (entry.source() == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeBytesReference(entry.source()); - } - out.writeOptionalBoolean(entry.queryCache()); - } - } - - @Override - public IndexWarmersMetaData fromMap(Map map) throws IOException { - // if it starts with the type, remove it - if (map.size() == 1 && map.containsKey(TYPE)) { - map = (Map) map.values().iterator().next(); - } - XContentBuilder builder = XContentFactory.smileBuilder().map(map); - try (XContentParser parser = XContentFactory.xContent(XContentType.SMILE).createParser(builder.bytes())) { - // move to START_OBJECT - parser.nextToken(); - return fromXContent(parser); - } - } - - @Override - public IndexWarmersMetaData fromXContent(XContentParser parser) throws IOException { - // we get here after we are at warmers token - String currentFieldName = null; - XContentParser.Token token; - List entries = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_OBJECT) { - String name = currentFieldName; - List types = new ArrayList<>(2); - BytesReference source = null; - Boolean queryCache = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_ARRAY) { - if ("types".equals(currentFieldName)) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - types.add(parser.text()); - } - } - } else if (token == XContentParser.Token.START_OBJECT) { - if ("source".equals(currentFieldName)) { - XContentBuilder builder = XContentFactory.jsonBuilder().map(parser.mapOrdered()); - source = builder.bytes(); - } - } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { - if ("source".equals(currentFieldName)) { - source = new BytesArray(parser.binaryValue()); - } - } else if (token.isValue()) { - if ("queryCache".equals(currentFieldName) || "query_cache".equals(currentFieldName)) { - queryCache = parser.booleanValue(); - } - } - } - entries.add(new Entry(name, types.size() == 0 ? Strings.EMPTY_ARRAY : types.toArray(new String[types.size()]), queryCache, source)); - } - } - return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); - } - - @Override - public void toXContent(IndexWarmersMetaData warmers, XContentBuilder builder, ToXContent.Params params) throws IOException { - //No need, IndexMetaData already writes it - //builder.startObject(TYPE, XContentBuilder.FieldCaseConversion.NONE); - for (Entry entry : warmers.entries()) { - toXContent(entry, builder, params); - } - //No need, IndexMetaData already writes it - //builder.endObject(); - } - - public void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { - boolean binary = params.paramAsBoolean("binary", false); - builder.startObject(entry.name(), XContentBuilder.FieldCaseConversion.NONE); - builder.field("types", entry.types()); - if (entry.queryCache() != null) { - builder.field("queryCache", entry.queryCache()); - } - builder.field("source"); - if (binary) { - builder.value(entry.source()); + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(entries().size()); + for (Entry entry : entries()) { + out.writeString(entry.name()); + out.writeStringArray(entry.types()); + if (entry.source() == null) { + out.writeBoolean(false); } else { - Map mapping = XContentFactory.xContent(entry.source()).createParser(entry.source()).mapOrderedAndClose(); - builder.map(mapping); + out.writeBoolean(true); + out.writeBytesReference(entry.source()); } - builder.endObject(); - } - - @Override - public IndexWarmersMetaData merge(IndexWarmersMetaData first, IndexWarmersMetaData second) { - List entries = Lists.newArrayList(); - entries.addAll(first.entries()); - for (Entry secondEntry : second.entries()) { - boolean found = false; - for (Entry firstEntry : first.entries()) { - if (firstEntry.name().equals(secondEntry.name())) { - found = true; - break; - } - } - if (!found) { - entries.add(secondEntry); - } - } - return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + out.writeOptionalBoolean(entry.queryCache()); } } + + @Override + public IndexWarmersMetaData fromMap(Map map) throws IOException { + // if it starts with the type, remove it + if (map.size() == 1 && map.containsKey(TYPE)) { + map = (Map) map.values().iterator().next(); + } + XContentBuilder builder = XContentFactory.smileBuilder().map(map); + try (XContentParser parser = XContentFactory.xContent(XContentType.SMILE).createParser(builder.bytes())) { + // move to START_OBJECT + parser.nextToken(); + return fromXContent(parser); + } + } + + @Override + public IndexWarmersMetaData fromXContent(XContentParser parser) throws IOException { + // we get here after we are at warmers token + String currentFieldName = null; + XContentParser.Token token; + List entries = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + String name = currentFieldName; + List types = new ArrayList<>(2); + BytesReference source = null; + Boolean queryCache = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_ARRAY) { + if ("types".equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + types.add(parser.text()); + } + } + } else if (token == XContentParser.Token.START_OBJECT) { + if ("source".equals(currentFieldName)) { + XContentBuilder builder = XContentFactory.jsonBuilder().map(parser.mapOrdered()); + source = builder.bytes(); + } + } else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { + if ("source".equals(currentFieldName)) { + source = new BytesArray(parser.binaryValue()); + } + } else if (token.isValue()) { + if ("queryCache".equals(currentFieldName) || "query_cache".equals(currentFieldName)) { + queryCache = parser.booleanValue(); + } + } + } + entries.add(new Entry(name, types.size() == 0 ? Strings.EMPTY_ARRAY : types.toArray(new String[types.size()]), queryCache, source)); + } + } + return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + //No need, IndexMetaData already writes it + //builder.startObject(TYPE, XContentBuilder.FieldCaseConversion.NONE); + for (Entry entry : entries()) { + toXContent(entry, builder, params); + } + //No need, IndexMetaData already writes it + //builder.endObject(); + return builder; + } + + public static void toXContent(Entry entry, XContentBuilder builder, ToXContent.Params params) throws IOException { + boolean binary = params.paramAsBoolean("binary", false); + builder.startObject(entry.name(), XContentBuilder.FieldCaseConversion.NONE); + builder.field("types", entry.types()); + if (entry.queryCache() != null) { + builder.field("queryCache", entry.queryCache()); + } + builder.field("source"); + if (binary) { + builder.value(entry.source()); + } else { + Map mapping = XContentFactory.xContent(entry.source()).createParser(entry.source()).mapOrderedAndClose(); + builder.map(mapping); + } + builder.endObject(); + } + + @Override + public IndexMetaData.Custom mergeWith(IndexMetaData.Custom other) { + IndexWarmersMetaData second = (IndexWarmersMetaData) other; + List entries = Lists.newArrayList(); + entries.addAll(entries()); + for (Entry secondEntry : second.entries()) { + boolean found = false; + for (Entry firstEntry : entries()) { + if (firstEntry.name().equals(secondEntry.name())) { + found = true; + break; + } + } + if (!found) { + entries.add(secondEntry); + } + } + return new IndexWarmersMetaData(entries.toArray(new Entry[entries.size()])); + } } diff --git a/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java b/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java index e88489c7e98..16914ab6eef 100644 --- a/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java +++ b/src/test/java/org/elasticsearch/ElasticsearchExceptionTests.java @@ -29,7 +29,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexException; import org.elasticsearch.index.query.QueryParsingException; -import org.elasticsearch.indices.IndexClosedException; +import org.elasticsearch.index.query.TestQueryParsingException; import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchShardTarget; @@ -73,15 +73,17 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { assertEquals(rootCauses.length, 1); assertEquals(ElasticsearchException.getExceptionName(rootCauses[0]), "index_exception"); assertEquals(rootCauses[0].getMessage(), "index is closed"); - ShardSearchFailure failure = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 1)); - ShardSearchFailure failure1 = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 2)); + ShardSearchFailure failure = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 1)); + ShardSearchFailure failure1 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 2)); SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[]{failure, failure1}); if (randomBoolean()) { rootCauses = (randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex).guessRootCauses(); } else { rootCauses = ElasticsearchException.guessRootCauses(randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex); } - assertEquals(ElasticsearchException.getExceptionName(rootCauses[0]), "query_parsing_exception"); + assertEquals(ElasticsearchException.getExceptionName(rootCauses[0]), "test_query_parsing_exception"); assertEquals(rootCauses[0].getMessage(), "foobar"); ElasticsearchException oneLevel = new ElasticsearchException("foo", new RuntimeException("foobar")); @@ -90,18 +92,23 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { assertEquals(rootCauses[0].getMessage(), "foo"); } { - ShardSearchFailure failure = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 1)); - ShardSearchFailure failure1 = new ShardSearchFailure(new QueryParsingException(new Index("foo1"), "foobar"), new SearchShardTarget("node_1", "foo1", 1)); - ShardSearchFailure failure2 = new ShardSearchFailure(new QueryParsingException(new Index("foo1"), "foobar"), new SearchShardTarget("node_1", "foo1", 2)); + ShardSearchFailure failure = new ShardSearchFailure( + new TestQueryParsingException(new Index("foo"), 1, 2, "foobar", null), + new SearchShardTarget("node_1", "foo", 1)); + ShardSearchFailure failure1 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo1"), 1, 2, "foobar", null), + new SearchShardTarget("node_1", "foo1", 1)); + ShardSearchFailure failure2 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo1"), 1, 2, "foobar", null), + new SearchShardTarget("node_1", "foo1", 2)); SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[]{failure, failure1, failure2}); final ElasticsearchException[] rootCauses = ex.guessRootCauses(); assertEquals(rootCauses.length, 2); - assertEquals(ElasticsearchException.getExceptionName(rootCauses[0]), "query_parsing_exception"); + assertEquals(ElasticsearchException.getExceptionName(rootCauses[0]), "test_query_parsing_exception"); assertEquals(rootCauses[0].getMessage(), "foobar"); assertEquals(((QueryParsingException)rootCauses[0]).index().name(), "foo"); - assertEquals(ElasticsearchException.getExceptionName(rootCauses[1]), "query_parsing_exception"); + assertEquals(ElasticsearchException.getExceptionName(rootCauses[1]), "test_query_parsing_exception"); assertEquals(rootCauses[1].getMessage(), "foobar"); - assertEquals(((QueryParsingException)rootCauses[1]).index().name(), "foo1"); + assertEquals(((QueryParsingException) rootCauses[1]).getLineNumber(), 1); + assertEquals(((QueryParsingException) rootCauses[1]).getColumnNumber(), 2); } @@ -118,26 +125,31 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { public void testDeduplicate() throws IOException { { - ShardSearchFailure failure = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 1)); - ShardSearchFailure failure1 = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 2)); + ShardSearchFailure failure = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 1)); + ShardSearchFailure failure1 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 2)); SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[]{failure, failure1}); XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); ex.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); - String expected = "{\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}}]}"; + String expected = "{\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}}]}"; assertEquals(expected, builder.string()); } { - ShardSearchFailure failure = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 1)); - ShardSearchFailure failure1 = new ShardSearchFailure(new QueryParsingException(new Index("foo1"), "foobar"), new SearchShardTarget("node_1", "foo1", 1)); - ShardSearchFailure failure2 = new ShardSearchFailure(new QueryParsingException(new Index("foo1"), "foobar"), new SearchShardTarget("node_1", "foo1", 2)); + ShardSearchFailure failure = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 1)); + ShardSearchFailure failure1 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo1"), "foobar", null), + new SearchShardTarget("node_1", "foo1", 1)); + ShardSearchFailure failure2 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo1"), "foobar", null), + new SearchShardTarget("node_1", "foo1", 2)); SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[]{failure, failure1, failure2}); XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); ex.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); - String expected = "{\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}},{\"shard\":1,\"index\":\"foo1\",\"node\":\"node_1\",\"reason\":{\"type\":\"query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo1\"}}]}"; + String expected = "{\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}},{\"shard\":1,\"index\":\"foo1\",\"node\":\"node_1\",\"reason\":{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo1\"}}]}"; assertEquals(expected, builder.string()); } } @@ -182,6 +194,16 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { assertEquals(expected, builder.string()); } + { + QueryParsingException ex = new TestQueryParsingException(new Index("foo"), 1, 2, "foobar", null); + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + ElasticsearchException.toXContent(builder, ToXContent.EMPTY_PARAMS, ex); + builder.endObject(); + String expected = "{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"line\":1,\"col\":2,\"index\":\"foo\"}"; + assertEquals(expected, builder.string()); + } + { // test equivalence ElasticsearchException ex = new RemoteTransportException("foobar", new FileNotFoundException("foo not found")); XContentBuilder builder = XContentFactory.jsonBuilder(); @@ -200,13 +222,15 @@ public class ElasticsearchExceptionTests extends ElasticsearchTestCase { public void testSerializeElasticsearchException() throws IOException { BytesStreamOutput out = new BytesStreamOutput(); - QueryParsingException ex = new QueryParsingException(new Index("foo"), "foobar"); + QueryParsingException ex = new TestQueryParsingException(new Index("foo"), 1, 2, "foobar", null); out.writeThrowable(ex); BytesStreamInput in = new BytesStreamInput(out.bytes()); QueryParsingException e = in.readThrowable(); assertEquals(ex.index(), e.index()); assertEquals(ex.getMessage(), e.getMessage()); + assertEquals(ex.getLineNumber(), e.getLineNumber()); + assertEquals(ex.getColumnNumber(), e.getColumnNumber()); } } \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java new file mode 100644 index 00000000000..33008fd63d2 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffPublishingTests.java @@ -0,0 +1,625 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.discovery.Discovery; +import org.elasticsearch.discovery.DiscoverySettings; +import org.elasticsearch.discovery.zen.DiscoveryNodesProvider; +import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction; +import org.elasticsearch.node.service.NodeService; +import org.elasticsearch.node.settings.NodeSettingsService; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportConnectionListener; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.transport.local.LocalTransport; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.collect.Maps.newHashMap; +import static org.hamcrest.Matchers.*; + +public class ClusterStateDiffPublishingTests extends ElasticsearchTestCase { + + protected ThreadPool threadPool; + protected Map nodes = newHashMap(); + + public static class MockNode { + public final DiscoveryNode discoveryNode; + public final MockTransportService service; + public final PublishClusterStateAction action; + public final MockDiscoveryNodesProvider nodesProvider; + + public MockNode(DiscoveryNode discoveryNode, MockTransportService service, PublishClusterStateAction action, MockDiscoveryNodesProvider nodesProvider) { + this.discoveryNode = discoveryNode; + this.service = service; + this.action = action; + this.nodesProvider = nodesProvider; + } + + public void connectTo(DiscoveryNode node) { + service.connectToNode(node); + nodesProvider.addNode(node); + } + } + + public MockNode createMockNode(final String name, Settings settings, Version version) throws Exception { + return createMockNode(name, settings, version, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + logger.debug("Node [{}] onNewClusterState version [{}], uuid [{}]", name, clusterState.version(), clusterState.uuid()); + newStateProcessed.onNewClusterStateProcessed(); + } + }); + } + + public MockNode createMockNode(String name, Settings settings, Version version, PublishClusterStateAction.NewClusterStateListener listener) throws Exception { + MockTransportService service = buildTransportService( + ImmutableSettings.builder().put(settings).put("name", name, TransportService.SETTING_TRACE_LOG_INCLUDE, "", TransportService.SETTING_TRACE_LOG_EXCLUDE, "NOTHING").build(), + version + ); + DiscoveryNode discoveryNode = new DiscoveryNode(name, name, service.boundAddress().publishAddress(), ImmutableMap.of(), version); + MockDiscoveryNodesProvider nodesProvider = new MockDiscoveryNodesProvider(discoveryNode); + PublishClusterStateAction action = buildPublishClusterStateAction(settings, service, nodesProvider, listener); + MockNode node = new MockNode(discoveryNode, service, action, nodesProvider); + nodesProvider.addNode(discoveryNode); + final CountDownLatch latch = new CountDownLatch(nodes.size() * 2 + 1); + TransportConnectionListener waitForConnection = new TransportConnectionListener() { + @Override + public void onNodeConnected(DiscoveryNode node) { + latch.countDown(); + } + + @Override + public void onNodeDisconnected(DiscoveryNode node) { + fail("disconnect should not be called " + node); + } + }; + node.service.addConnectionListener(waitForConnection); + for (MockNode curNode : nodes.values()) { + curNode.service.addConnectionListener(waitForConnection); + curNode.connectTo(node.discoveryNode); + node.connectTo(curNode.discoveryNode); + } + node.connectTo(node.discoveryNode); + assertThat("failed to wait for all nodes to connect", latch.await(5, TimeUnit.SECONDS), equalTo(true)); + for (MockNode curNode : nodes.values()) { + curNode.service.removeConnectionListener(waitForConnection); + } + node.service.removeConnectionListener(waitForConnection); + if (nodes.put(name, node) != null) { + fail("Node with the name " + name + " already exist"); + } + return node; + } + + public MockTransportService service(String name) { + MockNode node = nodes.get(name); + if (node != null) { + return node.service; + } + return null; + } + + public PublishClusterStateAction action(String name) { + MockNode node = nodes.get(name); + if (node != null) { + return node.action; + } + return null; + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + threadPool = new ThreadPool(getClass().getName()); + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + for (MockNode curNode : nodes.values()) { + curNode.action.close(); + curNode.service.close(); + } + terminate(threadPool); + } + + protected MockTransportService buildTransportService(Settings settings, Version version) { + MockTransportService transportService = new MockTransportService(settings, new LocalTransport(settings, threadPool, version), threadPool); + transportService.start(); + return transportService; + } + + protected PublishClusterStateAction buildPublishClusterStateAction(Settings settings, MockTransportService transportService, MockDiscoveryNodesProvider nodesProvider, + PublishClusterStateAction.NewClusterStateListener listener) { + DiscoverySettings discoverySettings = new DiscoverySettings(settings, new NodeSettingsService(settings)); + return new PublishClusterStateAction(settings, transportService, nodesProvider, listener, discoverySettings); + } + + + static class MockDiscoveryNodesProvider implements DiscoveryNodesProvider { + + private DiscoveryNodes discoveryNodes = DiscoveryNodes.EMPTY_NODES; + + public MockDiscoveryNodesProvider(DiscoveryNode localNode) { + discoveryNodes = DiscoveryNodes.builder().put(localNode).localNodeId(localNode.id()).build(); + } + + public void addNode(DiscoveryNode node) { + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(node).build(); + } + + @Override + public DiscoveryNodes nodes() { + return discoveryNodes; + } + + @Override + public NodeService nodeService() { + assert false; + throw new UnsupportedOperationException("Shouldn't be here"); + } + } + + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSimpleClusterStatePublishing() throws Exception { + MockNewClusterStateListener mockListenerA = new MockNewClusterStateListener(); + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerA); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + + // cluster state update - add nodeB + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build(); + ClusterState previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(1)); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - remove block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // Adding new node - this node should get full cluster state while nodeB should still be getting diffs + + MockNewClusterStateListener mockListenerC = new MockNewClusterStateListener(); + MockNode nodeC = createMockNode("nodeC", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerC); + + // cluster state update 3 - register node C + previousClusterState = clusterState; + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeC.discoveryNode).build(); + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }); + mockListenerC.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + // First state + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update 4 - update settings + previousClusterState = clusterState; + MetaData metaData = MetaData.builder(clusterState.metaData()).transientSettings(ImmutableSettings.settingsBuilder().put("foo", "bar").build()).build(); + clusterState = ClusterState.builder(clusterState).metaData(metaData).incrementVersion().build(); + NewClusterStateExpectation expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + assertThat(clusterState.blocks().global().size(), equalTo(0)); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - skipping one version change - should request full cluster state + previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); + clusterState = ClusterState.builder(clusterState).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - skipping one version change - should request full cluster state + previousClusterState = ClusterState.builder(clusterState).incrementVersion().build(); + clusterState = ClusterState.builder(clusterState).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerB.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // node B becomes the master and sends a version of the cluster state that goes back + discoveryNodes = DiscoveryNodes.builder(discoveryNodes) + .put(nodeA.discoveryNode) + .put(nodeB.discoveryNode) + .put(nodeC.discoveryNode) + .build(); + previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + expectation = new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }; + mockListenerA.add(expectation); + mockListenerC.add(expectation); + publishStateDiffAndWait(nodeB.action, clusterState, previousClusterState); + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testUnexpectedDiffPublishing() throws Exception { + + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testDisablingDiffPublishing() throws Exception { + Settings noDiffPublishingSettings = ImmutableSettings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, false).build(); + + MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNode nodeB = createMockNode("nodeB", noDiffPublishingSettings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + logger.debug("Got cluster state update, version [{}], guid [{}], from diff [{}]", clusterState.version(), clusterState.uuid(), clusterState.wasReadFromDiff()); + assertFalse(clusterState.wasReadFromDiff()); + newStateProcessed.onNewClusterStateProcessed(); + } + }); + + // Initial cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + + // cluster state update - add nodeB + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).put(nodeB.discoveryNode).build(); + ClusterState previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).nodes(discoveryNodes).incrementVersion().build(); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + } + + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSimultaneousClusterStatePublishing() throws Exception { + int numberOfNodes = randomIntBetween(2, 10); + int numberOfIterations = randomIntBetween(50, 200); + Settings settings = ImmutableSettings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "100ms").put(DiscoverySettings.PUBLISH_DIFF_ENABLE, true).build(); + MockNode[] nodes = new MockNode[numberOfNodes]; + DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < nodes.length; i++) { + final String name = "node" + i; + nodes[i] = createMockNode(name, settings, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public synchronized void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + assertProperMetaDataForVersion(clusterState.metaData(), clusterState.version()); + if (randomInt(10) < 2) { + // Cause timeouts from time to time + try { + Thread.sleep(randomInt(110)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + newStateProcessed.onNewClusterStateProcessed(); + } + }); + discoveryNodesBuilder.put(nodes[i].discoveryNode); + } + + AssertingAckListener[] listeners = new AssertingAckListener[numberOfIterations]; + DiscoveryNodes discoveryNodes = discoveryNodesBuilder.build(); + MetaData metaData = MetaData.EMPTY_META_DATA; + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).metaData(metaData).build(); + ClusterState previousState; + for (int i = 0; i < numberOfIterations; i++) { + previousState = clusterState; + metaData = buildMetaDataForVersion(metaData, i + 1); + clusterState = ClusterState.builder(clusterState).incrementVersion().metaData(metaData).nodes(discoveryNodes).build(); + listeners[i] = publishStateDiff(nodes[0].action, clusterState, previousState); + } + + for (int i = 0; i < numberOfIterations; i++) { + listeners[i].await(1, TimeUnit.SECONDS); + } + } + + @Test + @TestLogging("cluster:DEBUG,discovery.zen.publish:DEBUG") + public void testSerializationFailureDuringDiffPublishing() throws Exception { + + MockNode nodeA = createMockNode("nodeA", ImmutableSettings.EMPTY, Version.CURRENT, new PublishClusterStateAction.NewClusterStateListener() { + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + fail("Shouldn't send cluster state to myself"); + } + }); + + MockNewClusterStateListener mockListenerB = new MockNewClusterStateListener(); + MockNode nodeB = createMockNode("nodeB", ImmutableSettings.EMPTY, Version.CURRENT, mockListenerB); + + // Initial cluster state with both states - the second node still shouldn't get diff even though it's present in the previous cluster state + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(nodeA.discoveryNode).put(nodeB.discoveryNode).localNodeId(nodeA.discoveryNode.id()).build(); + ClusterState previousClusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterState = ClusterState.builder(previousClusterState).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertFalse(clusterState.wasReadFromDiff()); + } + }); + publishStateDiffAndWait(nodeA.action, clusterState, previousClusterState); + + // cluster state update - add block + previousClusterState = clusterState; + clusterState = ClusterState.builder(clusterState).blocks(ClusterBlocks.builder().addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK)).incrementVersion().build(); + mockListenerB.add(new NewClusterStateExpectation() { + @Override + public void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed) { + assertTrue(clusterState.wasReadFromDiff()); + } + }); + + ClusterState unserializableClusterState = new ClusterState(clusterState.version(), clusterState.uuid(), clusterState) { + @Override + public Diff diff(ClusterState previousState) { + return new Diff() { + @Override + public ClusterState apply(ClusterState part) { + fail("this diff shouldn't be applied"); + return part; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + throw new IOException("Simulated failure of diff serialization"); + } + }; + } + }; + List> errors = publishStateDiff(nodeA.action, unserializableClusterState, previousClusterState).awaitErrors(1, TimeUnit.SECONDS); + assertThat(errors.size(), equalTo(1)); + assertThat(errors.get(0).v2().getMessage(), containsString("Simulated failure of diff serialization")); + } + + private MetaData buildMetaDataForVersion(MetaData metaData, long version) { + ImmutableOpenMap.Builder indices = ImmutableOpenMap.builder(metaData.indices()); + indices.put("test" + version, IndexMetaData.builder("test" + version).settings(ImmutableSettings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards((int) version).numberOfReplicas(0).build()); + return MetaData.builder(metaData) + .transientSettings(ImmutableSettings.builder().put("test", version).build()) + .indices(indices.build()) + .build(); + } + + private void assertProperMetaDataForVersion(MetaData metaData, long version) { + for (long i = 1; i <= version; i++) { + assertThat(metaData.index("test" + i), notNullValue()); + assertThat(metaData.index("test" + i).numberOfShards(), equalTo((int) i)); + } + assertThat(metaData.index("test" + (version + 1)), nullValue()); + assertThat(metaData.transientSettings().get("test"), equalTo(Long.toString(version))); + } + + public void publishStateDiffAndWait(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException { + publishStateDiff(action, state, previousState).await(1, TimeUnit.SECONDS); + } + + public AssertingAckListener publishStateDiff(PublishClusterStateAction action, ClusterState state, ClusterState previousState) throws InterruptedException { + AssertingAckListener assertingAckListener = new AssertingAckListener(state.nodes().getSize() - 1); + ClusterChangedEvent changedEvent = new ClusterChangedEvent("test update", state, previousState); + action.publish(changedEvent, assertingAckListener); + return assertingAckListener; + } + + public static class AssertingAckListener implements Discovery.AckListener { + private final List> errors = new CopyOnWriteArrayList<>(); + private final AtomicBoolean timeoutOccured = new AtomicBoolean(); + private final CountDownLatch countDown; + + public AssertingAckListener(int nodeCount) { + countDown = new CountDownLatch(nodeCount); + } + + @Override + public void onNodeAck(DiscoveryNode node, @Nullable Throwable t) { + if (t != null) { + errors.add(new Tuple<>(node, t)); + } + countDown.countDown(); + } + + @Override + public void onTimeout() { + timeoutOccured.set(true); + // Fast forward the counter - no reason to wait here + long currentCount = countDown.getCount(); + for (long i = 0; i < currentCount; i++) { + countDown.countDown(); + } + } + + public void await(long timeout, TimeUnit unit) throws InterruptedException { + assertThat(awaitErrors(timeout, unit), emptyIterable()); + } + + public List> awaitErrors(long timeout, TimeUnit unit) throws InterruptedException { + countDown.await(timeout, unit); + assertFalse(timeoutOccured.get()); + return errors; + } + + } + + public interface NewClusterStateExpectation { + void check(ClusterState clusterState, PublishClusterStateAction.NewClusterStateListener.NewStateProcessed newStateProcessed); + } + + public static class MockNewClusterStateListener implements PublishClusterStateAction.NewClusterStateListener { + CopyOnWriteArrayList expectations = new CopyOnWriteArrayList(); + + @Override + public void onNewClusterState(ClusterState clusterState, NewStateProcessed newStateProcessed) { + final NewClusterStateExpectation expectation; + try { + expectation = expectations.remove(0); + } catch (ArrayIndexOutOfBoundsException ex) { + fail("Unexpected cluster state update " + clusterState.prettyPrint()); + return; + } + expectation.check(clusterState, newStateProcessed); + newStateProcessed.onNewClusterStateProcessed(); + } + + public void add(NewClusterStateExpectation expectation) { + expectations.add(expectation); + } + } + + public static class DelegatingClusterState extends ClusterState { + + public DelegatingClusterState(ClusterState clusterState) { + super(clusterState.version(), clusterState.uuid(), clusterState); + } + + + } + +} diff --git a/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java new file mode 100644 index 00000000000..b49b7586dc3 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/ClusterStateDiffTests.java @@ -0,0 +1,534 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.*; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.*; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.BytesStreamInput; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.discovery.DiscoverySettings; +import org.elasticsearch.gateway.GatewayService; +import org.elasticsearch.index.query.FilterBuilders; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.warmer.IndexWarmersMetaData; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import java.util.List; + +import static org.elasticsearch.cluster.metadata.AliasMetaData.newAliasMetaDataBuilder; +import static org.elasticsearch.test.XContentTestUtils.convertToMap; +import static org.elasticsearch.test.XContentTestUtils.mapsEqualIgnoringArrayOrder; +import static org.elasticsearch.test.VersionUtils.randomVersion; +import static org.hamcrest.Matchers.equalTo; + + +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE, numDataNodes = 0, numClientNodes = 0) +public class ClusterStateDiffTests extends ElasticsearchIntegrationTest { + + @Test + public void testClusterStateDiffSerialization() throws Exception { + DiscoveryNode masterNode = new DiscoveryNode("master", new LocalTransportAddress("master"), Version.CURRENT); + DiscoveryNode otherNode = new DiscoveryNode("other", new LocalTransportAddress("other"), Version.CURRENT); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().put(masterNode).put(otherNode).localNodeId(masterNode.id()).build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + ClusterState clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), otherNode); + + int iterationCount = randomIntBetween(10, 300); + for (int iteration = 0; iteration < iterationCount; iteration++) { + ClusterState previousClusterState = clusterState; + ClusterState previousClusterStateFromDiffs = clusterStateFromDiffs; + int changesCount = randomIntBetween(1, 4); + ClusterState.Builder builder = null; + for (int i = 0; i < changesCount; i++) { + if (i > 0) { + clusterState = builder.build(); + } + switch (randomInt(4)) { + case 0: + builder = randomNodes(clusterState); + break; + case 1: + builder = randomRoutingTable(clusterState); + break; + case 2: + builder = randomBlocks(clusterState); + break; + case 3: + case 4: + builder = randomMetaDataChanges(clusterState); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + clusterState = builder.incrementVersion().build(); + + if (randomIntBetween(0, 10) < 1) { + // Update cluster state via full serialization from time to time + clusterStateFromDiffs = ClusterState.Builder.fromBytes(ClusterState.Builder.toBytes(clusterState), previousClusterStateFromDiffs.nodes().localNode()); + } else { + // Update cluster states using diffs + Diff diffBeforeSerialization = clusterState.diff(previousClusterState); + BytesStreamOutput os = new BytesStreamOutput(); + diffBeforeSerialization.writeTo(os); + byte[] diffBytes = os.bytes().toBytes(); + Diff diff; + try (BytesStreamInput input = new BytesStreamInput(diffBytes)) { + diff = previousClusterStateFromDiffs.readDiffFrom(input); + clusterStateFromDiffs = diff.apply(previousClusterStateFromDiffs); + } + } + + + try { + // Check non-diffable elements + assertThat(clusterStateFromDiffs.version(), equalTo(clusterState.version())); + assertThat(clusterStateFromDiffs.uuid(), equalTo(clusterState.uuid())); + + // Check nodes + assertThat(clusterStateFromDiffs.nodes().nodes(), equalTo(clusterState.nodes().nodes())); + assertThat(clusterStateFromDiffs.nodes().localNodeId(), equalTo(previousClusterStateFromDiffs.nodes().localNodeId())); + assertThat(clusterStateFromDiffs.nodes().nodes(), equalTo(clusterState.nodes().nodes())); + for (ObjectCursor node : clusterStateFromDiffs.nodes().nodes().keys()) { + DiscoveryNode node1 = clusterState.nodes().get(node.value); + DiscoveryNode node2 = clusterStateFromDiffs.nodes().get(node.value); + assertThat(node1.version(), equalTo(node2.version())); + assertThat(node1.address(), equalTo(node2.address())); + assertThat(node1.attributes(), equalTo(node2.attributes())); + } + + // Check routing table + assertThat(clusterStateFromDiffs.routingTable().version(), equalTo(clusterState.routingTable().version())); + assertThat(clusterStateFromDiffs.routingTable().indicesRouting(), equalTo(clusterState.routingTable().indicesRouting())); + + // Check cluster blocks + assertThat(clusterStateFromDiffs.blocks().global(), equalTo(clusterStateFromDiffs.blocks().global())); + assertThat(clusterStateFromDiffs.blocks().indices(), equalTo(clusterStateFromDiffs.blocks().indices())); + assertThat(clusterStateFromDiffs.blocks().disableStatePersistence(), equalTo(clusterStateFromDiffs.blocks().disableStatePersistence())); + + // Check metadata + assertThat(clusterStateFromDiffs.metaData().version(), equalTo(clusterState.metaData().version())); + assertThat(clusterStateFromDiffs.metaData().uuid(), equalTo(clusterState.metaData().uuid())); + assertThat(clusterStateFromDiffs.metaData().transientSettings(), equalTo(clusterState.metaData().transientSettings())); + assertThat(clusterStateFromDiffs.metaData().persistentSettings(), equalTo(clusterState.metaData().persistentSettings())); + assertThat(clusterStateFromDiffs.metaData().indices(), equalTo(clusterState.metaData().indices())); + assertThat(clusterStateFromDiffs.metaData().templates(), equalTo(clusterState.metaData().templates())); + assertThat(clusterStateFromDiffs.metaData().customs(), equalTo(clusterState.metaData().customs())); + assertThat(clusterStateFromDiffs.metaData().aliases(), equalTo(clusterState.metaData().aliases())); + + // JSON Serialization test - make sure that both states produce similar JSON + assertThat(mapsEqualIgnoringArrayOrder(convertToMap(clusterStateFromDiffs), convertToMap(clusterState)), equalTo(true)); + + // Smoke test - we cannot compare bytes to bytes because some elements might get serialized in different order + // however, serialized size should remain the same + assertThat(ClusterState.Builder.toBytes(clusterStateFromDiffs).length, equalTo(ClusterState.Builder.toBytes(clusterState).length)); + } catch (AssertionError error) { + logger.error("Cluster state:\n{}\nCluster state from diffs:\n{}", clusterState.toString(), clusterStateFromDiffs.toString()); + throw error; + } + } + + logger.info("Final cluster state:[{}]", clusterState.toString()); + + } + + private ClusterState.Builder randomNodes(ClusterState clusterState) { + DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterState.nodes()); + List nodeIds = randomSubsetOf(randomInt(clusterState.nodes().nodes().size() - 1), clusterState.nodes().nodes().keys().toArray(String.class)); + for (String nodeId : nodeIds) { + if (nodeId.startsWith("node-")) { + if (randomBoolean()) { + nodes.remove(nodeId); + } else { + nodes.put(new DiscoveryNode(nodeId, new LocalTransportAddress(randomAsciiOfLength(10)), randomVersion(random()))); + } + } + } + int additionalNodeCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalNodeCount; i++) { + nodes.put(new DiscoveryNode("node-" + randomAsciiOfLength(10), new LocalTransportAddress(randomAsciiOfLength(10)), randomVersion(random()))); + } + return ClusterState.builder(clusterState).nodes(nodes); + } + + private ClusterState.Builder randomRoutingTable(ClusterState clusterState) { + RoutingTable.Builder builder = RoutingTable.builder(clusterState.routingTable()); + int numberOfIndices = clusterState.routingTable().indicesRouting().size(); + if (numberOfIndices > 0) { + List randomIndices = randomSubsetOf(randomInt(numberOfIndices - 1), clusterState.routingTable().indicesRouting().keySet().toArray(new String[numberOfIndices])); + for (String index : randomIndices) { + if (randomBoolean()) { + builder.remove(index); + } else { + builder.add(randomIndexRoutingTable(index, clusterState.nodes().nodes().keys().toArray(String.class))); + } + } + } + int additionalIndexCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalIndexCount; i++) { + builder.add(randomIndexRoutingTable("index-" + randomInt(), clusterState.nodes().nodes().keys().toArray(String.class))); + } + return ClusterState.builder(clusterState).routingTable(builder.build()); + } + + private IndexRoutingTable randomIndexRoutingTable(String index, String[] nodeIds) { + IndexRoutingTable.Builder builder = IndexRoutingTable.builder(index); + int shardCount = randomInt(10); + + for (int i = 0; i < shardCount; i++) { + IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(new ShardId(index, i), randomBoolean()); + int replicaCount = randomIntBetween(1, 10); + for (int j = 0; j < replicaCount; j++) { + indexShard.addShard( + new MutableShardRouting(index, i, randomFrom(nodeIds), j == 0, ShardRoutingState.fromValue((byte) randomIntBetween(1, 4)), 1)); + } + builder.addIndexShard(indexShard.build()); + } + return builder.build(); + } + + private ClusterState.Builder randomBlocks(ClusterState clusterState) { + ClusterBlocks.Builder builder = ClusterBlocks.builder().blocks(clusterState.blocks()); + int globalBlocksCount = clusterState.blocks().global().size(); + if (globalBlocksCount > 0) { + List blocks = randomSubsetOf(randomInt(globalBlocksCount - 1), clusterState.blocks().global().toArray(new ClusterBlock[globalBlocksCount])); + for (ClusterBlock block : blocks) { + builder.removeGlobalBlock(block); + } + } + int additionalGlobalBlocksCount = randomIntBetween(1, 3); + for (int i = 0; i < additionalGlobalBlocksCount; i++) { + builder.addGlobalBlock(randomGlobalBlock()); + } + return ClusterState.builder(clusterState).blocks(builder); + } + + private ClusterBlock randomGlobalBlock() { + switch (randomInt(2)) { + case 0: + return DiscoverySettings.NO_MASTER_BLOCK_ALL; + case 1: + return DiscoverySettings.NO_MASTER_BLOCK_WRITES; + default: + return GatewayService.STATE_NOT_RECOVERED_BLOCK; + } + } + + private ClusterState.Builder randomMetaDataChanges(ClusterState clusterState) { + MetaData metaData = clusterState.metaData(); + int changesCount = randomIntBetween(1, 10); + for (int i = 0; i < changesCount; i++) { + switch (randomInt(3)) { + case 0: + metaData = randomMetaDataSettings(metaData); + break; + case 1: + metaData = randomIndices(metaData); + break; + case 2: + metaData = randomTemplates(metaData); + break; + case 3: + metaData = randomMetaDataCustoms(metaData); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + return ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).version(metaData.version() + 1).build()); + } + + private Settings randomSettings(Settings settings) { + ImmutableSettings.Builder builder = ImmutableSettings.builder(); + if (randomBoolean()) { + builder.put(settings); + } + int settingsCount = randomInt(10); + for (int i = 0; i < settingsCount; i++) { + builder.put(randomAsciiOfLength(10), randomAsciiOfLength(10)); + } + return builder.build(); + + } + + private MetaData randomMetaDataSettings(MetaData metaData) { + if (randomBoolean()) { + return MetaData.builder(metaData).persistentSettings(randomSettings(metaData.persistentSettings())).build(); + } else { + return MetaData.builder(metaData).transientSettings(randomSettings(metaData.transientSettings())).build(); + } + } + + private interface RandomPart { + /** + * Returns list of parts from metadata + */ + ImmutableOpenMap parts(MetaData metaData); + + /** + * Puts the part back into metadata + */ + MetaData.Builder put(MetaData.Builder builder, T part); + + /** + * Remove the part from metadata + */ + MetaData.Builder remove(MetaData.Builder builder, String name); + + /** + * Returns a random part with the specified name + */ + T randomCreate(String name); + + /** + * Makes random modifications to the part + */ + T randomChange(T part); + + } + + private MetaData randomParts(MetaData metaData, String prefix, RandomPart randomPart) { + MetaData.Builder builder = MetaData.builder(metaData); + ImmutableOpenMap parts = randomPart.parts(metaData); + int partCount = parts.size(); + if (partCount > 0) { + List randomParts = randomSubsetOf(randomInt(partCount - 1), randomPart.parts(metaData).keys().toArray(String.class)); + for (String part : randomParts) { + if (randomBoolean()) { + randomPart.remove(builder, part); + } else { + randomPart.put(builder, randomPart.randomChange(parts.get(part))); + } + } + } + int additionalPartCount = randomIntBetween(1, 20); + for (int i = 0; i < additionalPartCount; i++) { + String name = randomName(prefix); + randomPart.put(builder, randomPart.randomCreate(name)); + } + return builder.build(); + } + + private MetaData randomIndices(MetaData metaData) { + return randomParts(metaData, "index", new RandomPart() { + + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.indices(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, IndexMetaData part) { + return builder.put(part, true); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.remove(name); + } + + @Override + public IndexMetaData randomCreate(String name) { + IndexMetaData.Builder builder = IndexMetaData.builder(name); + ImmutableSettings.Builder settingsBuilder = ImmutableSettings.builder(); + setRandomSettings(getRandom(), settingsBuilder); + settingsBuilder.put(randomSettings(ImmutableSettings.EMPTY)).put(IndexMetaData.SETTING_VERSION_CREATED, randomVersion(random())); + builder.settings(settingsBuilder); + builder.numberOfShards(randomIntBetween(1, 10)).numberOfReplicas(randomInt(10)); + int aliasCount = randomInt(10); + if (randomBoolean()) { + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + } + for (int i = 0; i < aliasCount; i++) { + builder.putAlias(randomAlias()); + } + return builder.build(); + } + + @Override + public IndexMetaData randomChange(IndexMetaData part) { + IndexMetaData.Builder builder = IndexMetaData.builder(part); + switch (randomIntBetween(0, 3)) { + case 0: + builder.settings(ImmutableSettings.builder().put(part.settings()).put(randomSettings(ImmutableSettings.EMPTY))); + break; + case 1: + if (randomBoolean() && part.aliases().isEmpty() == false) { + builder.removeAlias(randomFrom(part.aliases().keys().toArray(String.class))); + } else { + builder.putAlias(AliasMetaData.builder(randomAsciiOfLength(10))); + } + break; + case 2: + builder.settings(ImmutableSettings.builder().put(part.settings()).put(IndexMetaData.SETTING_UUID, Strings.randomBase64UUID())); + break; + case 3: + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + break; + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + return builder.build(); + } + }); + } + + private IndexWarmersMetaData randomWarmers() { + if (randomBoolean()) { + return new IndexWarmersMetaData( + new IndexWarmersMetaData.Entry( + randomName("warm"), + new String[]{randomName("type")}, + randomBoolean(), + new BytesArray(randomAsciiOfLength(1000))) + ); + } else { + return new IndexWarmersMetaData(); + } + } + + private MetaData randomTemplates(MetaData metaData) { + return randomParts(metaData, "template", new RandomPart() { + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.templates(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, IndexTemplateMetaData part) { + return builder.put(part); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.removeTemplate(name); + } + + @Override + public IndexTemplateMetaData randomCreate(String name) { + IndexTemplateMetaData.Builder builder = IndexTemplateMetaData.builder(name); + builder.order(randomInt(1000)) + .template(randomName("temp")) + .settings(randomSettings(ImmutableSettings.EMPTY)); + int aliasCount = randomIntBetween(0, 10); + for (int i = 0; i < aliasCount; i++) { + builder.putAlias(randomAlias()); + } + if (randomBoolean()) { + builder.putCustom(IndexWarmersMetaData.TYPE, randomWarmers()); + } + return builder.build(); + } + + @Override + public IndexTemplateMetaData randomChange(IndexTemplateMetaData part) { + IndexTemplateMetaData.Builder builder = new IndexTemplateMetaData.Builder(part); + builder.order(randomInt(1000)); + return builder.build(); + } + }); + } + + private AliasMetaData randomAlias() { + AliasMetaData.Builder builder = newAliasMetaDataBuilder(randomName("alias")); + if (randomBoolean()) { + builder.filter(FilterBuilders.termFilter("test", randomRealisticUnicodeOfCodepointLength(10)).toString()); + } + if (randomBoolean()) { + builder.routing(randomAsciiOfLength(10)); + } + return builder.build(); + } + + private MetaData randomMetaDataCustoms(final MetaData metaData) { + return randomParts(metaData, "custom", new RandomPart() { + + @Override + public ImmutableOpenMap parts(MetaData metaData) { + return metaData.customs(); + } + + @Override + public MetaData.Builder put(MetaData.Builder builder, MetaData.Custom part) { + if (part instanceof SnapshotMetaData) { + return builder.putCustom(SnapshotMetaData.TYPE, part); + } else if (part instanceof RepositoriesMetaData) { + return builder.putCustom(RepositoriesMetaData.TYPE, part); + } else if (part instanceof RestoreMetaData) { + return builder.putCustom(RestoreMetaData.TYPE, part); + } + throw new IllegalArgumentException("Unknown custom part " + part); + } + + @Override + public MetaData.Builder remove(MetaData.Builder builder, String name) { + return builder.removeCustom(name); + } + + @Override + public MetaData.Custom randomCreate(String name) { + switch (randomIntBetween(0, 2)) { + case 0: + return new SnapshotMetaData(new SnapshotMetaData.Entry( + new SnapshotId(randomName("repo"), randomName("snap")), + randomBoolean(), + SnapshotMetaData.State.fromValue((byte) randomIntBetween(0, 6)), + ImmutableList.of(), + Math.abs(randomLong()), + ImmutableMap.of())); + case 1: + return new RepositoriesMetaData(); + case 2: + return new RestoreMetaData(new RestoreMetaData.Entry( + new SnapshotId(randomName("repo"), randomName("snap")), + RestoreMetaData.State.fromValue((byte) randomIntBetween(0, 3)), + ImmutableList.of(), + ImmutableMap.of())); + default: + throw new IllegalArgumentException("Shouldn't be here"); + } + } + + @Override + public MetaData.Custom randomChange(MetaData.Custom part) { + return part; + } + }); + } + + private String randomName(String prefix) { + return prefix + Strings.randomBase64UUID(getRandom()); + } +} \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index cbbff463f20..83a27850591 100644 --- a/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -81,7 +81,7 @@ public class ClusterSerializationTests extends ElasticsearchAllocationTestCase { RoutingTable source = strategy.reroute(clusterState).routingTable(); BytesStreamOutput outStream = new BytesStreamOutput(); - RoutingTable.Builder.writeTo(source, outStream); + source.writeTo(outStream); BytesStreamInput inStream = new BytesStreamInput(outStream.bytes().toBytes()); RoutingTable target = RoutingTable.Builder.readFrom(inStream); diff --git a/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java new file mode 100644 index 00000000000..d87d900a0e8 --- /dev/null +++ b/src/test/java/org/elasticsearch/cluster/serialization/DiffableTests.java @@ -0,0 +1,127 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.serialization; + +import com.google.common.collect.ImmutableMap; +import org.elasticsearch.cluster.Diff; +import org.elasticsearch.cluster.DiffableUtils; +import org.elasticsearch.cluster.DiffableUtils.KeyedReader; +import org.elasticsearch.cluster.AbstractDiffable; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.io.stream.*; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.junit.Test; + +import java.io.IOException; +import java.util.Map; + +import static com.google.common.collect.Maps.newHashMap; +import static org.hamcrest.CoreMatchers.equalTo; + +public class DiffableTests extends ElasticsearchTestCase { + + @Test + public void testImmutableMapDiff() throws IOException { + ImmutableMap.Builder builder = ImmutableMap.builder(); + builder.put("foo", new TestDiffable("1")); + builder.put("bar", new TestDiffable("2")); + builder.put("baz", new TestDiffable("3")); + ImmutableMap before = builder.build(); + Map map = newHashMap(); + map.putAll(before); + map.remove("bar"); + map.put("baz", new TestDiffable("4")); + map.put("new", new TestDiffable("5")); + ImmutableMap after = ImmutableMap.copyOf(map); + Diff diff = DiffableUtils.diff(before, after); + BytesStreamOutput out = new BytesStreamOutput(); + diff.writeTo(out); + BytesStreamInput in = new BytesStreamInput(out.bytes()); + ImmutableMap serialized = DiffableUtils.readImmutableMapDiff(in, TestDiffable.PROTO).apply(before); + assertThat(serialized.size(), equalTo(3)); + assertThat(serialized.get("foo").value(), equalTo("1")); + assertThat(serialized.get("baz").value(), equalTo("4")); + assertThat(serialized.get("new").value(), equalTo("5")); + } + + @Test + public void testImmutableOpenMapDiff() throws IOException { + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); + builder.put("foo", new TestDiffable("1")); + builder.put("bar", new TestDiffable("2")); + builder.put("baz", new TestDiffable("3")); + ImmutableOpenMap before = builder.build(); + builder = ImmutableOpenMap.builder(before); + builder.remove("bar"); + builder.put("baz", new TestDiffable("4")); + builder.put("new", new TestDiffable("5")); + ImmutableOpenMap after = builder.build(); + Diff diff = DiffableUtils.diff(before, after); + BytesStreamOutput out = new BytesStreamOutput(); + diff.writeTo(out); + BytesStreamInput in = new BytesStreamInput(out.bytes()); + ImmutableOpenMap serialized = DiffableUtils.readImmutableOpenMapDiff(in, new KeyedReader() { + @Override + public TestDiffable readFrom(StreamInput in, String key) throws IOException { + return new TestDiffable(in.readString()); + } + + @Override + public Diff readDiffFrom(StreamInput in, String key) throws IOException { + return AbstractDiffable.readDiffFrom(new StreamableReader() { + @Override + public TestDiffable readFrom(StreamInput in) throws IOException { + return new TestDiffable(in.readString()); + } + }, in); + } + }).apply(before); + assertThat(serialized.size(), equalTo(3)); + assertThat(serialized.get("foo").value(), equalTo("1")); + assertThat(serialized.get("baz").value(), equalTo("4")); + assertThat(serialized.get("new").value(), equalTo("5")); + + } + public static class TestDiffable extends AbstractDiffable { + + public static final TestDiffable PROTO = new TestDiffable(""); + + private final String value; + + public TestDiffable(String value) { + this.value = value; + } + + public String value() { + return value; + } + + @Override + public TestDiffable readFrom(StreamInput in) throws IOException { + return new TestDiffable(in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + } + } + +} diff --git a/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java b/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java index f265869ec75..f1e7a249c59 100644 --- a/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/ZenUnicastDiscoveryTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope; import org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Before; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java b/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java index 58e177b1115..228faa8cf4d 100644 --- a/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java +++ b/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryTests.java @@ -32,9 +32,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Priority; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressorFactory; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; @@ -196,12 +193,7 @@ public class ZenDiscoveryTests extends ElasticsearchIntegrationTest { .put(new DiscoveryNode("abc", new LocalTransportAddress("abc"), Version.CURRENT)).masterNodeId("abc"); ClusterState.Builder builder = ClusterState.builder(state); builder.nodes(nodes); - BytesStreamOutput bStream = new BytesStreamOutput(); - StreamOutput stream = CompressorFactory.defaultCompressor().streamOutput(bStream); - stream.setVersion(node.version()); - ClusterState.Builder.writeTo(builder.build(), stream); - stream.close(); - BytesReference bytes = bStream.bytes(); + BytesReference bytes = PublishClusterStateAction.serializeFullClusterState(builder.build(), node.version()); final CountDownLatch latch = new CountDownLatch(1); final AtomicReference reference = new AtomicReference<>(); diff --git a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java index c5adf8cb50e..c97fa5b789d 100644 --- a/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java +++ b/src/test/java/org/elasticsearch/index/mapper/timestamp/TimestampMappingTests.java @@ -443,11 +443,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } @@ -460,11 +460,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } @@ -477,11 +477,11 @@ public class TimestampMappingTests extends ElasticsearchSingleNodeTest { new MappingMetaData.Id(null), new MappingMetaData.Routing(false, null), timestamp, false); BytesStreamOutput out = new BytesStreamOutput(); - MappingMetaData.writeTo(expected, out); + expected.writeTo(out); out.close(); BytesReference bytes = out.bytes(); - MappingMetaData metaData = MappingMetaData.readFrom(new BytesStreamInput(bytes)); + MappingMetaData metaData = MappingMetaData.PROTO.readFrom(new BytesStreamInput(bytes)); assertThat(metaData, is(expected)); } diff --git a/src/test/java/org/elasticsearch/index/query/TestQueryParsingException.java b/src/test/java/org/elasticsearch/index/query/TestQueryParsingException.java new file mode 100644 index 00000000000..951b31e59a6 --- /dev/null +++ b/src/test/java/org/elasticsearch/index/query/TestQueryParsingException.java @@ -0,0 +1,37 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.query; + +import org.elasticsearch.index.Index; + +/** + * Class used to avoid dragging QueryContext into unit testing framework for + * basic exception handling + */ +public class TestQueryParsingException extends QueryParsingException { + + public TestQueryParsingException(Index index, int line, int col, String msg, Throwable cause) { + super(index, line, col, msg, cause); + } + + public TestQueryParsingException(Index index, String msg, Throwable cause) { + super(index, UNKNOWN_POSITION, UNKNOWN_POSITION, msg, cause); + } +} diff --git a/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java b/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java index e1efe59776d..9e05d915803 100644 --- a/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java +++ b/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationComman import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.DiscoveryService; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; @@ -41,10 +42,12 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.disruption.SlowClusterStateProcessing; import org.junit.Test; +import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -58,7 +61,18 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest { @Override protected Settings nodeSettings(int nodeOrdinal) { // simplify this and only use a single data path - return ImmutableSettings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put("path.data", "").build(); + return ImmutableSettings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put("path.data", "") + // by default this value is 1 sec in tests (30 sec in practice) but we adding disruption here + // which is between 1 and 2 sec can cause each of the shard deletion requests to timeout. + // to prevent this we are setting the timeout here to something highish ie. the default in practice + .put(IndicesStore.INDICES_STORE_DELETE_SHARD_TIMEOUT, new TimeValue(30, TimeUnit.SECONDS)) + .build(); + } + + @Override + protected void ensureClusterStateConsistency() throws IOException { + // testShardActiveElseWhere might change the state of a non-master node + // so we cannot check state consistency of this cluster } @Test @@ -97,9 +111,8 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest { assertThat(Files.exists(indexDirectory(node_3, "test")), equalTo(false)); logger.info("--> move shard from node_1 to node_3, and wait for relocation to finish"); - SlowClusterStateProcessing disruption = null; - if (randomBoolean()) { - disruption = new SlowClusterStateProcessing(node_3, getRandom(), 0, 0, 1000, 2000); + if (randomBoolean()) { // sometimes add cluster-state delay to trigger observers in IndicesStore.ShardActiveRequestHandler + final SlowClusterStateProcessing disruption = new SlowClusterStateProcessing(node_3, getRandom(), 0, 0, 1000, 2000); internalCluster().setDisruptionScheme(disruption); disruption.startDisrupting(); } @@ -116,6 +129,7 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest { assertThat(Files.exists(indexDirectory(node_2, "test")), equalTo(true)); assertThat(Files.exists(shardDirectory(node_3, "test", 0)), equalTo(true)); assertThat(Files.exists(indexDirectory(node_3, "test")), equalTo(true)); + } @Test diff --git a/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java b/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java index ce96576ce15..bd664694c9f 100644 --- a/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java +++ b/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateTests.java @@ -284,6 +284,7 @@ public class SimpleIndexTemplateTests extends ElasticsearchIntegrationTest { } @Test + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/8802") public void testBrokenMapping() throws Exception { // clean all templates setup by the framework. client().admin().indices().prepareDeleteTemplate("*").get(); diff --git a/src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java b/src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java index 263af854883..4540cc75a06 100644 --- a/src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java +++ b/src/test/java/org/elasticsearch/percolator/PercolatorFacetsAndAggregationsTests.java @@ -20,11 +20,16 @@ package org.elasticsearch.percolator; import org.elasticsearch.action.percolate.PercolateRequestBuilder; import org.elasticsearch.action.percolate.PercolateResponse; +import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.bucket.terms.Terms; +import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order; +import org.elasticsearch.search.aggregations.reducers.ReducerBuilders; +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBucketMetricValue; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.junit.Test; @@ -40,6 +45,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertMatc import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; /** * @@ -66,20 +72,18 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati expectedCount[i % numUniqueQueries]++; QueryBuilder queryBuilder = matchQuery("field1", value); client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i)) - .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()) - .execute().actionGet(); + .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute() + .actionGet(); } client().admin().indices().prepareRefresh("test").execute().actionGet(); for (int i = 0; i < numQueries; i++) { String value = values[i % numUniqueQueries]; - PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate() - .setIndices("test").setDocumentType("type") + PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type") .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject())); SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values()); - percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2") - .collectMode(aggCollectionMode )); + percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode)); if (randomBoolean()) { percolateRequestBuilder.setPercolateQuery(matchAllQuery()); @@ -111,16 +115,153 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati } } + @Test + // Just test the integration with facets and aggregations, not the facet and aggregation functionality! + public void testAggregationsAndReducers() throws Exception { + assertAcked(prepareCreate("test").addMapping("type", "field1", "type=string", "field2", "type=string")); + ensureGreen(); + + int numQueries = scaledRandomIntBetween(250, 500); + int numUniqueQueries = between(1, numQueries / 2); + String[] values = new String[numUniqueQueries]; + for (int i = 0; i < values.length; i++) { + values[i] = "value" + i; + } + int[] expectedCount = new int[numUniqueQueries]; + + logger.info("--> registering {} queries", numQueries); + for (int i = 0; i < numQueries; i++) { + String value = values[i % numUniqueQueries]; + expectedCount[i % numUniqueQueries]++; + QueryBuilder queryBuilder = matchQuery("field1", value); + client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i)) + .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute() + .actionGet(); + } + client().admin().indices().prepareRefresh("test").execute().actionGet(); + + for (int i = 0; i < numQueries; i++) { + String value = values[i % numUniqueQueries]; + PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type") + .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject())); + + SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values()); + percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode)); + + if (randomBoolean()) { + percolateRequestBuilder.setPercolateQuery(matchAllQuery()); + } + if (randomBoolean()) { + percolateRequestBuilder.setScore(true); + } else { + percolateRequestBuilder.setSortByScore(true).setSize(numQueries); + } + + boolean countOnly = randomBoolean(); + if (countOnly) { + percolateRequestBuilder.setOnlyCount(countOnly); + } + + percolateRequestBuilder.addAggregation(ReducerBuilders.maxBucket("max_a").setBucketsPaths("a>_count")); + + PercolateResponse response = percolateRequestBuilder.execute().actionGet(); + assertMatchCount(response, expectedCount[i % numUniqueQueries]); + if (!countOnly) { + assertThat(response.getMatches(), arrayWithSize(expectedCount[i % numUniqueQueries])); + } + + Aggregations aggregations = response.getAggregations(); + assertThat(aggregations.asList().size(), equalTo(2)); + Terms terms = aggregations.get("a"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("a")); + List buckets = new ArrayList<>(terms.getBuckets()); + assertThat(buckets.size(), equalTo(1)); + assertThat(buckets.get(0).getKeyAsString(), equalTo("b")); + assertThat(buckets.get(0).getDocCount(), equalTo((long) expectedCount[i % values.length])); + + InternalBucketMetricValue maxA = aggregations.get("max_a"); + assertThat(maxA, notNullValue()); + assertThat(maxA.getName(), equalTo("max_a")); + assertThat(maxA.value(), equalTo((double) expectedCount[i % values.length])); + assertThat(maxA.keys(), equalTo(new String[] { "b" })); + } + } + @Test public void testSignificantAggs() throws Exception { client().admin().indices().prepareCreate("test").execute().actionGet(); ensureGreen(); - PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate() - .setIndices("test").setDocumentType("type") + PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type") .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value").endObject())) .addAggregation(AggregationBuilders.significantTerms("a").field("field2")); PercolateResponse response = percolateRequestBuilder.get(); assertNoFailures(response); } + @Test + public void testSingleShardAggregations() throws Exception { + assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder().put(indexSettings()).put("SETTING_NUMBER_OF_SHARDS", 1)) + .addMapping("type", "field1", "type=string", "field2", "type=string")); + ensureGreen(); + + int numQueries = scaledRandomIntBetween(250, 500); + + logger.info("--> registering {} queries", numQueries); + for (int i = 0; i < numQueries; i++) { + String value = "value0"; + QueryBuilder queryBuilder = matchQuery("field1", value); + client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i)) + .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", i % 3 == 0 ? "b" : "a").endObject()) + .execute() + .actionGet(); + } + client().admin().indices().prepareRefresh("test").execute().actionGet(); + + for (int i = 0; i < numQueries; i++) { + String value = "value0"; + PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type") + .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject())); + + SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values()); + percolateRequestBuilder.addAggregation(AggregationBuilders.terms("terms").field("field2").collectMode(aggCollectionMode) + .order(Order.term(true)).shardSize(2).size(1)); + + if (randomBoolean()) { + percolateRequestBuilder.setPercolateQuery(matchAllQuery()); + } + if (randomBoolean()) { + percolateRequestBuilder.setScore(true); + } else { + percolateRequestBuilder.setSortByScore(true).setSize(numQueries); + } + + boolean countOnly = randomBoolean(); + if (countOnly) { + percolateRequestBuilder.setOnlyCount(countOnly); + } + + percolateRequestBuilder.addAggregation(ReducerBuilders.maxBucket("max_terms").setBucketsPaths("terms>_count")); + + PercolateResponse response = percolateRequestBuilder.execute().actionGet(); + assertMatchCount(response, numQueries); + if (!countOnly) { + assertThat(response.getMatches(), arrayWithSize(numQueries)); + } + + Aggregations aggregations = response.getAggregations(); + assertThat(aggregations.asList().size(), equalTo(2)); + Terms terms = aggregations.get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List buckets = new ArrayList<>(terms.getBuckets()); + assertThat(buckets.size(), equalTo(1)); + assertThat(buckets.get(0).getKeyAsString(), equalTo("a")); + + InternalBucketMetricValue maxA = aggregations.get("max_terms"); + assertThat(maxA, notNullValue()); + assertThat(maxA.getName(), equalTo("max_terms")); + assertThat(maxA.keys(), equalTo(new String[] { "a" })); + } + } } diff --git a/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java b/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java index e110e6ca70d..579408366e9 100644 --- a/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java +++ b/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java @@ -23,7 +23,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.index.Index; -import org.elasticsearch.index.query.QueryParsingException; +import org.elasticsearch.index.query.TestQueryParsingException; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.rest.FakeRestRequest; @@ -141,12 +141,14 @@ public class BytesRestResponseTests extends ElasticsearchTestCase { public void testConvert() throws IOException { RestRequest request = new FakeRestRequest(); RestChannel channel = new DetailedExceptionRestChannel(request); - ShardSearchFailure failure = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 1)); - ShardSearchFailure failure1 = new ShardSearchFailure(new QueryParsingException(new Index("foo"), "foobar"), new SearchShardTarget("node_1", "foo", 2)); + ShardSearchFailure failure = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 1)); + ShardSearchFailure failure1 = new ShardSearchFailure(new TestQueryParsingException(new Index("foo"), "foobar", null), + new SearchShardTarget("node_1", "foo", 2)); SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[] {failure, failure1}); BytesRestResponse response = new BytesRestResponse(channel, new RemoteTransportException("foo", ex)); String text = response.content().toUtf8(); - String expected = "{\"error\":{\"root_cause\":[{\"type\":\"query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}],\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}}]},\"status\":400}"; + String expected = "{\"error\":{\"root_cause\":[{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}],\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"test_query_parsing_exception\",\"reason\":\"foobar\",\"index\":\"foo\"}}]},\"status\":400}"; assertEquals(expected.trim(), text.trim()); } diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTest.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTest.java index ebf55ac66c4..f585824a267 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTest.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregatorTest.java @@ -123,7 +123,7 @@ public class NestedAggregatorTest extends ElasticsearchSingleNodeTest { AggregationContext context = new AggregationContext(searchContext); AggregatorFactories.Builder builder = AggregatorFactories.builder(); - builder.add(new NestedAggregator.Factory("test", "nested_field", QueryCachingPolicy.ALWAYS_CACHE)); + builder.addAggregator(new NestedAggregator.Factory("test", "nested_field", QueryCachingPolicy.ALWAYS_CACHE)); AggregatorFactories factories = builder.build(); searchContext.aggregations(new SearchContextAggregations(factories)); Aggregator[] aggs = factories.createTopLevelAggregators(context); diff --git a/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java b/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java index 16ce8a964dc..eeff6304dd3 100644 --- a/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java +++ b/src/test/java/org/elasticsearch/search/aggregations/bucket/significant/SignificanceHeuristicTests.java @@ -29,7 +29,18 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.significant.heuristics.*; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.ChiSquare; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.GND; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.JLHScore; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.MutualInformation; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.PercentageScore; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.ScriptHeuristic; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicBuilder; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParserMapper; +import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicStreams; +import org.elasticsearch.search.aggregations.reducers.Reducer; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.TestSearchContext; @@ -39,6 +50,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -101,13 +113,15 @@ public class SignificanceHeuristicTests extends ElasticsearchTestCase { if (randomBoolean()) { BytesRef term = new BytesRef("123.0"); buckets.add(new SignificantLongTerms.Bucket(1, 2, 3, 4, 123, InternalAggregations.EMPTY, null)); - sTerms[0] = new SignificantLongTerms(10, 20, "some_name", null, 1, 1, heuristic, buckets, null); + sTerms[0] = new SignificantLongTerms(10, 20, "some_name", null, 1, 1, heuristic, buckets, + (List) Collections.EMPTY_LIST, null); sTerms[1] = new SignificantLongTerms(); } else { BytesRef term = new BytesRef("someterm"); buckets.add(new SignificantStringTerms.Bucket(term, 1, 2, 3, 4, InternalAggregations.EMPTY)); - sTerms[0] = new SignificantStringTerms(10, 20, "some_name", 1, 1, heuristic, buckets, null); + sTerms[0] = new SignificantStringTerms(10, 20, "some_name", 1, 1, heuristic, buckets, (List) Collections.EMPTY_LIST, + null); sTerms[1] = new SignificantStringTerms(); } return sTerms; diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/DateDerivativeTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/DateDerivativeTests.java new file mode 100644 index 00000000000..b1ac6756f1e --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/DateDerivativeTests.java @@ -0,0 +1,337 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.index.mapper.core.DateFieldMapper; +import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.metrics.sum.Sum; +import org.elasticsearch.search.aggregations.support.AggregationPath; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.junit.After; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; +import static org.elasticsearch.search.aggregations.reducers.ReducerBuilders.derivative; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.core.IsNull.notNullValue; +import static org.hamcrest.core.IsNull.nullValue; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class DateDerivativeTests extends ElasticsearchIntegrationTest { + + private DateTime date(int month, int day) { + return new DateTime(2012, month, day, 0, 0, DateTimeZone.UTC); + } + + private DateTime date(String date) { + return DateFieldMapper.Defaults.DATE_TIME_FORMATTER.parser().parseDateTime(date); + } + + private static String format(DateTime date, String pattern) { + return DateTimeFormat.forPattern(pattern).print(date); + } + + private IndexRequestBuilder indexDoc(String idx, DateTime date, int value) throws Exception { + return client().prepareIndex(idx, "type").setSource( + jsonBuilder().startObject().field("date", date).field("value", value).startArray("dates").value(date) + .value(date.plusMonths(1).plusDays(1)).endArray().endObject()); + } + + private IndexRequestBuilder indexDoc(int month, int day, int value) throws Exception { + return client().prepareIndex("idx", "type").setSource( + jsonBuilder().startObject().field("value", value).field("date", date(month, day)).startArray("dates") + .value(date(month, day)).value(date(month + 1, day + 1)).endArray().endObject()); + } + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx"); + createIndex("idx_unmapped"); + // TODO: would be nice to have more random data here + prepareCreate("empty_bucket_idx").addMapping("type", "value", "type=integer").execute().actionGet(); + List builders = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + builders.add(client().prepareIndex("empty_bucket_idx", "type", "" + i).setSource( + jsonBuilder().startObject().field("value", i * 2).endObject())); + } + builders.addAll(Arrays.asList(indexDoc(1, 2, 1), // date: Jan 2, dates: Jan 2, Feb 3 + indexDoc(2, 2, 2), // date: Feb 2, dates: Feb 2, Mar 3 + indexDoc(2, 15, 3), // date: Feb 15, dates: Feb 15, Mar 16 + indexDoc(3, 2, 4), // date: Mar 2, dates: Mar 2, Apr 3 + indexDoc(3, 15, 5), // date: Mar 15, dates: Mar 15, Apr 16 + indexDoc(3, 23, 6))); // date: Mar 23, dates: Mar 23, Apr 24 + indexRandom(true, builders); + ensureSearchable(); + } + + @After + public void afterEachTest() throws IOException { + internalCluster().wipeIndices("idx2"); + } + + @Test + public void singleValuedField() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + dateHistogram("histo").field("date").interval(DateHistogramInterval.MONTH).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(3)); + + DateTime key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC); + Histogram.Bucket bucket = buckets.get(0); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(1l)); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, nullValue()); + + key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(1); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(2l)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(1d)); + + key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(2); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(3l)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(1d)); + } + + @Test + public void singleValuedField_WithSubAggregation() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + dateHistogram("histo").field("date").interval(DateHistogramInterval.MONTH).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("sum")).subAggregation(sum("sum").field("value"))) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat(buckets.size(), equalTo(3)); + Object[] propertiesKeys = (Object[]) histo.getProperty("_key"); + Object[] propertiesDocCounts = (Object[]) histo.getProperty("_count"); + Object[] propertiesCounts = (Object[]) histo.getProperty("sum.value"); + + DateTime key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC); + Histogram.Bucket bucket = buckets.get(0); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(1l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + Sum sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + assertThat(sum.getValue(), equalTo(1.0)); + SimpleValue deriv = bucket.getAggregations().get("deriv"); + assertThat(deriv, nullValue()); + assertThat((DateTime) propertiesKeys[0], equalTo(key)); + assertThat((long) propertiesDocCounts[0], equalTo(1l)); + assertThat((double) propertiesCounts[0], equalTo(1.0)); + + key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(1); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(2l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + assertThat(sum.getValue(), equalTo(5.0)); + deriv = bucket.getAggregations().get("deriv"); + assertThat(deriv, notNullValue()); + assertThat(deriv.value(), equalTo(4.0)); + assertThat((double) bucket.getProperty("histo", AggregationPath.parse("deriv.value").getPathElementsAsStringList()), equalTo(4.0)); + assertThat((DateTime) propertiesKeys[1], equalTo(key)); + assertThat((long) propertiesDocCounts[1], equalTo(2l)); + assertThat((double) propertiesCounts[1], equalTo(5.0)); + + key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(2); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(3l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + assertThat(sum.getValue(), equalTo(15.0)); + deriv = bucket.getAggregations().get("deriv"); + assertThat(deriv, notNullValue()); + assertThat(deriv.value(), equalTo(10.0)); + assertThat((double) bucket.getProperty("histo", AggregationPath.parse("deriv.value").getPathElementsAsStringList()), equalTo(10.0)); + assertThat((DateTime) propertiesKeys[2], equalTo(key)); + assertThat((long) propertiesDocCounts[2], equalTo(3l)); + assertThat((double) propertiesCounts[2], equalTo(15.0)); + } + + @Test + public void multiValuedField() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + dateHistogram("histo").field("dates").interval(DateHistogramInterval.MONTH).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(4)); + + DateTime key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC); + Histogram.Bucket bucket = buckets.get(0); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(1l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(true)); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, nullValue()); + + key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(1); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(3l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(2.0)); + + key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(2); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(5l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(2.0)); + + key = new DateTime(2012, 4, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(3); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(3l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(-2.0)); + } + + @Test + public void unmapped() throws Exception { + SearchResponse response = client() + .prepareSearch("idx_unmapped") + .addAggregation( + dateHistogram("histo").field("date").interval(DateHistogramInterval.MONTH).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + assertThat(deriv.getBuckets().size(), equalTo(0)); + } + + @Test + public void partiallyUnmapped() throws Exception { + SearchResponse response = client() + .prepareSearch("idx", "idx_unmapped") + .addAggregation( + dateHistogram("histo").field("date").interval(DateHistogramInterval.MONTH).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(3)); + + DateTime key = new DateTime(2012, 1, 1, 0, 0, DateTimeZone.UTC); + Histogram.Bucket bucket = buckets.get(0); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(1l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(true)); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, nullValue()); + + key = new DateTime(2012, 2, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(1); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(2l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(1.0)); + + key = new DateTime(2012, 3, 1, 0, 0, DateTimeZone.UTC); + bucket = buckets.get(2); + assertThat(bucket, notNullValue()); + assertThat((DateTime) bucket.getKey(), equalTo(key)); + assertThat(bucket.getDocCount(), equalTo(3l)); + assertThat(bucket.getAggregations().asList().isEmpty(), is(false)); + docCountDeriv = bucket.getAggregations().get("deriv"); + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo(1.0)); + } + +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/DerivativeTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/DerivativeTests.java new file mode 100644 index 00000000000..0974d297d46 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/DerivativeTests.java @@ -0,0 +1,560 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.metrics.stats.Stats; +import org.elasticsearch.search.aggregations.metrics.sum.Sum; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.support.AggregationPath; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.hamcrest.Matchers; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.stats; +import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; +import static org.elasticsearch.search.aggregations.reducers.ReducerBuilders.derivative; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.core.IsNull.notNullValue; +import static org.hamcrest.core.IsNull.nullValue; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class DerivativeTests extends ElasticsearchIntegrationTest { + + private static final String SINGLE_VALUED_FIELD_NAME = "l_value"; + + private static int interval; + private static int numValueBuckets; + private static int numFirstDerivValueBuckets; + private static int numSecondDerivValueBuckets; + private static long[] valueCounts; + private static long[] firstDerivValueCounts; + private static long[] secondDerivValueCounts; + + private static Long[] valueCounts_empty; + private static long numDocsEmptyIdx; + private static Double[] firstDerivValueCounts_empty; + + // expected bucket values for random setup with gaps + private static int numBuckets_empty_rnd; + private static Long[] valueCounts_empty_rnd; + private static Double[] firstDerivValueCounts_empty_rnd; + private static long numDocsEmptyIdx_rnd; + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx"); + createIndex("idx_unmapped"); + + interval = 5; + numValueBuckets = randomIntBetween(6, 80); + + valueCounts = new long[numValueBuckets]; + for (int i = 0; i < numValueBuckets; i++) { + valueCounts[i] = randomIntBetween(1, 20); + } + + numFirstDerivValueBuckets = numValueBuckets - 1; + firstDerivValueCounts = new long[numFirstDerivValueBuckets]; + Long lastValueCount = null; + for (int i = 0; i < numValueBuckets; i++) { + long thisValue = valueCounts[i]; + if (lastValueCount != null) { + long diff = thisValue - lastValueCount; + firstDerivValueCounts[i - 1] = diff; + } + lastValueCount = thisValue; + } + + numSecondDerivValueBuckets = numFirstDerivValueBuckets - 1; + secondDerivValueCounts = new long[numSecondDerivValueBuckets]; + Long lastFirstDerivativeValueCount = null; + for (int i = 0; i < numFirstDerivValueBuckets; i++) { + long thisFirstDerivativeValue = firstDerivValueCounts[i]; + if (lastFirstDerivativeValueCount != null) { + long diff = thisFirstDerivativeValue - lastFirstDerivativeValueCount; + secondDerivValueCounts[i - 1] = diff; + } + lastFirstDerivativeValueCount = thisFirstDerivativeValue; + } + + List builders = new ArrayList<>(); + for (int i = 0; i < numValueBuckets; i++) { + for (int docs = 0; docs < valueCounts[i]; docs++) { + builders.add(client().prepareIndex("idx", "type").setSource(newDocBuilder(i * interval))); + } + } + + // setup for index with empty buckets + valueCounts_empty = new Long[] { 1l, 1l, 2l, 0l, 2l, 2l, 0l, 0l, 0l, 3l, 2l, 1l }; + firstDerivValueCounts_empty = new Double[] { null, 0d, 1d, -2d, 2d, 0d, -2d, 0d, 0d, 3d, -1d, -1d }; + + assertAcked(prepareCreate("empty_bucket_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer")); + for (int i = 0; i < valueCounts_empty.length; i++) { + for (int docs = 0; docs < valueCounts_empty[i]; docs++) { + builders.add(client().prepareIndex("empty_bucket_idx", "type").setSource(newDocBuilder(i))); + numDocsEmptyIdx++; + } + } + + // randomized setup for index with empty buckets + numBuckets_empty_rnd = randomIntBetween(20, 100); + valueCounts_empty_rnd = new Long[numBuckets_empty_rnd]; + firstDerivValueCounts_empty_rnd = new Double[numBuckets_empty_rnd]; + firstDerivValueCounts_empty_rnd[0] = null; + + assertAcked(prepareCreate("empty_bucket_idx_rnd").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer")); + for (int i = 0; i < numBuckets_empty_rnd; i++) { + valueCounts_empty_rnd[i] = (long) randomIntBetween(1, 10); + // make approximately half of the buckets empty + if (randomBoolean()) + valueCounts_empty_rnd[i] = 0l; + for (int docs = 0; docs < valueCounts_empty_rnd[i]; docs++) { + builders.add(client().prepareIndex("empty_bucket_idx_rnd", "type").setSource(newDocBuilder(i))); + numDocsEmptyIdx_rnd++; + } + if (i > 0) { + firstDerivValueCounts_empty_rnd[i] = (double) valueCounts_empty_rnd[i] - valueCounts_empty_rnd[i - 1]; + } + } + + indexRandom(true, builders); + ensureSearchable(); + } + + private XContentBuilder newDocBuilder(int singleValueFieldValue) throws IOException { + return jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, singleValueFieldValue).endObject(); + } + + /** + * test first and second derivative on the sing + */ + @Test + public void docCountDerivative() { + + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count")) + .subAggregation(derivative("2nd_deriv").setBucketsPaths("deriv"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(numValueBuckets)); + + for (int i = 0; i < numValueBuckets; ++i) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i * interval, valueCounts[i]); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + if (i > 0) { + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo((double) firstDerivValueCounts[i - 1])); + } else { + assertThat(docCountDeriv, nullValue()); + } + SimpleValue docCount2ndDeriv = bucket.getAggregations().get("2nd_deriv"); + if (i > 1) { + assertThat(docCount2ndDeriv, notNullValue()); + assertThat(docCount2ndDeriv.value(), equalTo((double) secondDerivValueCounts[i - 2])); + } else { + assertThat(docCount2ndDeriv, nullValue()); + } + } + } + + @Test + public void singleValueAggDerivative() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) + .subAggregation(derivative("deriv").setBucketsPaths("sum"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + assertThat(deriv.getBuckets().size(), equalTo(numValueBuckets)); + Object[] propertiesKeys = (Object[]) deriv.getProperty("_key"); + Object[] propertiesDocCounts = (Object[]) deriv.getProperty("_count"); + Object[] propertiesSumCounts = (Object[]) deriv.getProperty("sum.value"); + + List buckets = new ArrayList(deriv.getBuckets()); + Long expectedSumPreviousBucket = Long.MIN_VALUE; // start value, gets + // overwritten + for (int i = 0; i < numValueBuckets; ++i) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i * interval, valueCounts[i]); + Sum sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + long expectedSum = valueCounts[i] * (i * interval); + assertThat(sum.getValue(), equalTo((double) expectedSum)); + SimpleValue sumDeriv = bucket.getAggregations().get("deriv"); + if (i > 0) { + assertThat(sumDeriv, notNullValue()); + long sumDerivValue = expectedSum - expectedSumPreviousBucket; + assertThat(sumDeriv.value(), equalTo((double) sumDerivValue)); + assertThat((double) bucket.getProperty("histo", AggregationPath.parse("deriv.value").getPathElementsAsStringList()), + equalTo((double) sumDerivValue)); + } else { + assertThat(sumDeriv, nullValue()); + } + expectedSumPreviousBucket = expectedSum; + assertThat((long) propertiesKeys[i], equalTo((long) i * interval)); + assertThat((long) propertiesDocCounts[i], equalTo(valueCounts[i])); + assertThat((double) propertiesSumCounts[i], equalTo((double) expectedSum)); + } + } + + @Test + public void multiValueAggDerivative() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .subAggregation(stats("stats").field(SINGLE_VALUED_FIELD_NAME)) + .subAggregation(derivative("deriv").setBucketsPaths("stats.sum"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + assertThat(deriv.getBuckets().size(), equalTo(numValueBuckets)); + Object[] propertiesKeys = (Object[]) deriv.getProperty("_key"); + Object[] propertiesDocCounts = (Object[]) deriv.getProperty("_count"); + Object[] propertiesSumCounts = (Object[]) deriv.getProperty("stats.sum"); + + List buckets = new ArrayList(deriv.getBuckets()); + Long expectedSumPreviousBucket = Long.MIN_VALUE; // start value, gets + // overwritten + for (int i = 0; i < numValueBuckets; ++i) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i * interval, valueCounts[i]); + Stats stats = bucket.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + long expectedSum = valueCounts[i] * (i * interval); + assertThat(stats.getSum(), equalTo((double) expectedSum)); + SimpleValue sumDeriv = bucket.getAggregations().get("deriv"); + if (i > 0) { + assertThat(sumDeriv, notNullValue()); + long sumDerivValue = expectedSum - expectedSumPreviousBucket; + assertThat(sumDeriv.value(), equalTo((double) sumDerivValue)); + assertThat((double) bucket.getProperty("histo", AggregationPath.parse("deriv.value").getPathElementsAsStringList()), + equalTo((double) sumDerivValue)); + } else { + assertThat(sumDeriv, nullValue()); + } + expectedSumPreviousBucket = expectedSum; + assertThat((long) propertiesKeys[i], equalTo((long) i * interval)); + assertThat((long) propertiesDocCounts[i], equalTo(valueCounts[i])); + assertThat((double) propertiesSumCounts[i], equalTo((double) expectedSum)); + } + } + + @Test + public void unmapped() throws Exception { + SearchResponse response = client() + .prepareSearch("idx_unmapped") + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + assertThat(deriv.getBuckets().size(), equalTo(0)); + } + + @Test + public void partiallyUnmapped() throws Exception { + SearchResponse response = client() + .prepareSearch("idx", "idx_unmapped") + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram deriv = response.getAggregations().get("histo"); + assertThat(deriv, notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(deriv.getBuckets().size(), equalTo(numValueBuckets)); + + for (int i = 0; i < numValueBuckets; ++i) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i * interval, valueCounts[i]); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + if (i > 0) { + assertThat(docCountDeriv, notNullValue()); + assertThat(docCountDeriv.value(), equalTo((double) firstDerivValueCounts[i - 1])); + } else { + assertThat(docCountDeriv, nullValue()); + } + } + } + + @Test + public void docCountDerivativeWithGaps() throws Exception { + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count"))).execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(valueCounts_empty.length)); + + for (int i = 0; i < valueCounts_empty.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i, valueCounts_empty[i]); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + if (firstDerivValueCounts_empty[i] == null) { + assertThat(docCountDeriv, nullValue()); + } else { + assertThat(docCountDeriv.value(), equalTo(firstDerivValueCounts_empty[i])); + } + } + } + + @Test + public void docCountDerivativeWithGaps_random() throws Exception { + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx_rnd") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .extendedBounds(0l, (long) numBuckets_empty_rnd - 1) + .subAggregation(derivative("deriv").setBucketsPaths("_count").gapPolicy(randomFrom(GapPolicy.values())))) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx_rnd)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(numBuckets_empty_rnd)); + + for (int i = 0; i < valueCounts_empty_rnd.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i, valueCounts_empty_rnd[i]); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + if (firstDerivValueCounts_empty_rnd[i] == null) { + assertThat(docCountDeriv, nullValue()); + } else { + assertThat(docCountDeriv.value(), equalTo(firstDerivValueCounts_empty_rnd[i])); + } + } + } + + @Test + public void docCountDerivativeWithGaps_insertZeros() throws Exception { + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .subAggregation(derivative("deriv").setBucketsPaths("_count").gapPolicy(GapPolicy.INSERT_ZEROS))).execute() + .actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(valueCounts_empty.length)); + + for (int i = 0; i < valueCounts_empty.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i + ": ", bucket, i, valueCounts_empty[i]); + SimpleValue docCountDeriv = bucket.getAggregations().get("deriv"); + if (firstDerivValueCounts_empty[i] == null) { + assertThat(docCountDeriv, nullValue()); + } else { + assertThat(docCountDeriv.value(), equalTo(firstDerivValueCounts_empty[i])); + } + } + } + + @Test + public void singleValueAggDerivativeWithGaps() throws Exception { + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) + .subAggregation(derivative("deriv").setBucketsPaths("sum"))).execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(valueCounts_empty.length)); + + double lastSumValue = Double.NaN; + for (int i = 0; i < valueCounts_empty.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i, valueCounts_empty[i]); + Sum sum = bucket.getAggregations().get("sum"); + double thisSumValue = sum.value(); + if (bucket.getDocCount() == 0) { + thisSumValue = Double.NaN; + } + SimpleValue sumDeriv = bucket.getAggregations().get("deriv"); + if (i == 0) { + assertThat(sumDeriv, nullValue()); + } else { + double expectedDerivative = thisSumValue - lastSumValue; + if (Double.isNaN(expectedDerivative)) { + assertThat(sumDeriv.value(), equalTo(expectedDerivative)); + } else { + assertThat(sumDeriv.value(), closeTo(expectedDerivative, 0.00001)); + } + } + lastSumValue = thisSumValue; + } + } + + @Test + public void singleValueAggDerivativeWithGaps_insertZeros() throws Exception { + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) + .subAggregation(derivative("deriv").setBucketsPaths("sum").gapPolicy(GapPolicy.INSERT_ZEROS))).execute() + .actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(valueCounts_empty.length)); + + double lastSumValue = Double.NaN; + for (int i = 0; i < valueCounts_empty.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i, valueCounts_empty[i]); + Sum sum = bucket.getAggregations().get("sum"); + double thisSumValue = sum.value(); + if (bucket.getDocCount() == 0) { + thisSumValue = 0; + } + SimpleValue sumDeriv = bucket.getAggregations().get("deriv"); + if (i == 0) { + assertThat(sumDeriv, nullValue()); + } else { + double expectedDerivative = thisSumValue - lastSumValue; + assertThat(sumDeriv.value(), closeTo(expectedDerivative, 0.00001)); + } + lastSumValue = thisSumValue; + } + } + + @Test + public void singleValueAggDerivativeWithGaps_random() throws Exception { + GapPolicy gapPolicy = randomFrom(GapPolicy.values()); + SearchResponse searchResponse = client() + .prepareSearch("empty_bucket_idx_rnd") + .setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(1).minDocCount(0) + .extendedBounds(0l, (long) numBuckets_empty_rnd - 1) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME)) + .subAggregation(derivative("deriv").setBucketsPaths("sum").gapPolicy(gapPolicy))).execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(numDocsEmptyIdx_rnd)); + + InternalHistogram deriv = searchResponse.getAggregations().get("histo"); + assertThat(deriv, Matchers.notNullValue()); + assertThat(deriv.getName(), equalTo("histo")); + List buckets = deriv.getBuckets(); + assertThat(buckets.size(), equalTo(numBuckets_empty_rnd)); + + double lastSumValue = Double.NaN; + for (int i = 0; i < valueCounts_empty_rnd.length; i++) { + Histogram.Bucket bucket = buckets.get(i); + checkBucketKeyAndDocCount("Bucket " + i, bucket, i, valueCounts_empty_rnd[i]); + Sum sum = bucket.getAggregations().get("sum"); + double thisSumValue = sum.value(); + if (bucket.getDocCount() == 0) { + thisSumValue = gapPolicy == GapPolicy.INSERT_ZEROS ? 0 : Double.NaN; + } + SimpleValue sumDeriv = bucket.getAggregations().get("deriv"); + if (i == 0) { + assertThat(sumDeriv, nullValue()); + } else { + double expectedDerivative = thisSumValue - lastSumValue; + if (Double.isNaN(expectedDerivative)) { + assertThat(sumDeriv.value(), equalTo(expectedDerivative)); + } else { + assertThat(sumDeriv.value(), closeTo(expectedDerivative, 0.00001)); + } + } + lastSumValue = thisSumValue; + } + } + + private void checkBucketKeyAndDocCount(final String msg, final Histogram.Bucket bucket, final long expectedKey, + final long expectedDocCount) { + assertThat(msg, bucket, notNullValue()); + assertThat(msg + " key", ((Number) bucket.getKey()).longValue(), equalTo(expectedKey)); + assertThat(msg + " docCount", bucket.getDocCount(), equalTo(expectedDocCount)); + } +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/MaxBucketTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/MaxBucketTests.java new file mode 100644 index 00000000000..84e559e4970 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/MaxBucketTests.java @@ -0,0 +1,433 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram.Bucket; +import org.elasticsearch.search.aggregations.bucket.terms.Terms; +import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order; +import org.elasticsearch.search.aggregations.metrics.sum.Sum; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers.GapPolicy; +import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBucketMetricValue; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; +import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; +import static org.elasticsearch.search.aggregations.reducers.ReducerBuilders.maxBucket; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.core.IsNull.notNullValue; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class MaxBucketTests extends ElasticsearchIntegrationTest { + + private static final String SINGLE_VALUED_FIELD_NAME = "l_value"; + + static int numDocs; + static int interval; + static int minRandomValue; + static int maxRandomValue; + static int numValueBuckets; + static long[] valueCounts; + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx"); + createIndex("idx_unmapped"); + + numDocs = randomIntBetween(6, 20); + interval = randomIntBetween(2, 5); + + minRandomValue = 0; + maxRandomValue = 20; + + numValueBuckets = ((maxRandomValue - minRandomValue) / interval) + 1; + valueCounts = new long[numValueBuckets]; + + List builders = new ArrayList<>(); + + for (int i = 0; i < numDocs; i++) { + int fieldValue = randomIntBetween(minRandomValue, maxRandomValue); + builders.add(client().prepareIndex("idx", "type").setSource( + jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, fieldValue).field("tag", "tag" + (i % interval)) + .endObject())); + final int bucket = (fieldValue / interval); // + (fieldValue < 0 ? -1 : 0) - (minRandomValue / interval - 1); + valueCounts[bucket]++; + } + + assertAcked(prepareCreate("empty_bucket_idx").addMapping("type", SINGLE_VALUED_FIELD_NAME, "type=integer")); + for (int i = 0; i < 2; i++) { + builders.add(client().prepareIndex("empty_bucket_idx", "type", "" + i).setSource( + jsonBuilder().startObject().field(SINGLE_VALUED_FIELD_NAME, i * 2).endObject())); + } + indexRandom(true, builders); + ensureSearchable(); + } + + @Test + public void testDocCount_topLevel() throws Exception { + SearchResponse response = client().prepareSearch("idx") + .addAggregation(histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .extendedBounds((long) minRandomValue, (long) maxRandomValue)) + .addAggregation(maxBucket("max_bucket").setBucketsPaths("histo>_count")).execute().actionGet(); + + assertSearchResponse(response); + + Histogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat(buckets.size(), equalTo(numValueBuckets)); + + List maxKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < numValueBuckets; ++i) { + Histogram.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) i * interval)); + assertThat(bucket.getDocCount(), equalTo(valueCounts[i])); + if (bucket.getDocCount() > maxValue) { + maxValue = bucket.getDocCount(); + maxKeys = new ArrayList<>(); + maxKeys.add(bucket.getKeyAsString()); + } else if (bucket.getDocCount() == maxValue) { + maxKeys.add(bucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = response.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxValue)); + assertThat(maxBucketValue.keys(), equalTo(maxKeys.toArray(new String[maxKeys.size()]))); + } + + @Test + public void testDocCount_asSubAgg() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + terms("terms") + .field("tag") + .order(Order.term(true)) + .subAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .extendedBounds((long) minRandomValue, (long) maxRandomValue)) + .subAggregation(maxBucket("max_bucket").setBucketsPaths("histo>_count"))).execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List termsBuckets = terms.getBuckets(); + assertThat(termsBuckets.size(), equalTo(interval)); + + for (int i = 0; i < interval; ++i) { + Terms.Bucket termsBucket = termsBuckets.get(i); + assertThat(termsBucket, notNullValue()); + assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval))); + + Histogram histo = termsBucket.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + List maxKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + for (int j = 0; j < numValueBuckets; ++j) { + Histogram.Bucket bucket = buckets.get(j); + assertThat(bucket, notNullValue()); + assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval)); + if (bucket.getDocCount() > maxValue) { + maxValue = bucket.getDocCount(); + maxKeys = new ArrayList<>(); + maxKeys.add(bucket.getKeyAsString()); + } else if (bucket.getDocCount() == maxValue) { + maxKeys.add(bucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = termsBucket.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxValue)); + assertThat(maxBucketValue.keys(), equalTo(maxKeys.toArray(new String[maxKeys.size()]))); + } + } + + @Test + public void testMetric_topLevel() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation(terms("terms").field("tag").subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) + .addAggregation(maxBucket("max_bucket").setBucketsPaths("terms>sum")).execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List buckets = terms.getBuckets(); + assertThat(buckets.size(), equalTo(interval)); + + List maxKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < interval; ++i) { + Terms.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + assertThat((String) bucket.getKey(), equalTo("tag" + (i % interval))); + assertThat(bucket.getDocCount(), greaterThan(0l)); + Sum sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + if (sum.value() > maxValue) { + maxValue = sum.value(); + maxKeys = new ArrayList<>(); + maxKeys.add(bucket.getKeyAsString()); + } else if (sum.value() == maxValue) { + maxKeys.add(bucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = response.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxValue)); + assertThat(maxBucketValue.keys(), equalTo(maxKeys.toArray(new String[maxKeys.size()]))); + } + + @Test + public void testMetric_asSubAgg() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + terms("terms") + .field("tag") + .order(Order.term(true)) + .subAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .extendedBounds((long) minRandomValue, (long) maxRandomValue) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) + .subAggregation(maxBucket("max_bucket").setBucketsPaths("histo>sum"))).execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List termsBuckets = terms.getBuckets(); + assertThat(termsBuckets.size(), equalTo(interval)); + + for (int i = 0; i < interval; ++i) { + Terms.Bucket termsBucket = termsBuckets.get(i); + assertThat(termsBucket, notNullValue()); + assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval))); + + Histogram histo = termsBucket.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + List maxKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + for (int j = 0; j < numValueBuckets; ++j) { + Histogram.Bucket bucket = buckets.get(j); + assertThat(bucket, notNullValue()); + assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval)); + if (bucket.getDocCount() != 0) { + Sum sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + if (sum.value() > maxValue) { + maxValue = sum.value(); + maxKeys = new ArrayList<>(); + maxKeys.add(bucket.getKeyAsString()); + } else if (sum.value() == maxValue) { + maxKeys.add(bucket.getKeyAsString()); + } + } + } + + InternalBucketMetricValue maxBucketValue = termsBucket.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxValue)); + assertThat(maxBucketValue.keys(), equalTo(maxKeys.toArray(new String[maxKeys.size()]))); + } + } + + @Test + public void testMetric_asSubAggWithInsertZeros() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + terms("terms") + .field("tag") + .order(Order.term(true)) + .subAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .extendedBounds((long) minRandomValue, (long) maxRandomValue) + .subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) + .subAggregation(maxBucket("max_bucket").setBucketsPaths("histo>sum").gapPolicy(GapPolicy.INSERT_ZEROS))) + .execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List termsBuckets = terms.getBuckets(); + assertThat(termsBuckets.size(), equalTo(interval)); + + for (int i = 0; i < interval; ++i) { + Terms.Bucket termsBucket = termsBuckets.get(i); + assertThat(termsBucket, notNullValue()); + assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval))); + + Histogram histo = termsBucket.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + List maxKeys = new ArrayList<>(); + double maxValue = Double.NEGATIVE_INFINITY; + for (int j = 0; j < numValueBuckets; ++j) { + Histogram.Bucket bucket = buckets.get(j); + assertThat(bucket, notNullValue()); + assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval)); + Sum sum = bucket.getAggregations().get("sum"); + assertThat(sum, notNullValue()); + if (sum.value() > maxValue) { + maxValue = sum.value(); + maxKeys = new ArrayList<>(); + maxKeys.add(bucket.getKeyAsString()); + } else if (sum.value() == maxValue) { + maxKeys.add(bucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = termsBucket.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxValue)); + assertThat(maxBucketValue.keys(), equalTo(maxKeys.toArray(new String[maxKeys.size()]))); + } + } + + @Test + public void testNoBuckets() throws Exception { + SearchResponse response = client().prepareSearch("idx") + .addAggregation(terms("terms").field("tag").exclude("tag.*").subAggregation(sum("sum").field(SINGLE_VALUED_FIELD_NAME))) + .addAggregation(maxBucket("max_bucket").setBucketsPaths("terms>sum")).execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List buckets = terms.getBuckets(); + assertThat(buckets.size(), equalTo(0)); + + InternalBucketMetricValue maxBucketValue = response.getAggregations().get("max_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_bucket")); + assertThat(maxBucketValue.value(), equalTo(Double.NEGATIVE_INFINITY)); + assertThat(maxBucketValue.keys(), equalTo(new String[0])); + } + + @Test + public void testNested() throws Exception { + SearchResponse response = client() + .prepareSearch("idx") + .addAggregation( + terms("terms") + .field("tag") + .order(Order.term(true)) + .subAggregation( + histogram("histo").field(SINGLE_VALUED_FIELD_NAME).interval(interval).minDocCount(0) + .extendedBounds((long) minRandomValue, (long) maxRandomValue)) + .subAggregation(maxBucket("max_histo_bucket").setBucketsPaths("histo>_count"))) + .addAggregation(maxBucket("max_terms_bucket").setBucketsPaths("terms>max_histo_bucket")).execute().actionGet(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + assertThat(terms.getName(), equalTo("terms")); + List termsBuckets = terms.getBuckets(); + assertThat(termsBuckets.size(), equalTo(interval)); + + List maxTermsKeys = new ArrayList<>(); + double maxTermsValue = Double.NEGATIVE_INFINITY; + for (int i = 0; i < interval; ++i) { + Terms.Bucket termsBucket = termsBuckets.get(i); + assertThat(termsBucket, notNullValue()); + assertThat((String) termsBucket.getKey(), equalTo("tag" + (i % interval))); + + Histogram histo = termsBucket.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + + List maxHistoKeys = new ArrayList<>(); + double maxHistoValue = Double.NEGATIVE_INFINITY; + for (int j = 0; j < numValueBuckets; ++j) { + Histogram.Bucket bucket = buckets.get(j); + assertThat(bucket, notNullValue()); + assertThat(((Number) bucket.getKey()).longValue(), equalTo((long) j * interval)); + if (bucket.getDocCount() > maxHistoValue) { + maxHistoValue = bucket.getDocCount(); + maxHistoKeys = new ArrayList<>(); + maxHistoKeys.add(bucket.getKeyAsString()); + } else if (bucket.getDocCount() == maxHistoValue) { + maxHistoKeys.add(bucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = termsBucket.getAggregations().get("max_histo_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_histo_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxHistoValue)); + assertThat(maxBucketValue.keys(), equalTo(maxHistoKeys.toArray(new String[maxHistoKeys.size()]))); + if (maxHistoValue > maxTermsValue) { + maxTermsValue = maxHistoValue; + maxTermsKeys = new ArrayList<>(); + maxTermsKeys.add(termsBucket.getKeyAsString()); + } else if (maxHistoValue == maxTermsValue) { + maxTermsKeys.add(termsBucket.getKeyAsString()); + } + } + + InternalBucketMetricValue maxBucketValue = response.getAggregations().get("max_terms_bucket"); + assertThat(maxBucketValue, notNullValue()); + assertThat(maxBucketValue.getName(), equalTo("max_terms_bucket")); + assertThat(maxBucketValue.value(), equalTo(maxTermsValue)); + assertThat(maxBucketValue.keys(), equalTo(maxTermsKeys.toArray(new String[maxTermsKeys.size()]))); + } +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/ReducerHelperTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/ReducerHelperTests.java new file mode 100644 index 00000000000..0b0f720344f --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/ReducerHelperTests.java @@ -0,0 +1,131 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers; + + +import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregationBuilder; +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.sum.SumBuilder; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.util.ArrayList; + +/** + * Provides helper methods and classes for use in Reducer tests, such as creating mock histograms or computing + * simple metrics + */ +public class ReducerHelperTests extends ElasticsearchTestCase { + + /** + * Generates a mock histogram to use for testing. Each MockBucket holds a doc count, key and document values + * which can later be used to compute metrics and compare against the real aggregation results. Gappiness can be + * controlled via parameters + * + * @param interval Interval between bucket keys + * @param size Size of mock histogram to generate (in buckets) + * @param gapProbability Probability of generating an empty bucket. 0.0-1.0 inclusive + * @param runProbability Probability of extending a gap once one has been created. 0.0-1.0 inclusive + * @return + */ + public static ArrayList generateHistogram(int interval, int size, double gapProbability, double runProbability) { + ArrayList values = new ArrayList<>(size); + + boolean lastWasGap = false; + + for (int i = 0; i < size; i++) { + MockBucket bucket = new MockBucket(); + if (randomDouble() < gapProbability) { + // start a gap + bucket.count = 0; + bucket.docValues = new double[0]; + + lastWasGap = true; + + } else if (lastWasGap && randomDouble() < runProbability) { + // add to the existing gap + bucket.count = 0; + bucket.docValues = new double[0]; + + lastWasGap = true; + } else { + bucket.count = randomIntBetween(1, 50); + bucket.docValues = new double[bucket.count]; + for (int j = 0; j < bucket.count; j++) { + bucket.docValues[j] = randomDouble() * randomIntBetween(-20,20); + } + lastWasGap = false; + } + + bucket.key = i * interval; + values.add(bucket); + } + + return values; + } + + /** + * Simple mock bucket container + */ + public static class MockBucket { + public int count; + public double[] docValues; + public long key; + } + + /** + * Computes a simple agg metric (min, sum, etc) from the provided values + * + * @param values Array of values to compute metric for + * @param metric A metric builder which defines what kind of metric should be returned for the values + * @return + */ + public static double calculateMetric(double[] values, ValuesSourceMetricsAggregationBuilder metric) { + + if (metric instanceof MinBuilder) { + double accumulator = Double.MAX_VALUE; + for (double value : values) { + accumulator = Math.min(accumulator, value); + } + return accumulator; + } else if (metric instanceof MaxBuilder) { + double accumulator = Double.MIN_VALUE; + for (double value : values) { + accumulator = Math.max(accumulator, value); + } + return accumulator; + } else if (metric instanceof SumBuilder) { + double accumulator = 0; + for (double value : values) { + accumulator += value; + } + return accumulator; + } else if (metric instanceof AvgBuilder) { + double accumulator = 0; + for (double value : values) { + accumulator += value; + } + return accumulator / values.length; + } + + return 0.0; + } +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgTests.java new file mode 100644 index 00000000000..069f9904a3f --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgTests.java @@ -0,0 +1,1086 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.moving.avg; + + +import com.google.common.collect.EvictingQueue; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.index.query.RangeFilterBuilder; +import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter; +import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram; +import org.elasticsearch.search.aggregations.bucket.histogram.InternalHistogram.Bucket; +import org.elasticsearch.search.aggregations.metrics.ValuesSourceMetricsAggregationBuilder; +import org.elasticsearch.search.aggregations.reducers.BucketHelpers; +import org.elasticsearch.search.aggregations.reducers.ReducerHelperTests; +import org.elasticsearch.search.aggregations.reducers.SimpleValue; +import org.elasticsearch.search.aggregations.reducers.movavg.models.DoubleExpModel; +import org.elasticsearch.search.aggregations.reducers.movavg.models.LinearModel; +import org.elasticsearch.search.aggregations.reducers.movavg.models.MovAvgModelBuilder; +import org.elasticsearch.search.aggregations.reducers.movavg.models.SimpleModel; +import org.elasticsearch.search.aggregations.reducers.movavg.models.SingleExpModel; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.hamcrest.Matchers; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; +import static org.elasticsearch.search.aggregations.AggregationBuilders.filter; +import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.max; +import static org.elasticsearch.search.aggregations.AggregationBuilders.min; +import static org.elasticsearch.search.aggregations.AggregationBuilders.range; +import static org.elasticsearch.search.aggregations.reducers.ReducerBuilders.movingAvg; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.core.IsNull.notNullValue; +import static org.hamcrest.core.IsNull.nullValue; + +@ElasticsearchIntegrationTest.SuiteScopeTest +public class MovAvgTests extends ElasticsearchIntegrationTest { + + private static final String INTERVAL_FIELD = "l_value"; + private static final String VALUE_FIELD = "v_value"; + private static final String GAP_FIELD = "g_value"; + + static int interval; + static int numBuckets; + static int windowSize; + static double alpha; + static double beta; + static BucketHelpers.GapPolicy gapPolicy; + static ValuesSourceMetricsAggregationBuilder metric; + static List mockHisto; + + static Map> testValues; + + + enum MovAvgType { + SIMPLE ("simple"), LINEAR("linear"), SINGLE("single"), DOUBLE("double"); + + private final String name; + + MovAvgType(String s) { + name = s; + } + + public String toString(){ + return name; + } + } + + enum MetricTarget { + VALUE ("value"), COUNT("count"); + + private final String name; + + MetricTarget(String s) { + name = s; + } + + public String toString(){ + return name; + } + } + + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx"); + createIndex("idx_unmapped"); + List builders = new ArrayList<>(); + + + interval = 5; + numBuckets = randomIntBetween(6, 80); + windowSize = randomIntBetween(3, 10); + alpha = randomDouble(); + beta = randomDouble(); + + gapPolicy = randomBoolean() ? BucketHelpers.GapPolicy.SKIP : BucketHelpers.GapPolicy.INSERT_ZEROS; + metric = randomMetric("the_metric", VALUE_FIELD); + mockHisto = ReducerHelperTests.generateHistogram(interval, numBuckets, randomDouble(), randomDouble()); + + testValues = new HashMap<>(8); + + for (MovAvgType type : MovAvgType.values()) { + for (MetricTarget target : MetricTarget.values()) { + setupExpected(type, target); + } + } + + for (ReducerHelperTests.MockBucket mockBucket : mockHisto) { + for (double value : mockBucket.docValues) { + builders.add(client().prepareIndex("idx", "type").setSource(jsonBuilder().startObject() + .field(INTERVAL_FIELD, mockBucket.key) + .field(VALUE_FIELD, value).endObject())); + } + } + + // Used for specially crafted gap tests + builders.add(client().prepareIndex("idx", "gap_type").setSource(jsonBuilder().startObject() + .field(INTERVAL_FIELD, 0) + .field(GAP_FIELD, 1).endObject())); + + builders.add(client().prepareIndex("idx", "gap_type").setSource(jsonBuilder().startObject() + .field(INTERVAL_FIELD, 49) + .field(GAP_FIELD, 1).endObject())); + + indexRandom(true, builders); + ensureSearchable(); + } + + /** + * Calculates the moving averages for a specific (model, target) tuple based on the previously generated mock histogram. + * Computed values are stored in the testValues map. + * + * @param type The moving average model to use + * @param target The document field "target", e.g. _count or a field value + */ + private void setupExpected(MovAvgType type, MetricTarget target) { + ArrayList values = new ArrayList<>(numBuckets); + EvictingQueue window = EvictingQueue.create(windowSize); + + for (ReducerHelperTests.MockBucket mockBucket : mockHisto) { + double metricValue; + double[] docValues = mockBucket.docValues; + + // Gaps only apply to metric values, not doc _counts + if (mockBucket.count == 0 && target.equals(MetricTarget.VALUE)) { + // If there was a gap in doc counts and we are ignoring, just skip this bucket + if (gapPolicy.equals(BucketHelpers.GapPolicy.SKIP)) { + values.add(null); + continue; + } else if (gapPolicy.equals(BucketHelpers.GapPolicy.INSERT_ZEROS)) { + // otherwise insert a zero instead of the true value + metricValue = 0.0; + } else { + metricValue = ReducerHelperTests.calculateMetric(docValues, metric); + } + + } else { + // If this isn't a gap, or is a _count, just insert the value + metricValue = target.equals(MetricTarget.VALUE) ? ReducerHelperTests.calculateMetric(docValues, metric) : mockBucket.count; + } + + window.offer(metricValue); + switch (type) { + case SIMPLE: + values.add(simple(window)); + break; + case LINEAR: + values.add(linear(window)); + break; + case SINGLE: + values.add(singleExp(window)); + break; + case DOUBLE: + values.add(doubleExp(window)); + break; + } + + } + testValues.put(type.toString() + "_" + target.toString(), values); + } + + /** + * Simple, unweighted moving average + * + * @param window Window of values to compute movavg for + * @return + */ + private double simple(Collection window) { + double movAvg = 0; + for (double value : window) { + movAvg += value; + } + movAvg /= window.size(); + return movAvg; + } + + /** + * Linearly weighted moving avg + * + * @param window Window of values to compute movavg for + * @return + */ + private double linear(Collection window) { + double avg = 0; + long totalWeight = 1; + long current = 1; + + for (double value : window) { + avg += value * current; + totalWeight += current; + current += 1; + } + return avg / totalWeight; + } + + /** + * Single exponential moving avg + * + * @param window Window of values to compute movavg for + * @return + */ + private double singleExp(Collection window) { + double avg = 0; + boolean first = true; + + for (double value : window) { + if (first) { + avg = value; + first = false; + } else { + avg = (value * alpha) + (avg * (1 - alpha)); + } + } + return avg; + } + + /** + * Double exponential moving avg + * @param window Window of values to compute movavg for + * @return + */ + private double doubleExp(Collection window) { + double s = 0; + double last_s = 0; + + // Trend value + double b = 0; + double last_b = 0; + + int counter = 0; + + double last; + for (double value : window) { + last = value; + if (counter == 1) { + s = value; + b = value - last; + } else { + s = alpha * value + (1.0d - alpha) * (last_s + last_b); + b = beta * (s - last_s) + (1 - beta) * last_b; + } + + counter += 1; + last_s = s; + last_b = b; + } + + return s + (0 * b) ; + } + + + + + /** + * test simple moving average on single value field + */ + @Test + @AwaitsFix(bugUrl = "Fails with certain seeds including -Dtests.seed=D9EF60095522804F") + public void simpleSingleValuedField() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(metric) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("_count")) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size())); + + List expectedCounts = testValues.get(MovAvgType.SIMPLE.toString() + "_" + MetricTarget.COUNT.toString()); + List expectedValues = testValues.get(MovAvgType.SIMPLE.toString() + "_" + MetricTarget.VALUE.toString()); + + Iterator actualIter = buckets.iterator(); + Iterator expectedBucketIter = mockHisto.iterator(); + Iterator expectedCountsIter = expectedCounts.iterator(); + Iterator expectedValuesIter = expectedValues.iterator(); + + while (actualIter.hasNext()) { + assertValidIterators(expectedBucketIter, expectedCountsIter, expectedValuesIter); + + Histogram.Bucket actual = actualIter.next(); + ReducerHelperTests.MockBucket expected = expectedBucketIter.next(); + Double expectedCount = expectedCountsIter.next(); + Double expectedValue = expectedValuesIter.next(); + + assertThat("keys do not match", ((Number) actual.getKey()).longValue(), equalTo(expected.key)); + assertThat("doc counts do not match", actual.getDocCount(), equalTo((long)expected.count)); + + assertBucketContents(actual, expectedCount, expectedValue); + } + } + + @Test + @AwaitsFix(bugUrl = "Fails with certain seeds including -Dtests.seed=D9EF60095522804F") + public void linearSingleValuedField() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(metric) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new LinearModel.LinearModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("_count")) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(new LinearModel.LinearModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size())); + + List expectedCounts = testValues.get(MovAvgType.LINEAR.toString() + "_" + MetricTarget.COUNT.toString()); + List expectedValues = testValues.get(MovAvgType.LINEAR.toString() + "_" + MetricTarget.VALUE.toString()); + + Iterator actualIter = buckets.iterator(); + Iterator expectedBucketIter = mockHisto.iterator(); + Iterator expectedCountsIter = expectedCounts.iterator(); + Iterator expectedValuesIter = expectedValues.iterator(); + + while (actualIter.hasNext()) { + assertValidIterators(expectedBucketIter, expectedCountsIter, expectedValuesIter); + + Histogram.Bucket actual = actualIter.next(); + ReducerHelperTests.MockBucket expected = expectedBucketIter.next(); + Double expectedCount = expectedCountsIter.next(); + Double expectedValue = expectedValuesIter.next(); + + assertThat("keys do not match", ((Number) actual.getKey()).longValue(), equalTo(expected.key)); + assertThat("doc counts do not match", actual.getDocCount(), equalTo((long)expected.count)); + + assertBucketContents(actual, expectedCount, expectedValue); + } + } + + @Test + @AwaitsFix(bugUrl = "Fails with certain seeds including -Dtests.seed=D9EF60095522804F") + public void singleSingleValuedField() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(metric) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new SingleExpModel.SingleExpModelBuilder().alpha(alpha)) + .gapPolicy(gapPolicy) + .setBucketsPaths("_count")) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(new SingleExpModel.SingleExpModelBuilder().alpha(alpha)) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size())); + + List expectedCounts = testValues.get(MovAvgType.SINGLE.toString() + "_" + MetricTarget.COUNT.toString()); + List expectedValues = testValues.get(MovAvgType.SINGLE.toString() + "_" + MetricTarget.VALUE.toString()); + + Iterator actualIter = buckets.iterator(); + Iterator expectedBucketIter = mockHisto.iterator(); + Iterator expectedCountsIter = expectedCounts.iterator(); + Iterator expectedValuesIter = expectedValues.iterator(); + + while (actualIter.hasNext()) { + assertValidIterators(expectedBucketIter, expectedCountsIter, expectedValuesIter); + + Histogram.Bucket actual = actualIter.next(); + ReducerHelperTests.MockBucket expected = expectedBucketIter.next(); + Double expectedCount = expectedCountsIter.next(); + Double expectedValue = expectedValuesIter.next(); + + assertThat("keys do not match", ((Number) actual.getKey()).longValue(), equalTo(expected.key)); + assertThat("doc counts do not match", actual.getDocCount(), equalTo((long)expected.count)); + + assertBucketContents(actual, expectedCount, expectedValue); + } + } + + @Test + @AwaitsFix(bugUrl = "Fails with certain seeds including -Dtests.seed=D9EF60095522804F") + public void doubleSingleValuedField() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(metric) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new DoubleExpModel.DoubleExpModelBuilder().alpha(alpha).beta(beta)) + .gapPolicy(gapPolicy) + .setBucketsPaths("_count")) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(new DoubleExpModel.DoubleExpModelBuilder().alpha(alpha).beta(beta)) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(mockHisto.size())); + + List expectedCounts = testValues.get(MovAvgType.DOUBLE.toString() + "_" + MetricTarget.COUNT.toString()); + List expectedValues = testValues.get(MovAvgType.DOUBLE.toString() + "_" + MetricTarget.VALUE.toString()); + + Iterator actualIter = buckets.iterator(); + Iterator expectedBucketIter = mockHisto.iterator(); + Iterator expectedCountsIter = expectedCounts.iterator(); + Iterator expectedValuesIter = expectedValues.iterator(); + + while (actualIter.hasNext()) { + assertValidIterators(expectedBucketIter, expectedCountsIter, expectedValuesIter); + + Histogram.Bucket actual = actualIter.next(); + ReducerHelperTests.MockBucket expected = expectedBucketIter.next(); + Double expectedCount = expectedCountsIter.next(); + Double expectedValue = expectedValuesIter.next(); + + assertThat("keys do not match", ((Number) actual.getKey()).longValue(), equalTo(expected.key)); + assertThat("doc counts do not match", actual.getDocCount(), equalTo((long)expected.count)); + + assertBucketContents(actual, expectedCount, expectedValue); + } + } + + @Test + public void testSizeZeroWindow() { + try { + client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(0) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + fail("MovingAvg should not accept a window that is zero"); + + } catch (SearchPhaseExecutionException exception) { + // All good + } + } + + @Test + public void testBadParent() { + try { + client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + range("histo").field(INTERVAL_FIELD).addRange(0, 10) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(0) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + fail("MovingAvg should not accept non-histogram as parent"); + + } catch (SearchPhaseExecutionException exception) { + // All good + } + } + + @Test + public void testNegativeWindow() { + try { + client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(-10) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("_count")) + ).execute().actionGet(); + fail("MovingAvg should not accept a window that is negative"); + + } catch (SearchPhaseExecutionException exception) { + //Throwable rootCause = exception.unwrapCause(); + //assertThat(rootCause, instanceOf(SearchParseException.class)); + //assertThat("[window] value must be a positive, non-zero integer. Value supplied was [0] in [movingAvg].", equalTo(exception.getMessage())); + } + } + + @Test + public void testNoBucketsInHistogram() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field("test").interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat(buckets.size(), equalTo(0)); + } + + @Test + public void testNoBucketsInHistogramWithPredict() { + int numPredictions = randomIntBetween(1,10); + SearchResponse response = client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field("test").interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(new SimpleModel.SimpleModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric") + .predict(numPredictions)) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat(buckets.size(), equalTo(0)); + } + + @Test + public void testZeroPrediction() { + try { + client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .predict(0) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + fail("MovingAvg should not accept a prediction size that is zero"); + + } catch (SearchPhaseExecutionException exception) { + // All Good + } + } + + @Test + public void testNegativePrediction() { + try { + client() + .prepareSearch("idx").setTypes("type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(interval).minDocCount(0) + .extendedBounds(0L, (long) (interval * (numBuckets - 1))) + .subAggregation(randomMetric("the_metric", VALUE_FIELD)) + .subAggregation(movingAvg("movavg_counts") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .predict(-10) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + fail("MovingAvg should not accept a prediction size that is negative"); + + } catch (SearchPhaseExecutionException exception) { + // All Good + } + } + + /** + * This test uses the "gap" dataset, which is simply a doc at the beginning and end of + * the INTERVAL_FIELD range. These docs have a value of 1 in GAP_FIELD. + * This test verifies that large gaps don't break things, and that the mov avg roughly works + * in the correct manner (checks direction of change, but not actual values) + */ + @Test + public void testGiantGap() { + + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(min("the_metric").field(GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50)); + + double lastValue = ((SimpleValue)(buckets.get(0).getAggregations().get("movavg_values"))).value(); + assertThat(Double.compare(lastValue, 0.0d), greaterThanOrEqualTo(0)); + + double currentValue; + for (int i = 1; i < 49; i++) { + SimpleValue current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + // Since there are only two values in this test, at the beginning and end, the moving average should + // decrease every step (until it reaches zero). Crude way to check that it's doing the right thing + // without actually verifying the computed values. Should work for all types of moving avgs and + // gap policies + assertThat(Double.compare(lastValue, currentValue), greaterThanOrEqualTo(0)); + lastValue = currentValue; + } + } + + + SimpleValue current = buckets.get(49).getAggregations().get("movavg_values"); + assertThat(current, notNullValue()); + currentValue = current.value(); + + if (gapPolicy.equals(BucketHelpers.GapPolicy.SKIP)) { + // if we are ignoring, movavg could go up (double_exp) or stay the same (simple, linear, single_exp) + assertThat(Double.compare(lastValue, currentValue), lessThanOrEqualTo(0)); + } else if (gapPolicy.equals(BucketHelpers.GapPolicy.INSERT_ZEROS)) { + // If we insert zeros, this should always increase the moving avg since the last bucket has a real value + assertThat(Double.compare(lastValue, currentValue), equalTo(-1)); + } + } + + /** + * Big gap, but with prediction at the end. + */ + @Test + public void testGiantGapWithPredict() { + int numPredictions = randomIntBetween(1, 10); + + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(min("the_metric").field(GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric") + .predict(numPredictions)) + ).execute().actionGet(); + + assertSearchResponse(response); + + InternalHistogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50 + numPredictions)); + + double lastValue = ((SimpleValue)(buckets.get(0).getAggregations().get("movavg_values"))).value(); + assertThat(Double.compare(lastValue, 0.0d), greaterThanOrEqualTo(0)); + + double currentValue; + for (int i = 1; i < 49; i++) { + SimpleValue current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + // Since there are only two values in this test, at the beginning and end, the moving average should + // decrease every step (until it reaches zero). Crude way to check that it's doing the right thing + // without actually verifying the computed values. Should work for all types of moving avgs and + // gap policies + assertThat(Double.compare(lastValue, currentValue), greaterThanOrEqualTo(0)); + lastValue = currentValue; + } + } + + SimpleValue current = buckets.get(49).getAggregations().get("movavg_values"); + assertThat(current, notNullValue()); + currentValue = current.value(); + + if (gapPolicy.equals(BucketHelpers.GapPolicy.SKIP)) { + // if we are ignoring, movavg could go up (double_exp) or stay the same (simple, linear, single_exp) + assertThat(Double.compare(lastValue, currentValue), lessThanOrEqualTo(0)); + } else if (gapPolicy.equals(BucketHelpers.GapPolicy.INSERT_ZEROS)) { + // If we insert zeros, this should always increase the moving avg since the last bucket has a real value + assertThat(Double.compare(lastValue, currentValue), equalTo(-1)); + } + + // Now check predictions + for (int i = 50; i < 50 + numPredictions; i++) { + // Unclear at this point which direction the predictions will go, just verify they are + // not null, and that we don't have the_metric anymore + assertThat((buckets.get(i).getAggregations().get("movavg_values")), notNullValue()); + assertThat((buckets.get(i).getAggregations().get("the_metric")), nullValue()); + } + } + + /** + * This test filters the "gap" data so that the first doc is excluded. This leaves a long stretch of empty + * buckets until the final bucket. The moving avg should be zero up until the last bucket, and should work + * regardless of mov avg type or gap policy. + */ + @Test + public void testLeftGap() { + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + filter("filtered").filter(new RangeFilterBuilder(INTERVAL_FIELD).from(1)).subAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(randomMetric("the_metric", GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + )) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalFilter filtered = response.getAggregations().get("filtered"); + assertThat(filtered, notNullValue()); + assertThat(filtered.getName(), equalTo("filtered")); + + InternalHistogram histo = filtered.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50)); + + double lastValue = 0; + + double currentValue; + for (int i = 0; i < 50; i++) { + SimpleValue current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + assertThat(Double.compare(lastValue, currentValue), lessThanOrEqualTo(0)); + lastValue = currentValue; + } + } + } + + @Test + public void testLeftGapWithPredict() { + int numPredictions = randomIntBetween(1, 10); + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + filter("filtered").filter(new RangeFilterBuilder(INTERVAL_FIELD).from(1)).subAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(randomMetric("the_metric", GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric") + .predict(numPredictions)) + )) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalFilter filtered = response.getAggregations().get("filtered"); + assertThat(filtered, notNullValue()); + assertThat(filtered.getName(), equalTo("filtered")); + + InternalHistogram histo = filtered.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50 + numPredictions)); + + double lastValue = 0; + + double currentValue; + for (int i = 0; i < 50; i++) { + SimpleValue current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + assertThat(Double.compare(lastValue, currentValue), lessThanOrEqualTo(0)); + lastValue = currentValue; + } + } + + // Now check predictions + for (int i = 50; i < 50 + numPredictions; i++) { + // Unclear at this point which direction the predictions will go, just verify they are + // not null, and that we don't have the_metric anymore + assertThat((buckets.get(i).getAggregations().get("movavg_values")), notNullValue()); + assertThat((buckets.get(i).getAggregations().get("the_metric")), nullValue()); + } + } + + /** + * This test filters the "gap" data so that the last doc is excluded. This leaves a long stretch of empty + * buckets after the first bucket. The moving avg should be one at the beginning, then zero for the rest + * regardless of mov avg type or gap policy. + */ + @Test + public void testRightGap() { + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + filter("filtered").filter(new RangeFilterBuilder(INTERVAL_FIELD).to(1)).subAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(randomMetric("the_metric", GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric")) + )) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalFilter filtered = response.getAggregations().get("filtered"); + assertThat(filtered, notNullValue()); + assertThat(filtered.getName(), equalTo("filtered")); + + InternalHistogram histo = filtered.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50)); + + + SimpleValue current = buckets.get(0).getAggregations().get("movavg_values"); + assertThat(current, notNullValue()); + + double lastValue = current.value(); + + double currentValue; + for (int i = 1; i < 50; i++) { + current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + assertThat(Double.compare(lastValue, currentValue), greaterThanOrEqualTo(0)); + lastValue = currentValue; + } + } + } + + @Test + public void testRightGapWithPredict() { + int numPredictions = randomIntBetween(1, 10); + SearchResponse response = client() + .prepareSearch("idx").setTypes("gap_type") + .addAggregation( + filter("filtered").filter(new RangeFilterBuilder(INTERVAL_FIELD).to(1)).subAggregation( + histogram("histo").field(INTERVAL_FIELD).interval(1).minDocCount(0).extendedBounds(0L, 49L) + .subAggregation(randomMetric("the_metric", GAP_FIELD)) + .subAggregation(movingAvg("movavg_values") + .window(windowSize) + .modelBuilder(randomModelBuilder()) + .gapPolicy(gapPolicy) + .setBucketsPaths("the_metric") + .predict(numPredictions)) + )) + .execute().actionGet(); + + assertSearchResponse(response); + + InternalFilter filtered = response.getAggregations().get("filtered"); + assertThat(filtered, notNullValue()); + assertThat(filtered.getName(), equalTo("filtered")); + + InternalHistogram histo = filtered.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + assertThat(histo.getName(), equalTo("histo")); + List buckets = histo.getBuckets(); + assertThat("Size of buckets array is not correct.", buckets.size(), equalTo(50 + numPredictions)); + + + SimpleValue current = buckets.get(0).getAggregations().get("movavg_values"); + assertThat(current, notNullValue()); + + double lastValue = current.value(); + + double currentValue; + for (int i = 1; i < 50; i++) { + current = buckets.get(i).getAggregations().get("movavg_values"); + if (current != null) { + currentValue = current.value(); + + assertThat(Double.compare(lastValue, currentValue), greaterThanOrEqualTo(0)); + lastValue = currentValue; + } + } + + // Now check predictions + for (int i = 50; i < 50 + numPredictions; i++) { + // Unclear at this point which direction the predictions will go, just verify they are + // not null, and that we don't have the_metric anymore + assertThat((buckets.get(i).getAggregations().get("movavg_values")), notNullValue()); + assertThat((buckets.get(i).getAggregations().get("the_metric")), nullValue()); + } + } + + + private void assertValidIterators(Iterator expectedBucketIter, Iterator expectedCountsIter, Iterator expectedValuesIter) { + if (!expectedBucketIter.hasNext()) { + fail("`expectedBucketIter` iterator ended before `actual` iterator, size mismatch"); + } + if (!expectedCountsIter.hasNext()) { + fail("`expectedCountsIter` iterator ended before `actual` iterator, size mismatch"); + } + if (!expectedValuesIter.hasNext()) { + fail("`expectedValuesIter` iterator ended before `actual` iterator, size mismatch"); + } + } + + private void assertBucketContents(Histogram.Bucket actual, Double expectedCount, Double expectedValue) { + // This is a gap bucket + SimpleValue countMovAvg = actual.getAggregations().get("movavg_counts"); + if (expectedCount == null) { + assertThat("[_count] movavg is not null", countMovAvg, nullValue()); + } else { + assertThat("[_count] movavg is null", countMovAvg, notNullValue()); + assertThat("[_count] movavg does not match expected ["+countMovAvg.value()+" vs "+expectedCount+"]", + Math.abs(countMovAvg.value() - expectedCount) <= 0.000001, equalTo(true)); + } + + // This is a gap bucket + SimpleValue valuesMovAvg = actual.getAggregations().get("movavg_values"); + if (expectedValue == null) { + assertThat("[value] movavg is not null", valuesMovAvg, Matchers.nullValue()); + } else { + assertThat("[value] movavg is null", valuesMovAvg, notNullValue()); + assertThat("[value] movavg does not match expected ["+valuesMovAvg.value()+" vs "+expectedValue+"]", Math.abs(valuesMovAvg.value() - expectedValue) <= 0.000001, equalTo(true)); + } + } + + private MovAvgModelBuilder randomModelBuilder() { + int rand = randomIntBetween(0,3); + + switch (rand) { + case 0: + return new SimpleModel.SimpleModelBuilder(); + case 1: + return new LinearModel.LinearModelBuilder(); + case 2: + return new SingleExpModel.SingleExpModelBuilder().alpha(alpha); + case 3: + return new DoubleExpModel.DoubleExpModelBuilder().alpha(alpha).beta(beta); + default: + return new SimpleModel.SimpleModelBuilder(); + } + } + + private ValuesSourceMetricsAggregationBuilder randomMetric(String name, String field) { + int rand = randomIntBetween(0,3); + + switch (rand) { + case 0: + return min(name).field(field); + case 2: + return max(name).field(field); + case 3: + return avg(name).field(field); + default: + return avg(name).field(field); + } + } + +} diff --git a/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgUnitTests.java b/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgUnitTests.java new file mode 100644 index 00000000000..156f4f873a7 --- /dev/null +++ b/src/test/java/org/elasticsearch/search/aggregations/reducers/moving/avg/MovAvgUnitTests.java @@ -0,0 +1,297 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.reducers.moving.avg; + +import com.google.common.collect.EvictingQueue; +import org.elasticsearch.search.aggregations.reducers.movavg.models.*; +import org.elasticsearch.test.ElasticsearchTestCase; +import static org.hamcrest.Matchers.equalTo; +import org.junit.Test; + +public class MovAvgUnitTests extends ElasticsearchTestCase { + + @Test + public void testSimpleMovAvgModel() { + MovAvgModel model = new SimpleModel(); + + int numValues = randomIntBetween(1, 100); + int windowSize = randomIntBetween(1, 50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < numValues; i++) { + + double randValue = randomDouble(); + double expected = 0; + + window.offer(randValue); + + for (double value : window) { + expected += value; + } + expected /= window.size(); + + double actual = model.next(window); + assertThat(Double.compare(expected, actual), equalTo(0)); + } + } + + @Test + public void testSimplePredictionModel() { + MovAvgModel model = new SimpleModel(); + + int windowSize = randomIntBetween(1, 50); + int numPredictions = randomIntBetween(1,50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < windowSize; i++) { + window.offer(randomDouble()); + } + double actual[] = model.predict(window, numPredictions); + + double expected[] = new double[numPredictions]; + for (int i = 0; i < numPredictions; i++) { + for (double value : window) { + expected[i] += value; + } + expected[i] /= window.size(); + window.offer(expected[i]); + } + + for (int i = 0; i < numPredictions; i++) { + assertThat(Double.compare(expected[i], actual[i]), equalTo(0)); + } + } + + @Test + public void testLinearMovAvgModel() { + MovAvgModel model = new LinearModel(); + + int numValues = randomIntBetween(1, 100); + int windowSize = randomIntBetween(1, 50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < numValues; i++) { + double randValue = randomDouble(); + window.offer(randValue); + + double avg = 0; + long totalWeight = 1; + long current = 1; + + for (double value : window) { + avg += value * current; + totalWeight += current; + current += 1; + } + double expected = avg / totalWeight; + double actual = model.next(window); + assertThat(Double.compare(expected, actual), equalTo(0)); + } + } + + @Test + public void testLinearPredictionModel() { + MovAvgModel model = new LinearModel(); + + int windowSize = randomIntBetween(1, 50); + int numPredictions = randomIntBetween(1,50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < windowSize; i++) { + window.offer(randomDouble()); + } + double actual[] = model.predict(window, numPredictions); + double expected[] = new double[numPredictions]; + + for (int i = 0; i < numPredictions; i++) { + double avg = 0; + long totalWeight = 1; + long current = 1; + + for (double value : window) { + avg += value * current; + totalWeight += current; + current += 1; + } + expected[i] = avg / totalWeight; + window.offer(expected[i]); + } + + for (int i = 0; i < numPredictions; i++) { + assertThat(Double.compare(expected[i], actual[i]), equalTo(0)); + } + } + + @Test + public void testSingleExpMovAvgModel() { + double alpha = randomDouble(); + MovAvgModel model = new SingleExpModel(alpha); + + int numValues = randomIntBetween(1, 100); + int windowSize = randomIntBetween(1, 50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < numValues; i++) { + double randValue = randomDouble(); + window.offer(randValue); + + double avg = 0; + boolean first = true; + + for (double value : window) { + if (first) { + avg = value; + first = false; + } else { + avg = (value * alpha) + (avg * (1 - alpha)); + } + } + double expected = avg; + double actual = model.next(window); + assertThat(Double.compare(expected, actual), equalTo(0)); + } + } + + @Test + public void testSinglePredictionModel() { + double alpha = randomDouble(); + MovAvgModel model = new SingleExpModel(alpha); + + int windowSize = randomIntBetween(1, 50); + int numPredictions = randomIntBetween(1,50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < windowSize; i++) { + window.offer(randomDouble()); + } + double actual[] = model.predict(window, numPredictions); + double expected[] = new double[numPredictions]; + + for (int i = 0; i < numPredictions; i++) { + double avg = 0; + boolean first = true; + + for (double value : window) { + if (first) { + avg = value; + first = false; + } else { + avg = (value * alpha) + (avg * (1 - alpha)); + } + } + expected[i] = avg; + window.offer(expected[i]); + } + + for (int i = 0; i < numPredictions; i++) { + assertThat(Double.compare(expected[i], actual[i]), equalTo(0)); + } + } + + @Test + public void testDoubleExpMovAvgModel() { + double alpha = randomDouble(); + double beta = randomDouble(); + MovAvgModel model = new DoubleExpModel(alpha, beta); + + int numValues = randomIntBetween(1, 100); + int windowSize = randomIntBetween(1, 50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < numValues; i++) { + double randValue = randomDouble(); + window.offer(randValue); + + double s = 0; + double last_s = 0; + + // Trend value + double b = 0; + double last_b = 0; + int counter = 0; + + double last; + for (double value : window) { + last = value; + if (counter == 1) { + s = value; + b = value - last; + } else { + s = alpha * value + (1.0d - alpha) * (last_s + last_b); + b = beta * (s - last_s) + (1 - beta) * last_b; + } + + counter += 1; + last_s = s; + last_b = b; + } + + double expected = s + (0 * b) ; + double actual = model.next(window); + assertThat(Double.compare(expected, actual), equalTo(0)); + } + } + + @Test + public void testDoublePredictionModel() { + double alpha = randomDouble(); + double beta = randomDouble(); + MovAvgModel model = new DoubleExpModel(alpha, beta); + + int windowSize = randomIntBetween(1, 50); + int numPredictions = randomIntBetween(1,50); + + EvictingQueue window = EvictingQueue.create(windowSize); + for (int i = 0; i < windowSize; i++) { + window.offer(randomDouble()); + } + double actual[] = model.predict(window, numPredictions); + double expected[] = new double[numPredictions]; + + double s = 0; + double last_s = 0; + + // Trend value + double b = 0; + double last_b = 0; + int counter = 0; + + double last; + for (double value : window) { + last = value; + if (counter == 1) { + s = value; + b = value - last; + } else { + s = alpha * value + (1.0d - alpha) * (last_s + last_b); + b = beta * (s - last_s) + (1 - beta) * last_b; + } + + counter += 1; + last_s = s; + last_b = b; + } + + for (int i = 0; i < numPredictions; i++) { + expected[i] = s + (i * b); + assertThat(Double.compare(expected[i], actual[i]), equalTo(0)); + } + } +} diff --git a/src/test/java/org/elasticsearch/search/query/SearchQueryTests.java b/src/test/java/org/elasticsearch/search/query/SearchQueryTests.java index 6a575fc38c7..8d8e948f769 100644 --- a/src/test/java/org/elasticsearch/search/query/SearchQueryTests.java +++ b/src/test/java/org/elasticsearch/search/query/SearchQueryTests.java @@ -587,6 +587,44 @@ public class SearchQueryTests extends ElasticsearchIntegrationTest { assertHitCount(searchResponse, 1l); } + @Test // https://github.com/elasticsearch/elasticsearch/issues/10477 + public void testDateRangeInQueryStringWithTimeZone_10477() { + //the mapping needs to be provided upfront otherwise we are not sure how many failures we get back + //as with dynamic mappings some shards might be lacking behind and parse a different query + assertAcked(prepareCreate("test").addMapping( + "type", "past", "type=date" + )); + ensureGreen(); + + client().prepareIndex("test", "type", "1").setSource("past", "2015-04-05T23:00:00+0000").get(); + client().prepareIndex("test", "type", "2").setSource("past", "2015-04-06T00:00:00+0000").get(); + refresh(); + + // Timezone set with dates + SearchResponse searchResponse = client().prepareSearch() + .setQuery(queryStringQuery("past:[2015-04-06T00:00:00+0200 TO 2015-04-06T23:00:00+0200]")) + .get(); + assertHitCount(searchResponse, 2l); + + // Same timezone set with time_zone + searchResponse = client().prepareSearch() + .setQuery(queryStringQuery("past:[2015-04-06T00:00:00 TO 2015-04-06T23:00:00]").timeZone("+0200")) + .get(); + assertHitCount(searchResponse, 2l); + + // We set a timezone which will give no result + searchResponse = client().prepareSearch() + .setQuery(queryStringQuery("past:[2015-04-06T00:00:00-0200 TO 2015-04-06T23:00:00-0200]")) + .get(); + assertHitCount(searchResponse, 0l); + + // Same timezone set with time_zone but another timezone is set directly within dates which has the precedence + searchResponse = client().prepareSearch() + .setQuery(queryStringQuery("past:[2015-04-06T00:00:00-0200 TO 2015-04-06T23:00:00-0200]").timeZone("+0200")) + .get(); + assertHitCount(searchResponse, 0l); + } + @Test public void typeFilterTypeIndexedTests() throws Exception { typeFilterTests("not_analyzed"); diff --git a/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java b/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java index ff8264fdc03..8d569275aea 100644 --- a/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java +++ b/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreTests.java @@ -38,7 +38,9 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ProcessedClusterStateUpdateTask; +import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaData.Custom; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.io.stream.StreamInput; @@ -748,7 +750,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests )); } - public static abstract class TestCustomMetaData implements MetaData.Custom { + public static abstract class TestCustomMetaData extends AbstractDiffable implements MetaData.Custom { private final String data; protected TestCustomMetaData(String data) { @@ -776,194 +778,182 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests return data.hashCode(); } - public static abstract class TestCustomMetaDataFactory extends MetaData.Custom.Factory { + protected abstract TestCustomMetaData newTestCustomMetaData(String data); - protected abstract TestCustomMetaData newTestCustomMetaData(String data); + @Override + public Custom readFrom(StreamInput in) throws IOException { + return newTestCustomMetaData(in.readString()); + } - @Override - public T readFrom(StreamInput in) throws IOException { - return (T) newTestCustomMetaData(in.readString()); - } + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(getData()); + } - @Override - public void writeTo(T metadata, StreamOutput out) throws IOException { - out.writeString(metadata.getData()); - } - - @Override - public T fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token; - String data = null; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - String currentFieldName = parser.currentName(); - if ("data".equals(currentFieldName)) { - if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, invalid data type"); - } - data = parser.text(); - } else { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, unknown field [" + currentFieldName + "]"); + @Override + public Custom fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token; + String data = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + String currentFieldName = parser.currentName(); + if ("data".equals(currentFieldName)) { + if (parser.nextToken() != XContentParser.Token.VALUE_STRING) { + throw new ElasticsearchParseException("failed to parse snapshottable metadata, invalid data type"); } + data = parser.text(); } else { - throw new ElasticsearchParseException("failed to parse snapshottable metadata"); + throw new ElasticsearchParseException("failed to parse snapshottable metadata, unknown field [" + currentFieldName + "]"); } + } else { + throw new ElasticsearchParseException("failed to parse snapshottable metadata"); } - if (data == null) { - throw new ElasticsearchParseException("failed to parse snapshottable metadata, data not found"); - } - return (T) newTestCustomMetaData(data); } + if (data == null) { + throw new ElasticsearchParseException("failed to parse snapshottable metadata, data not found"); + } + return newTestCustomMetaData(data); + } - @Override - public void toXContent(T metadata, XContentBuilder builder, ToXContent.Params params) throws IOException { - builder.field("data", metadata.getData()); - } + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.field("data", getData()); + return builder; } } + static { - MetaData.registerFactory(SnapshottableMetadata.TYPE, SnapshottableMetadata.FACTORY); - MetaData.registerFactory(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata.FACTORY); - MetaData.registerFactory(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata.FACTORY); - MetaData.registerFactory(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata.FACTORY); - MetaData.registerFactory(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata.FACTORY); + MetaData.registerPrototype(SnapshottableMetadata.TYPE, SnapshottableMetadata.PROTO); + MetaData.registerPrototype(NonSnapshottableMetadata.TYPE, NonSnapshottableMetadata.PROTO); + MetaData.registerPrototype(SnapshottableGatewayMetadata.TYPE, SnapshottableGatewayMetadata.PROTO); + MetaData.registerPrototype(NonSnapshottableGatewayMetadata.TYPE, NonSnapshottableGatewayMetadata.PROTO); + MetaData.registerPrototype(SnapshotableGatewayNoApiMetadata.TYPE, SnapshotableGatewayNoApiMetadata.PROTO); } public static class SnapshottableMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable"; - public static final Factory FACTORY = new Factory(); + public static final SnapshottableMetadata PROTO = new SnapshottableMetadata(""); public SnapshottableMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new SnapshottableMetadata(data); + } - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableMetadata(data); - } - - @Override - public EnumSet context() { - return MetaData.API_AND_SNAPSHOT; - } + @Override + public EnumSet context() { + return MetaData.API_AND_SNAPSHOT; } } public static class NonSnapshottableMetadata extends TestCustomMetaData { public static final String TYPE = "test_non_snapshottable"; - public static final Factory FACTORY = new Factory(); + public static final NonSnapshottableMetadata PROTO = new NonSnapshottableMetadata(""); public NonSnapshottableMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected NonSnapshottableMetadata newTestCustomMetaData(String data) { + return new NonSnapshottableMetadata(data); + } - @Override - protected NonSnapshottableMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableMetadata(data); - } + @Override + public EnumSet context() { + return MetaData.API_ONLY; } } public static class SnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable_gateway"; - public static final Factory FACTORY = new Factory(); + public static final SnapshottableGatewayMetadata PROTO = new SnapshottableGatewayMetadata(""); public SnapshottableGatewayMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new SnapshottableGatewayMetadata(data); + } - @Override - protected TestCustomMetaData newTestCustomMetaData(String data) { - return new SnapshottableGatewayMetadata(data); - } - - @Override - public EnumSet context() { - return EnumSet.of(MetaData.XContentContext.API, MetaData.XContentContext.SNAPSHOT, MetaData.XContentContext.GATEWAY); - } + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.API, MetaData.XContentContext.SNAPSHOT, MetaData.XContentContext.GATEWAY); } } public static class NonSnapshottableGatewayMetadata extends TestCustomMetaData { public static final String TYPE = "test_non_snapshottable_gateway"; - public static final Factory FACTORY = new Factory(); + public static final NonSnapshottableGatewayMetadata PROTO = new NonSnapshottableGatewayMetadata(""); public NonSnapshottableGatewayMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { - - @Override - public String type() { - return TYPE; - } - - @Override - protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) { - return new NonSnapshottableGatewayMetadata(data); - } - - @Override - public EnumSet context() { - return MetaData.API_AND_GATEWAY; - } - + @Override + public String type() { + return TYPE; } + + @Override + protected NonSnapshottableGatewayMetadata newTestCustomMetaData(String data) { + return new NonSnapshottableGatewayMetadata(data); + } + + @Override + public EnumSet context() { + return MetaData.API_AND_GATEWAY; + } + } public static class SnapshotableGatewayNoApiMetadata extends TestCustomMetaData { public static final String TYPE = "test_snapshottable_gateway_no_api"; - public static final Factory FACTORY = new Factory(); + public static final SnapshotableGatewayNoApiMetadata PROTO = new SnapshotableGatewayNoApiMetadata(""); public SnapshotableGatewayNoApiMetadata(String data) { super(data); } - private static class Factory extends TestCustomMetaDataFactory { + @Override + public String type() { + return TYPE; + } - @Override - public String type() { - return TYPE; - } - - @Override - protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) { - return new SnapshotableGatewayNoApiMetadata(data); - } - - @Override - public EnumSet context() { - return EnumSet.of(MetaData.XContentContext.GATEWAY, MetaData.XContentContext.SNAPSHOT); - } + @Override + protected SnapshotableGatewayNoApiMetadata newTestCustomMetaData(String data) { + return new SnapshotableGatewayNoApiMetadata(data); + } + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.GATEWAY, MetaData.XContentContext.SNAPSHOT); } } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 9e68d16caa0..187afe1b658 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -166,6 +166,8 @@ import java.util.concurrent.atomic.AtomicLong; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; +import static org.elasticsearch.test.XContentTestUtils.convertToMap; +import static org.elasticsearch.test.XContentTestUtils.mapsEqualIgnoringArrayOrder; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; @@ -357,7 +359,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase * Creates a randomized index template. This template is used to pass in randomized settings on a * per index basis. Allows to enable/disable the randomization for number of shards and replicas */ - private void randomIndexTemplate() throws IOException { + public void randomIndexTemplate() throws IOException { // TODO move settings for random directory etc here into the index based randomized settings. if (cluster().size() > 0) { @@ -647,6 +649,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase .transientSettings().getAsMap().size(), equalTo(0)); } ensureClusterSizeConsistency(); + ensureClusterStateConsistency(); cluster().wipe(); // wipe after to make sure we fail in the test that didn't ack the delete if (afterClass || currentClusterScope == Scope.TEST) { cluster().close(); @@ -1085,8 +1088,8 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase */ public void setMinimumMasterNodes(int n) { assertTrue(client().admin().cluster().prepareUpdateSettings().setTransientSettings( - settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, n)) - .get().isAcknowledged()); + settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, n)) + .get().isAcknowledged()); } /** @@ -1133,6 +1136,50 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase } } + /** + * Verifies that all nodes that have the same version of the cluster state as master have same cluster state + */ + protected void ensureClusterStateConsistency() throws IOException { + if (cluster() != null) { + boolean getResolvedAddress = InetSocketTransportAddress.getResolveAddress(); + try { + InetSocketTransportAddress.setResolveAddress(false); + ClusterState masterClusterState = client().admin().cluster().prepareState().all().get().getState(); + byte[] masterClusterStateBytes = ClusterState.Builder.toBytes(masterClusterState); + // remove local node reference + masterClusterState = ClusterState.Builder.fromBytes(masterClusterStateBytes, null); + Map masterStateMap = convertToMap(masterClusterState); + int masterClusterStateSize = ClusterState.Builder.toBytes(masterClusterState).length; + String masterId = masterClusterState.nodes().masterNodeId(); + for (Client client : cluster()) { + ClusterState localClusterState = client.admin().cluster().prepareState().all().setLocal(true).get().getState(); + byte[] localClusterStateBytes = ClusterState.Builder.toBytes(localClusterState); + // remove local node reference + localClusterState = ClusterState.Builder.fromBytes(localClusterStateBytes, null); + Map localStateMap = convertToMap(localClusterState); + int localClusterStateSize = localClusterStateBytes.length; + // Check that the non-master node has the same version of the cluster state as the master and that this node didn't disconnect from the master + if (masterClusterState.version() == localClusterState.version() && localClusterState.nodes().nodes().containsKey(masterId)) { + try { + assertThat(masterClusterState.uuid(), equalTo(localClusterState.uuid())); + // We cannot compare serialization bytes since serialization order of maps is not guaranteed + // but we can compare serialization sizes - they should be the same + assertThat(masterClusterStateSize, equalTo(localClusterStateSize)); + // Compare JSON serialization + assertThat(mapsEqualIgnoringArrayOrder(masterStateMap, localStateMap), equalTo(true)); + } catch (AssertionError error) { + logger.error("Cluster state from master:\n{}\nLocal cluster state:\n{}", masterClusterState.toString(), localClusterState.toString()); + throw error; + } + } + } + } finally { + InetSocketTransportAddress.setResolveAddress(getResolvedAddress); + } + } + + } + /** * Ensures the cluster is in a searchable state for the given indices. This means a searchable copy of each * shard is available on the cluster. diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java index 1276089b182..0f71b7239e0 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchTestCase.java @@ -71,6 +71,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import static com.google.common.collect.Lists.newArrayList; /** * Base testcase for randomized unit testing with Elasticsearch @@ -595,4 +596,17 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase { return threadGroup.getName(); } } + + /** + * Returns size random values + */ + public static List randomSubsetOf(int size, T... values) { + if (size > values.length) { + throw new IllegalArgumentException("Can\'t pick " + size + " random objects from a list of " + values.length + " objects"); + } + List list = newArrayList(values); + Collections.shuffle(list); + return list.subList(0, size); + } + } diff --git a/src/test/java/org/elasticsearch/test/XContentTestUtils.java b/src/test/java/org/elasticsearch/test/XContentTestUtils.java new file mode 100644 index 00000000000..1f1b8eff710 --- /dev/null +++ b/src/test/java/org/elasticsearch/test/XContentTestUtils.java @@ -0,0 +1,104 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.test; + +import com.carrotsearch.ant.tasks.junit4.dependencies.com.google.common.collect.Lists; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; + +public final class XContentTestUtils { + private XContentTestUtils() { + + } + + public static Map convertToMap(ToXContent part) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + part.toXContent(builder, EMPTY_PARAMS); + builder.endObject(); + return XContentHelper.convertToMap(builder.bytes(), false).v2(); + } + + + /** + * Compares to maps generated from XContentObjects. The order of elements in arrays is ignored + */ + public static boolean mapsEqualIgnoringArrayOrder(Map first, Map second) { + if (first.size() != second.size()) { + return false; + } + + for (String key : first.keySet()) { + if (objectsEqualIgnoringArrayOrder(first.get(key), second.get(key)) == false) { + return false; + } + } + return true; + } + + @SuppressWarnings("unchecked") + private static boolean objectsEqualIgnoringArrayOrder(Object first, Object second) { + if (first == null ) { + return second == null; + } else if (first instanceof List) { + if (second instanceof List) { + List secondList = Lists.newArrayList((List) second); + List firstList = (List) first; + if (firstList.size() == secondList.size()) { + for (Object firstObj : firstList) { + boolean found = false; + for (Object secondObj : secondList) { + if (objectsEqualIgnoringArrayOrder(firstObj, secondObj)) { + secondList.remove(secondObj); + found = true; + break; + } + } + if (found == false) { + return false; + } + } + return secondList.isEmpty(); + } else { + return false; + } + } else { + return false; + } + } else if (first instanceof Map) { + if (second instanceof Map) { + return mapsEqualIgnoringArrayOrder((Map) first, (Map) second); + } else { + return false; + } + } else { + return first.equals(second); + } + } + +} diff --git a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java index dd6ae14612d..b7b207a6b11 100644 --- a/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java +++ b/src/test/java/org/elasticsearch/test/rest/ElasticsearchRestTestCase.java @@ -20,8 +20,6 @@ package org.elasticsearch.test.rest; import com.carrotsearch.randomizedtesting.RandomizedTest; -import com.carrotsearch.randomizedtesting.annotations.Name; -import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import com.carrotsearch.randomizedtesting.annotations.TestGroup; import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import com.google.common.collect.Lists; @@ -221,9 +219,6 @@ public abstract class ElasticsearchRestTestCase extends ElasticsearchIntegration if (!restApi.getMethods().contains("POST")) { errorMessage.append("\n- ").append(restApi.getName()).append(" supports GET with a body but doesn't support POST"); } - if (!restApi.getParams().contains("source")) { - errorMessage.append("\n- ").append(restApi.getName()).append(" supports GET with a body but doesn't support the source query string parameter"); - } } } if (errorMessage.length() > 0) {