From 8abe7a55b1df40b1098e0c2d208a80d1a7a0e054 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 8 Oct 2018 14:22:50 -0700 Subject: [PATCH 01/45] Contributing Moving-Average Query to open source. --- distribution/pom.xml | 2 + .../moving-average-query.md | 335 ++++++++ docs/content/development/extensions.md | 1 + .../moving-average-query/README.md | 16 + .../moving-average-query/pom.xml | 89 ++ .../movingaverage/AveragerFactoryWrapper.java | 174 ++++ .../movingaverage/BucketingAccumulator.java | 65 ++ .../DefaultMovingAverageQueryMetrics.java | 61 ++ ...faultMovingAverageQueryMetricsFactory.java | 54 ++ .../IdentityYieldingAccumulator.java | 39 + .../movingaverage/MovingAverageHelper.java | 53 ++ .../movingaverage/MovingAverageIterable.java | 302 +++++++ .../movingaverage/MovingAverageQuery.java | 372 ++++++++ .../MovingAverageQueryMetrics.java | 41 + .../MovingAverageQueryMetricsFactory.java | 35 + .../MovingAverageQueryModule.java | 60 ++ .../MovingAverageQueryRunner.java | 238 ++++++ .../MovingAverageQueryToolChest.java | 146 ++++ .../movingaverage/PostAveragerCalculator.java | 59 ++ .../druid/query/movingaverage/RowBucket.java | 61 ++ .../movingaverage/RowBucketIterable.java | 152 ++++ .../movingaverage/averagers/Averager.java | 57 ++ .../averagers/AveragerFactory.java | 106 +++ .../movingaverage/averagers/BaseAverager.java | 199 +++++ .../averagers/BaseAveragerFactory.java | 102 +++ .../averagers/ComparableAveragerFactory.java | 53 ++ .../averagers/ConstantAverager.java | 80 ++ .../averagers/ConstantAveragerFactory.java | 101 +++ .../averagers/DoubleMaxAverager.java | 48 ++ .../averagers/DoubleMaxAveragerFactory.java | 43 + .../averagers/DoubleMeanAverager.java | 52 ++ .../averagers/DoubleMeanAveragerFactory.java | 43 + .../averagers/DoubleMeanNoNullAverager.java | 50 ++ .../DoubleMeanNoNullAveragerFactory.java | 42 + .../averagers/DoubleMinAverager.java | 48 ++ .../averagers/DoubleMinAveragerFactory.java | 42 + .../averagers/LongMaxAverager.java | 48 ++ .../averagers/LongMaxAveragerFactory.java | 42 + .../averagers/LongMeanAverager.java | 52 ++ .../averagers/LongMeanAveragerFactory.java | 43 + .../averagers/LongMeanNoNullAverager.java | 50 ++ .../LongMeanNoNullAveragerFactory.java | 43 + .../averagers/LongMinAverager.java | 49 ++ .../averagers/LongMinAveragerFactory.java | 43 + .../averagers/SketchUnionAverager.java | 56 ++ .../averagers/SketchUnionAveragerFactory.java | 80 ++ ...rg.apache.druid.initialization.DruidModule | 1 + .../IdentityYieldingAccumulatorTest.java | 59 ++ .../MovingAverageIterableTest.java | 802 ++++++++++++++++++ .../movingaverage/MovingAverageQueryTest.java | 442 ++++++++++ .../PostAveragerCalculatorTest.java | 108 +++ .../movingaverage/RowBucketIterableTest.java | 672 +++++++++++++++ .../averagers/BaseAveragerFactoryTest.java | 67 ++ .../averagers/BaseAveragerTest.java | 155 ++++ .../DoubleMaxAveragerFactoryTest.java | 37 + .../averagers/DoubleMaxAveragerTest.java | 56 ++ .../DoubleMeanAveragerFactoryTest.java | 36 + .../averagers/DoubleMeanAveragerTest.java | 57 ++ .../DoubleMeanAveragerWithPeriodTest.java | 80 ++ .../DoubleMeanNoNullAveragerFactoryTest.java | 36 + .../DoubleMeanNoNullAveragerTest.java | 81 ++ .../DoubleMinAveragerFactoryTest.java | 36 + .../averagers/DoubleMinAveragerTest.java | 57 ++ .../averagers/LongMaxAveragerFactoryTest.java | 36 + .../averagers/LongMaxAveragerTest.java | 56 ++ .../LongMeanAveragerFactoryTest.java | 36 + .../averagers/LongMeanAveragerTest.java | 56 ++ .../LongMeanNoNullAveragerFactoryTest.java | 36 + .../averagers/LongMeanNoNullAveragerTest.java | 56 ++ .../averagers/LongMinAveragerFactoryTest.java | 36 + .../averagers/LongMinAveragerTest.java | 57 ++ .../SketchUnionAveragerFactoryTest.java | 36 + .../averagers/SketchUnionAveragerTest.java | 81 ++ .../query/movingaverage/test/TestConfig.java | 34 + .../queryTests/basicGroupByMovingAverage.yaml | 57 ++ .../basicGroupByMovingAverage2.yaml | 57 ++ .../basicTimeseriesMovingAverage.yaml | 51 ++ .../queryTests/missingGroupByValues.yaml | 79 ++ .../queryTests/movingAverageWithSketches.yaml | 70 ++ .../queryTests/sortingAveragersAsc.yaml | 81 ++ .../queryTests/sortingAveragersDesc.yaml | 82 ++ ...ortingWithNonMovingAndMovingAvgMetric.yaml | 84 ++ .../sortingWithNonMovingAvgMetric.yaml | 82 ++ .../src/test/resources/runtime.properties | 57 ++ pom.xml | 1 + 85 files changed, 7930 insertions(+) create mode 100644 docs/content/development/extensions-contrib/moving-average-query.md create mode 100644 extensions-contrib/moving-average-query/README.md create mode 100644 extensions-contrib/moving-average-query/pom.xml create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/runtime.properties diff --git a/distribution/pom.xml b/distribution/pom.xml index 4876d8416e2a..37947cbc8d2f 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -215,6 +215,8 @@ org.apache.druid.extensions.contrib:materialized-view-maintenance -c org.apache.druid.extensions.contrib:materialized-view-selection + -c + org.apache.druid.extensions.contrib:druid-moving-average-query diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md new file mode 100644 index 000000000000..387b397ae991 --- /dev/null +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -0,0 +1,335 @@ +--- +layout: doc_page +--- + +# Moving Average Queries + +## Overview +**Moving Average Query** is an extension which provides support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries. + +These Aggregate Window Functions consume standard Druid Aggregators and outputs additional windowed aggregates called [Averagers](#averagers). + +#### High level algorithm + +Moving Average encapsulates the [groupBy query](../../querying/groupbyquery.html) (Or [timeseries](../../querying/timeseriesquery.html) in case of no dimensions) in order to rely on the maturity of these query types. + +It runs the query in two main phases: +1. Runs an inner [groupBy](../../querying/groupbyquery.html) or [timeseries](../../querying/timeseriesquery.html) query to compute Aggregators (i.e. daily count of events). +2. Passes over aggregated results in Broker, in order to compute Averagers (i.e. moving 7 day average of the daily count). + +#### Main enhancements provided by this extension: +1. Functionality: Extending druid query functionality (i.e. initial introduction of Window Functions). +2. Performance: Improving performance of such moving aggregations by eliminating multiple segment scans. + +#### Further reading +[Moving Average](https://en.wikipedia.org/wiki/Moving_average) + +[Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) + +[Analytic Functions](https://cloud.google.com/bigquery/docs/reference/standard-sql/analytic-function-concepts) + + +## Operations +To use this extension, make sure to [load](../../operations/including-extensions.html) `druid-moving-average-query` only to the Broker. + +##Configuration +There are currently no configuration properties specific to Moving Average. + +##Query spec: +* Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. +* Note: movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. +* Note: movingAverage is missing support for the following timeseries properties: `descending`. + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information |yes| +|dimensions|A JSON list of dimensions to do the groupBy over; can be ommitted for using timeseries back-engine; or see [DimensionSpec](../../querying/dimensionspecs.html) for ways to extract dimensions |no| +|limitSpec|See [LimitSpec](../../querying/limitspec.html).|no| +|having|See [Having](../../querying/having.html).|no| +|granularity|Granularity type must be:`period`; See [Granularities](../../querying/granularities.html)|yes| +|filter|See [Filters](../../querying/filters.html)|no| +|aggregations|Aggregations forms the input to Averagers; See [Aggregations](../../querying/aggregations.html)|yes| +|postAggregations|Supports only aggregations as input; See [Post Aggregations](../../querying/post-aggregations.html)|no| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| +|averagers|Defines the moving average function; See [Aggregations](../../querying/aggregations.html)|yes| +|postAveragers|Support input of both averagers and aggregations; Syntax is identical to postAggregations (See [Post Aggregations](../../querying/post-aggregations.html))|no| + +##Averagers + +Averagers are used to define the Moving-Average function. Averagers are not limited to an average - they can also provide other types of window functions such as MAX()/MIN(). + +### Properties + +These are properties which are common to all Averagers: + +|property|description|required?| +|--------|-----------|---------| +|type|Averager type; See [Averager types](#averager-types)|yes| +|name|Averager name|yes| +|fieldName|Input name (An aggregation name)|yes| +|buckets|Number of lookback buckets (time periods), including current one. Must be >0|yes| +|cycleSize|Cycle size; Used to calculate day-of-week option; See [Cycle size (Day of Week)](#cycle-size-day-of-week)|no, defaults to 1| + + +###Averager types: + +* [Standard averagers](#standard-averagers): + * doubleMean + * doubleMeanNoNulls + * doubleMax + * doubleMin + * longMean + * longMeanNoNulls + * longMax + * longMin +* [Sketch averagers](#sketch-averagers): + * sketchUnion + +#### Standard averagers + +These averagers offer four functions: +* Mean (Average) +* MeanNoNulls (Ignores empty buckets). +* Max +* Min + +**Ignoring nulls**: +Using a MeanNoNulls averager is useful when the interval starts at the dataset beginning time. +In that case, the first records will ignore missing buckets and average won't be artificially low. +However, this also means that empty days in a sparse dataset will also be ignored. + +Example of usage: +```json +{ "type" : "doubleMean", "name" : , "fieldName": } +``` + +#### Sketch averagers + +Sketch averager are meant to perform merge operations on [DataSketches](../extensions-core/datasketches-extension.html) (When using a Sketch averager, please include the [DataSketches](../extensions-core/datasketches-extension.html) extension as well). + +Extra properties for Sketch averagers: + +|property|description|required?| +|--------|-----------|---------| +|size|Sketch size; See [DataSketches aggregator](../extensions-core/datasketches-aggregators.html)|no, defaults to 4096| + +Available functions: +* sketchUnion + +Example of usage: +```json +{ "type" : "sketchUnion", "name" : , "fieldName": } +``` + +### Cycle size (Day of Week) +This optional parameter is used to calculate over a single bucket within each cycle instead of all buckets. +A prime example would be weekly buckets, resulting in a Day of Week calculation. (Other examples: Month of year, Hour of day). + +I.e. when using these parameters: +* *granularity*: period=P1D (daily) +* *buckets*: 28 +* *cycleSize*: 7 + +Within each output record, the averager will compute the result over the following buckets: current (#0), #7, #14, #21. +Whereas without specifying cycleSize it would have computed over all 28 buckets. + +## Examples + +All examples are based on the Wikipedia dataset provided in the Druid [tutorials](../../tutorials/index.html). + +### Basic example + +Calculating a 7-buckets moving average for Wikipedia edit deltas. + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT30M" + }, + "intervals": [ + "2015-09-12T00:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta30Min", + "fieldName": "delta", + "type": "longSum" + } + ], + "averagers": [ + { + "name": "trailing30MinChanges", + "fieldName": "delta30Min", + "type": "longMean", + "buckets": 7 + } + ] +} +``` + +Result: +```json +[ { + "version" : "v1", + "timestamp" : "2015-09-12T00:30:00.000Z", + "event" : { + "delta30Min" : 30490, + "trailing30MinChanges" : 4355.714285714285 + } + }, { + "version" : "v1", + "timestamp" : "2015-09-12T01:00:00.000Z", + "event" : { + "delta30Min" : 96526, + "trailing30MinChanges" : 18145.14285714286 + } + }, { +... +... +... +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:00:00.000Z", + "event" : { + "delta30Min" : 119100, + "trailing30MinChanges" : 198697.2857142857 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:30:00.000Z", + "event" : { + "delta30Min" : 177882, + "trailing30MinChanges" : 193890.0 + } +} +``` + +### Post averager example + +Calculating a 7-buckets moving average for Wikipedia edit deltas, plus a ratio between the current period and the moving average. + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT30M" + }, + "intervals": [ + "2015-09-12T22:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta30Min", + "fieldName": "delta", + "type": "longSum" + } + ], + "averagers": [ + { + "name": "trailing30MinChanges", + "fieldName": "delta30Min", + "type": "longMean", + "buckets": 7 + } + ], + "postAveragers" : [ + { + "name": "ratioTrailing30MinChanges", + "type": "arithmetic", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "fieldName": "delta30Min" + }, + { + "type": "fieldAccess", + "fieldName": "trailing30MinChanges" + } + ] + } + ] +} +``` + +Result: +```json +[ { + "version" : "v1", + "timestamp" : "2015-09-12T22:00:00.000Z", + "event" : { + "delta30Min" : 144269, + "trailing30MinChanges" : 204088.14285714287, + "ratioTrailing30MinChanges" : 0.7068955500319539 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T22:30:00.000Z", + "event" : { + "delta30Min" : 242860, + "trailing30MinChanges" : 214031.57142857142, + "ratioTrailing30MinChanges" : 1.134692411867141 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:00:00.000Z", + "event" : { + "delta30Min" : 119100, + "trailing30MinChanges" : 198697.2857142857, + "ratioTrailing30MinChanges" : 0.5994042624782422 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:30:00.000Z", + "event" : { + "delta30Min" : 177882, + "trailing30MinChanges" : 193890.0, + "ratioTrailing30MinChanges" : 0.9174377224199288 + } +} ] +``` + + +### Cycle size example + +Calculating an average of every first 10-minutes of the last 3 hours: + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT10M" + }, + "intervals": [ + "2015-09-12T00:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta10Min", + "fieldName": "delta", + "type": "doubleSum" + } + ], + "averagers": [ + { + "name": "trailing10MinPerHourChanges", + "fieldName": "delta10Min", + "type": "doubleMeanNoNulls", + "buckets": 18, + "cycleSize": 6 + } + ] +} +``` diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 7a1e5dd70a49..981a0be4ea13 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -73,6 +73,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c |kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)| |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| |druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)| +|druid-moving-average-query|Support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries.|[link](../development/extensions-contrib/moving-average-query.html)| ## Promoting Community Extension to Core Extension diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md new file mode 100644 index 000000000000..b92a6dbce303 --- /dev/null +++ b/extensions-contrib/moving-average-query/README.md @@ -0,0 +1,16 @@ +druid-moving-average-query +============= + +Overview +============= +**Moving Average Query** is an extension which provides support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries. + +Documentation +============= +See the druid.io website or under [Druid Github Repo](https://github.com/apache/incubator-druid/tree/master/docs/content/development/extensions-contrib/moving-average-query.md). + +Thanks +=========== + +This module was created by [Will Lauer](https://github.com/will-lauer) and [Mangesh Pardeshi](https://github.com/mangesh-pardeshi), with additional contributions by: [Deepak Babu](https://github.com/deepakb91), [Asif Mansoor](https://github.com/asifmansoora), [Ke Wang](https://github.com/kenuiuc), [Eyal Yurman](https://github.com/yurmix). +Thanks to all contributors! diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml new file mode 100644 index 000000000000..e9c4e77671c0 --- /dev/null +++ b/extensions-contrib/moving-average-query/pom.xml @@ -0,0 +1,89 @@ + + + + 4.0.0 + + + + druid + org.apache.druid + 0.13.0-SNAPSHOT + ../../pom.xml + + + org.apache.druid.extensions.contrib + druid-moving-average-query + druid-moving-average-query + + + UTF-8 + + + + + org.jmockit + jmockit + 1.25 + test + + + junit + junit + test + + + org.hamcrest + hamcrest-library + 1.3 + test + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + 2.8.3 + test + + + org.apache.druid + druid-api + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid.extensions + druid-datasketches + ${project.parent.version} + provided + + + diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java new file mode 100644 index 000000000000..00b5982db946 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Comparator; +import java.util.List; + +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; + +/** + * A wrapper around averagers that makes them appear to be aggregators. + * This is necessary purely to allow existing common druid code that only knows + * about aggregators to work with the MovingAverageQuery query as well. + * + * @param Result type + * @param Finalized Result type + */ +public class AveragerFactoryWrapper extends AggregatorFactory +{ + + private final AveragerFactory af; + private final String prefix; + + /** + * Simple constructor + * + * @param af + * @param prefix + */ + public AveragerFactoryWrapper(AveragerFactory af, String prefix) + { + this.af = af; + this.prefix = prefix; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return null; + } + + /* (non-Javadoc) + * @see org.apache.druid.query.aggregation.AggregatorFactory#getComparator() + */ + @Override + public Comparator getComparator() + { + return af.getComparator(); + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Object combine(Object lhs, Object rhs) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public AggregatorFactory getCombiningFactory() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public List getRequiredColumns() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Object deserialize(Object object) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @SuppressWarnings("unchecked") + @Override + public Object finalizeComputation(Object object) + { + return af.finalizeComputation((T) object); + } + + /* (non-Javadoc) + * @see org.apache.druid.query.aggregation.AggregatorFactory#getName() + */ + @Override + public String getName() + { + return prefix + af.getName(); + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public List requiredFields() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public byte[] getCacheKey() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public String getTypeName() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public int getMaxIntermediateSize() + { + return 0; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java new file mode 100644 index 000000000000..34ec4353e65a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.YieldingAccumulator; + +/** + * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. + */ +public class BucketingAccumulator extends YieldingAccumulator +{ + + /* (non-Javadoc) + * @see YieldingAccumulator#accumulate(java.lang.Object, java.lang.Object) + */ + @Override + public RowBucket accumulate(RowBucket accumulated, Row in) + { + Collection rows; + + if (accumulated == null) { + // first row, initializing + rows = new ArrayList<>(); + accumulated = new RowBucket(in.getTimestamp(), rows); + } else if (accumulated.getNextBucket() != null) { + accumulated = accumulated.getNextBucket(); + } + + if (!accumulated.getDateTime().equals(in.getTimestamp())) { + // day change detected + rows = new ArrayList<>(); + rows.add(in); + RowBucket nextBucket = new RowBucket(in.getTimestamp(), rows); + accumulated.setNextBucket(nextBucket); + yield(); + } else { + // still on the same day + rows = accumulated.getRows(); + rows.add(in); + } + + return accumulated; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java new file mode 100644 index 000000000000..857e48201f2e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.query.DefaultQueryMetrics; +import org.apache.druid.query.DruidMetrics; + +public class DefaultMovingAverageQueryMetrics extends DefaultQueryMetrics implements + MovingAverageQueryMetrics +{ + + public DefaultMovingAverageQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(MovingAverageQuery query) + { + super.query(query); + numDimensions(query); + numMetrics(query); + numComplexMetrics(query); + } + + @Override + public void numDimensions(MovingAverageQuery query) + { + setDimension("numDimensions", String.valueOf(query.getDimensions().size())); + } + + @Override + public void numMetrics(MovingAverageQuery query) + { + setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + } + + @Override + public void numComplexMetrics(MovingAverageQuery query) + { + int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); + setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java new file mode 100644 index 000000000000..d494a2e48b9b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.jackson.DefaultObjectMapper; + +@LazySingleton +public class DefaultMovingAverageQueryMetricsFactory implements MovingAverageQueryMetricsFactory +{ + + private static final MovingAverageQueryMetricsFactory INSTANCE = + new DefaultMovingAverageQueryMetricsFactory(new DefaultObjectMapper()); + + @VisibleForTesting + public static MovingAverageQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultMovingAverageQueryMetricsFactory(@Json ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public MovingAverageQueryMetrics makeMetrics() + { + return new DefaultMovingAverageQueryMetrics(jsonMapper); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java new file mode 100644 index 000000000000..b0250b0a5c7e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.java.util.common.guava.YieldingAccumulator; + +/** + * Simple yielding accumulator that yields each row. + */ +public class IdentityYieldingAccumulator extends YieldingAccumulator +{ + + /* (non-Javadoc) + * @see YieldingAccumulator#accumulate(java.lang.Object, java.lang.Object) + */ + @Override + public T accumulate(T accumulated, T in) + { + yield(); + return in; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java new file mode 100644 index 000000000000..4188af4cba7c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.dimension.DimensionSpec; + +public class MovingAverageHelper +{ + + /** + * @param dimensions A list of DimensionSpec in the specified in the query + * @param row The Row to be used for looking up dimension values + * + * @return A Map of dimension/value from the row + */ + + public static Map getDimKeyFromRow(Collection dimensions, Row row) + { + + Map key = new HashMap<>(); + Map event = ((MapBasedRow) row).getEvent(); + + for (DimensionSpec dimension : dimensions) { + key.put(dimension.getOutputName(), event.get(dimension.getOutputName())); + } + + return key; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java new file mode 100644 index 000000000000..1cf6aa25a51a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.movingaverage.averagers.Averager; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.NullDimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link MovingAverageIterable} iterates over days {@link RowBucket}, producing rows for each dimension combination, + * filling in missing entries with "empty" rows so that the averaging buckets have enough data to operate on. + * It then computes the moving average on the buckets and returns the row. + * See {@link MovingAverageIterator#computeMovingAverage(Map, Row, boolean)} for more details. + */ +public class MovingAverageIterable implements Iterable +{ + + private final Sequence seq; + private final Collection dims; + private final Collection> factories; + private final Map postAggMap; + private final Map aggMap; + private final Map fakeEvents; + + public MovingAverageIterable( + Sequence buckets, + Collection dims, + Collection> factories, + List postAggList, + List aggList + ) + { + this.dims = dims; + this.factories = factories; + this.seq = buckets; + + postAggMap = postAggList.stream().collect(Collectors.toMap(postAgg -> postAgg.getName(), postAgg -> postAgg)); + aggMap = aggList.stream().collect(Collectors.toMap(agg -> agg.getName(), agg -> agg)); + + ColumnSelectorFactory colFact = new ColumnSelectorFactory() + { + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + // Generating empty records while aggregating on Filtered aggregators requires a dimension selector + // for initialization. This dimension selector is not actually used for generating values + return NullDimensionSelector.instance(); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String s) + { + return null; + } + + @Override + public ColumnCapabilities getColumnCapabilities(String s) + { + return null; + } + }; + // Fill in all the fake events + fakeEvents = new LinkedHashMap<>(); + aggMap.values().forEach(agg -> { + Aggregator aggFactorized = agg.factorize(colFact); + fakeEvents.put(agg.getName(), aggFactorized.get()); + }); + postAggMap.values().stream().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); + } + + /* (non-Javadoc) + * @see java.lang.Iterable#iterator() + */ + @Override + public Iterator iterator() + { + return new MovingAverageIterator(seq, dims, factories, fakeEvents, aggMap); + } + + static class MovingAverageIterator implements Iterator + { + + private final Collection dims; + private final Map, Collection>> averagers = new HashMap<>(); + private final Collection> factories; + + private Yielder yielder; + private RowBucket cache = null; + private Iterator cacheIter; + private Iterator> averagersKeysIter; + private Set> seenKeys = new HashSet<>(); + private Row saveNext; + private Map aggMap; + private Map fakeEvents; + + public MovingAverageIterator( + Sequence rows, + Collection dims, + Collection> factories, + Map fakeEvents, + Map aggMap + ) + { + this.dims = dims; + this.factories = factories; + this.fakeEvents = fakeEvents; + this.aggMap = aggMap; + + yielder = rows.toYielder(null, new IdentityYieldingAccumulator()); + } + + /* (non-Javadoc) + * @see java.util.Iterator#hasNext() + */ + @Override + public boolean hasNext() + { + + if (saveNext != null) { + return true; + } + + try { + saveNext = internalNext(); + return true; + } + catch (NoSuchElementException e) { + return false; + } + + + } + + /* (non-Javadoc) + * @see java.util.Iterator#next() + */ + @Override + public Row next() + { + + if (saveNext != null) { + Row retVal = saveNext; + saveNext = null; + return retVal; + } + + return internalNext(); + } + + private Row internalNext() + { + if (cache == null && !yielder.isDone()) { + cache = yielder.get(); + yielder = yielder.next(cache); + + cacheIter = cache.getRows().iterator(); + } + + Row r = null; + + // return rows from the cached RowBucket + if (cacheIter != null) { + if (cacheIter.hasNext()) { + r = cacheIter.next(); + Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); + seenKeys.add(key); + r = computeMovingAverage(key, r, false); + if (r != null) { + return r; + } else { + throw new NoSuchElementException(); + } + } else { + Set> averagerKeys = new HashSet<>(averagers.keySet()); + averagerKeys.removeAll(seenKeys); + averagersKeysIter = averagerKeys.iterator(); + cacheIter = null; + } + } + + // return fake rows for unseen dimension combinations + if (averagersKeysIter != null) { + while (averagersKeysIter.hasNext()) { + Map dims = averagersKeysIter.next(); + Map fakeEventsCopy = new HashMap<>(fakeEvents); + + dims.forEach((dim, value) -> { + fakeEventsCopy.put(dim, value); + }); + + r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), fakeEventsCopy), true); + if (r != null) { + return r; + } + } + + seenKeys.clear(); + averagersKeysIter = null; + cache = null; + } + + if (cacheIter == null && averagersKeysIter == null && yielder.isDone()) { + // we should never get here. For some reason, there is + // no more work to do, so continuing to iterate will infinite loop + throw new NoSuchElementException(); + } + + // nothing to do here, so move on to the next row + return internalNext(); + } + + /** + * Compute and add any moving average columns. + * + *

Normally, the row passed in will be added to all the {@link Averager}'s and then results pulled + * from each averager. If skip is true, then the incoming row is actually a dummy value due to + * no data being present for this dimension combination in the current bucket. When this happens, + * {@link Averager#skip()} should be called instead of {@link Averager#addElement(Map, Map)}()} to force proper + * decaying of the average values. + * + *

Usually, the contents of key will be contained by the row R being passed in, but in the case of a + * dummy row, its possible that the dimensions will be known but the row empty. Hence, the values are + * passed as two separate arguments. + * + * @param key The dimension set that this row applies to. + * @param r The Row to operate on + * @param skip Indicates whether skip or add should be called + * + * @return The updated row containing averager results, or null if no averagers computed a result + */ + private Row computeMovingAverage(Map key, Row r, boolean skip) + { + Map event = ((MapBasedRow) r).getEvent(); + Map result = new HashMap<>(event); + + Collection> avg = averagers.get(key); + + if (avg == null) { + avg = factories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); + averagers.put(key, avg); + } + + if (!skip) { + avg.stream().forEach(af -> af.addElement(event, aggMap)); + } else { + avg.stream().forEach(af -> af.skip()); + } + + avg.stream().forEach(af -> result.put(af.getName(), af.getResult())); + + // At least one non-dimension value must be in the record for it to be valid. + if (result.entrySet().stream().anyMatch(e -> !key.containsKey(e.getKey()) && e.getValue() != null)) { + result.putAll(event); + return new MapBasedRow(r.getTimestamp(), result); + } else { + // No averagers returned anything. All buckets must be empty. + // skip this row. + return null; + } + } + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java new file mode 100644 index 000000000000..52ddd83a230c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.groupby.having.HavingSpec; +import org.apache.druid.query.groupby.orderby.LimitSpec; +import org.apache.druid.query.groupby.orderby.NoopLimitSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Class that defines druid MovingAverage query fields + */ +@JsonTypeName("movingAverage") +public class MovingAverageQuery extends BaseQuery +{ + + public static final String MOVING_AVG_QUERY_TYPE = "movingAverage"; + public static final String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst"; + + private final LimitSpec limitSpec; + private final HavingSpec havingSpec; + private final DimFilter dimFilter; + private final Function, Sequence> limitFn; + private final Granularity granularity; + private final List dimensions; + private final List aggregatorSpecs; + private final List postAggregatorSpecs; + private final List> averagerSpecs; + private final List postAveragerSpecs; + + @JsonCreator + public MovingAverageQuery( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("filter") DimFilter dimFilter, + @JsonProperty("granularity") Granularity granularity, + @JsonProperty("dimensions") List dimensions, + @JsonProperty("aggregations") List aggregatorSpecs, + @JsonProperty("postAggregations") List postAggregatorSpecs, + @JsonProperty("having") HavingSpec havingSpec, + @JsonProperty("averagers") List> averagerSpecs, + @JsonProperty("postAveragers") List postAveragerSpecs, + @JsonProperty("limitSpec") LimitSpec limitSpec, + @JsonProperty("context") Map context + ) + { + super(dataSource, querySegmentSpec, false, context); + + this.dimFilter = dimFilter; + this.granularity = granularity; + this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + for (DimensionSpec spec : this.dimensions) { + Preconditions.checkArgument(spec != null, "dimensions has null DimensionSpec"); + } + this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; + this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; + this.averagerSpecs = averagerSpecs == null ? ImmutableList.>of() : averagerSpecs; + this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; + this.havingSpec = havingSpec; + this.limitSpec = (limitSpec == null) ? NoopLimitSpec.INSTANCE : limitSpec; + + Preconditions.checkNotNull(this.granularity, "Must specify a granularity"); + + verifyOutputNames(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs); + + // build combined list of aggregators and averagers so that limit spec building is happy + List combinedAggregatorSpecs = new ArrayList<>(); + combinedAggregatorSpecs.addAll(this.aggregatorSpecs); + for (AveragerFactory avg : this.averagerSpecs) { + combinedAggregatorSpecs.add(new AveragerFactoryWrapper(avg, "")); + } + + Function, Sequence> postProcFn = + this.limitSpec.build( + this.dimensions, + combinedAggregatorSpecs, + this.postAggregatorSpecs, + this.granularity, + getContextSortByDimsFirst() + ); + + if (havingSpec != null) { + postProcFn = Functions.compose( + postProcFn, + new Function, Sequence>() + { + @Override + public Sequence apply(Sequence input) + { + return Sequences.filter( + input, + new Predicate() + { + @Override + public boolean apply(Row input) + { + return MovingAverageQuery.this.havingSpec.eval(input); + } + } + ); + } + } + ); + } + + this.limitFn = postProcFn; + + } + + private static void verifyOutputNames( + List dimensions, + List aggregators, + List postAggregators + ) + { + + final Set outputNames = Sets.newHashSet(); + for (DimensionSpec dimension : dimensions) { + if (!outputNames.add(dimension.getOutputName())) { + throw new IAE("Duplicate output name[%s]", dimension.getOutputName()); + } + } + + for (AggregatorFactory aggregator : aggregators) { + if (!outputNames.add(aggregator.getName())) { + throw new IAE("Duplicate output name[%s]", aggregator.getName()); + } + } + + for (PostAggregator postAggregator : postAggregators) { + if (!outputNames.add(postAggregator.getName())) { + throw new IAE("Duplicate output name[%s]", postAggregator.getName()); + } + } + } + + /** + * A private constructor that avoids all of the various state checks. Used by the with*() methods where the checks + * have already passed in order for the object to exist. + */ + private MovingAverageQuery( + DataSource dataSource, + QuerySegmentSpec querySegmentSpec, + DimFilter dimFilter, + Granularity granularity, + List dimensions, + List aggregatorSpecs, + List> averagerSpecs, + List postAggregatorSpecs, + List postAveragerSpecs, + HavingSpec havingSpec, + LimitSpec orderBySpec, + Function, Sequence> limitFn, + Map context + ) + { + super(dataSource, querySegmentSpec, false, context); + + this.dimFilter = dimFilter; + this.granularity = granularity; + this.dimensions = dimensions; + this.aggregatorSpecs = aggregatorSpecs; + this.averagerSpecs = averagerSpecs; + this.postAggregatorSpecs = postAggregatorSpecs; + this.postAveragerSpecs = postAveragerSpecs; + this.havingSpec = havingSpec; + this.limitSpec = orderBySpec; + this.limitFn = limitFn; + } + + @Override + public boolean hasFilters() + { + return dimFilter != null; + } + + @Override + public String getType() + { + return MOVING_AVG_QUERY_TYPE; + } + + @JsonIgnore + public boolean getContextSortByDimsFirst() + { + return getContextBoolean(CTX_KEY_SORT_BY_DIMS_FIRST, false); + } + + @JsonProperty + public DimFilter getFilter() + { + return dimFilter; + } + + @JsonProperty + public Granularity getGranularity() + { + return granularity; + } + + @JsonProperty + public List getDimensions() + { + return dimensions; + } + + @JsonProperty("aggregations") + public List getAggregatorSpecs() + { + return aggregatorSpecs; + } + + @JsonProperty("averagers") + public List> getAveragerSpecs() + { + return averagerSpecs; + } + + @JsonProperty("postAggregations") + public List getPostAggregatorSpecs() + { + return postAggregatorSpecs; + } + + @JsonProperty("postAveragers") + public List getPostAveragerSpecs() + { + return postAveragerSpecs; + } + + @JsonProperty("having") + public HavingSpec getHavingSpec() + { + return havingSpec; + } + + @JsonProperty + public LimitSpec getLimitSpec() + { + return limitSpec; + } + + @Override + public MovingAverageQuery withOverriddenContext(Map contextOverride) + { + return new MovingAverageQuery( + getDataSource(), + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + computeOverridenContext(contextOverride) + ); + } + + @Override + public MovingAverageQuery withQuerySegmentSpec(QuerySegmentSpec spec) + { + return new MovingAverageQuery( + getDataSource(), + spec, + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + @Override + public Query withDataSource(DataSource dataSource) + { + return new MovingAverageQuery( + dataSource, + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + public Query withPostAveragers(List postAveragerSpecs) + { + return new MovingAverageQuery( + getDataSource(), + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + public Sequence applyLimit(Sequence results) + { + return limitFn.apply(results); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java new file mode 100644 index 000000000000..48a8e564a2c3 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.QueryMetrics; + +public interface MovingAverageQueryMetrics extends QueryMetrics +{ + /** + * Sets the size of {@link MovingAverageQuery#getDimensions()} of the given query as dimension. + */ + void numDimensions(MovingAverageQuery query); + + /** + * Sets the number of metrics of the given groupBy query as dimension. + */ + void numMetrics(MovingAverageQuery query); + + /** + * Sets the number of "complex" metrics of the given groupBy query as dimension. By default it is assumed that + * "complex" metric is a metric of not long or double type, but it could be redefined in the implementation of this + * method. + */ + void numComplexMetrics(MovingAverageQuery query); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java new file mode 100644 index 000000000000..3857d323b7fd --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +/** + * Implementations could be injected using + *

+ * PolyBind + * .optionBinder(binder, Key.get(MovingAverageQueryMetricsFactory.class)) + * .addBinding("myCustomMovingAverageQueryMetricsFactory") + * .to(MyCustomMovingAverageQueryMetricsFactory.class); + *

+ * And then setting property: + * druid.query.movingavgquery.queryMetricsFactory=myCustomMovingAverageQueryMetricsFactory + */ +public interface MovingAverageQueryMetricsFactory +{ + MovingAverageQueryMetrics makeMetrics(); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java new file mode 100644 index 000000000000..a7efa358e666 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; + +import com.google.inject.Binder; +import com.google.inject.multibindings.MapBinder; + +import org.apache.druid.guice.DruidBinders; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryToolChest; + +import java.util.Arrays; +import java.util.List; + +public class MovingAverageQueryModule implements DruidModule +{ + + @Override + public void configure(Binder binder) + { + MapBinder, QueryToolChest> toolChests = DruidBinders.queryToolChestBinder(binder); + + //Bind the query toolchest to the query class and add the binding to toolchest + toolChests.addBinding(MovingAverageQuery.class).to(MovingAverageQueryToolChest.class); + + //Bind the query toolchest to binder + binder.bind(MovingAverageQueryToolChest.class).in(LazySingleton.class); + } + + @Override + public List getJacksonModules() + { + return Arrays.asList(new SimpleModule("MovingAverageQueryModule") + .registerSubtypes(new NamedType(MovingAverageQuery.class, "movingAverage"))); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java new file mode 100644 index 000000000000..1bf421ee1899 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.QueryContexts; +import org.joda.time.Interval; +import org.joda.time.Period; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnionDataSource; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.server.QueryStats; +import org.apache.druid.server.RequestLogLine; +import org.apache.druid.server.log.RequestLogger; + +import javax.annotation.Nullable; + +/** + * The QueryRunner for MovingAverage query. + * High level flow: + * 1. Invokes an inner groupBy query (Or timeseries for no dimensions scenario) to get Aggregations/PostAggregtions. + * 2. Result is passed to {@link RowBucketIterable}, which groups rows of all dimension combinations into period-based (e.g. daily) buckets of rows ({@link RowBucket}). + * 3. The sequence is passed to {@link MovingAverageIterable}, which performs the main part of the query of adding Averagers computation into the records. + * 4. Finishes up by applying post averagers, removing redundant dates, and applying post phases (having, sorting, limits). + */ +public class MovingAverageQueryRunner implements QueryRunner +{ + + public static final String QUERY_FAIL_TIME = "queryFailTime"; + public static final String QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered"; + + private final QuerySegmentWalker walker; + private final RequestLogger requestLogger; + + public MovingAverageQueryRunner( + QueryToolChestWarehouse warehouse, + @Nullable QuerySegmentWalker walker, + RequestLogger requestLogger + ) + { + this.walker = walker; + this.requestLogger = requestLogger; + } + + @Override + public Sequence run(QueryPlus query, Map responseContext) + { + + MovingAverageQuery maq = (MovingAverageQuery) query.getQuery(); + List intervals; + final Period period; + + // Get the largest bucket from the list of averagers + Optional opt = + maq.getAveragerSpecs().stream().map(AveragerFactory::getNumBuckets).max(Integer::compare); + int buckets = opt.orElse(0); + + //Extend the interval beginning by specified bucket - 1 + if (maq.getGranularity() instanceof PeriodGranularity) { + period = ((PeriodGranularity) maq.getGranularity()).getPeriod(); + int offset = buckets <= 0 ? 0 : (1 - buckets); + intervals = maq.getIntervals() + .stream() + .map(i -> new Interval(i.getStart().withPeriodAdded(period, offset), i.getEnd())) + .collect(Collectors.toList()); + } else { + throw new ISE("Only PeriodGranulaity is supported for movingAverage queries"); + } + + Sequence resultsSeq; + DataSource dataSource = maq.getDataSource(); + if (maq.getDimensions() != null && !maq.getDimensions().isEmpty() && + (dataSource instanceof TableDataSource || dataSource instanceof UnionDataSource || + dataSource instanceof QueryDataSource)) { + // build groupBy query from movingAverage query + GroupByQuery.Builder builder = GroupByQuery.builder() + .setDataSource(dataSource) + .setInterval(intervals) + .setDimFilter(maq.getFilter()) + .setGranularity(maq.getGranularity()) + .setDimensions(maq.getDimensions()) + .setAggregatorSpecs(maq.getAggregatorSpecs()) + .setPostAggregatorSpecs(maq.getPostAggregatorSpecs()) + .setContext(maq.getContext()); + GroupByQuery gbq = builder.build(); + + HashMap gbqResponse = new HashMap<>(); + gbqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq)); + gbqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); + + Sequence results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponse); + try { + // use localhost for remote address + requestLogger.log(new RequestLogLine( + DateTimes.nowUtc(), + "127.0.0.1", + gbq, + new QueryStats( + ImmutableMap.of( + "query/time", 0, + "query/bytes", 0, + "success", true + )) + )); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + resultsSeq = results; + } else { + // no dimensions, so optimize this as a TimeSeries + TimeseriesQuery tsq = new TimeseriesQuery( + dataSource, + new MultipleIntervalSegmentSpec(intervals), + false, + null, + maq.getFilter(), + maq.getGranularity(), + maq.getAggregatorSpecs(), + maq.getPostAggregatorSpecs(), + 0, + maq.getContext() + ); + HashMap tsqResponse = new HashMap<>(); + tsqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(tsq)); + tsqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); + + Sequence> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponse); + try { + // use localhost for remote address + requestLogger.log(new RequestLogLine( + DateTimes.nowUtc(), + "127.0.0.1", + tsq, + new QueryStats( + ImmutableMap.of( + "query/time", 0, + "query/bytes", 0, + "success", true + )) + )); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + resultsSeq = Sequences.map(results, new TimeseriesResultToRow()); + } + + // Process into day buckets + Sequence bucketedMovingAvgResults = + Sequences.simple(new RowBucketIterable(resultsSeq, intervals, period)); + + // Apply the windows analysis functions + Sequence movingAvgResults = Sequences.simple( + new MovingAverageIterable( + bucketedMovingAvgResults, + maq.getDimensions(), + maq.getAveragerSpecs(), + maq.getPostAggregatorSpecs(), + maq.getAggregatorSpecs() + )); + + // Apply any postAveragers + Sequence movingAvgResultsWithPostAveragers = + Sequences.map(movingAvgResults, new PostAveragerCalculator(maq)); + + // remove rows outside the reporting window + List reportingIntervals = maq.getIntervals(); + movingAvgResults = + Sequences.filter( + movingAvgResultsWithPostAveragers, + row -> reportingIntervals.stream().anyMatch(i -> i.contains(row.getTimestamp())) + ); + + // Apply any having, sorting, and limits + movingAvgResults = ((MovingAverageQuery) maq).applyLimit(movingAvgResults); + + return movingAvgResults; + + } + + static class TimeseriesResultToRow implements Function, Row> + { + public Row apply(Result lookbackResult) + { + Map event = lookbackResult.getValue().getBaseObject(); + MapBasedRow row = new MapBasedRow(lookbackResult.getTimestamp(), event); + return row; + } + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java new file mode 100644 index 000000000000..e35b22df32d0 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Map; + +import javax.annotation.Nullable; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.collect.Maps; +import com.google.inject.Inject; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.server.log.RequestLogger; + +/** + * The QueryToolChest for MovingAverage Query + */ +public class MovingAverageQueryToolChest extends QueryToolChest +{ + + private final QuerySegmentWalker walker; + private final RequestLogger requestLogger; + private QueryToolChestWarehouse warehouse; + + public static final String MOVING_AVERAGE_MERGE_KEY = "movingAverageMerge"; + + private final MovingAverageQueryMetricsFactory movingAverageQueryMetricsFactory; + + /** + * Construct a MovingAverageQueryToolChest for processing moving-average queries. + * MovingAverage queries are expected to be processed on broker nodes and never hit historical nodes. + * + * @param walker + * @param requestLogger + */ + @Inject + public MovingAverageQueryToolChest(@Nullable QuerySegmentWalker walker, RequestLogger requestLogger) + { + + this.walker = walker; + this.requestLogger = requestLogger; + this.movingAverageQueryMetricsFactory = DefaultMovingAverageQueryMetricsFactory.instance(); + } + + @Inject(optional = true) + public void setWarehouse(QueryToolChestWarehouse warehouse) + { + this.warehouse = warehouse; + } + + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return new MovingAverageQueryRunner(warehouse, walker, requestLogger); + } + + @Override + public QueryMetrics makeMetrics(MovingAverageQuery query) + { + MovingAverageQueryMetrics movingAverageQueryMetrics = movingAverageQueryMetricsFactory.makeMetrics(); + movingAverageQueryMetrics.query(query); + return movingAverageQueryMetrics; + } + + @Override + public Function makePostComputeManipulatorFn(MovingAverageQuery query, MetricManipulationFn fn) + { + + return new Function() + { + + @Override + public Row apply(Row result) + { + MapBasedRow mRow = (MapBasedRow) result; + final Map values = Maps.newHashMap(mRow.getEvent()); + + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + Object aggVal = values.get(agg.getName()); + if (aggVal != null) { + values.put(agg.getName(), fn.manipulate(agg, aggVal)); + } else { + values.put(agg.getName(), null); + } + } + + for (AveragerFactory avg : query.getAveragerSpecs()) { + Object aggVal = values.get(avg.getName()); + if (aggVal != null) { + values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper(avg, avg.getName() + "_"), aggVal)); + } else { + values.put(avg.getName(), null); + } + } + + return new MapBasedRow(result.getTimestamp(), values); + + } + }; + + } + + + @Override + public TypeReference getResultTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public Function makePreComputeManipulatorFn(MovingAverageQuery query, MetricManipulationFn fn) + { + return Functions.identity(); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java new file mode 100644 index 000000000000..39e988b4bb76 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.List; +import java.util.Map; + +import com.google.common.base.Function; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.aggregation.PostAggregator; + +/** + * Function that can be applied to a Sequence to calculate PostAverager results + */ +public class PostAveragerCalculator implements Function +{ + + private final List postAveragers; + + public PostAveragerCalculator(MovingAverageQuery maq) + { + this.postAveragers = maq.getPostAveragerSpecs(); + } + + /* (non-Javadoc) + * @see com.google.common.base.Function#apply(java.lang.Object) + */ + @Override + public Row apply(Row input) + { + MapBasedRow row = (MapBasedRow) input; + Map event = row.getEvent(); + + for (PostAggregator postAverager : postAveragers) { + boolean allColsPresent = postAverager.getDependentFields().stream().allMatch(c -> event.get(c) != null); + event.put(postAverager.getName(), allColsPresent ? postAverager.compute(event) : null); + } + + return input; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java new file mode 100644 index 000000000000..8bf058fdf68f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.joda.time.DateTime; + +import java.util.Collection; + +import org.apache.druid.data.input.Row; + +/** + * Represents a set of rows for a specific date + */ +public class RowBucket +{ + private final DateTime dateTime; + private final Collection rows; + private RowBucket nextBucket = null; + + public RowBucket(DateTime dateTime, Collection rows) + { + this.dateTime = dateTime; + this.rows = rows; + } + + public DateTime getDateTime() + { + return dateTime; + } + + public Collection getRows() + { + return rows; + } + + public RowBucket getNextBucket() + { + return nextBucket; + } + + public void setNextBucket(RowBucket nextRow) + { + this.nextBucket = nextRow; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java new file mode 100644 index 000000000000..25dfe246ffca --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; + +/** + * It is the iterable used to bucket data into days, + * doing appropriate lookahead to see if the next row is in the same day or a new day. + */ +public class RowBucketIterable implements Iterable +{ + + public final Sequence seq; + private List intervals; + private Period period; + + public RowBucketIterable(Sequence seq, List intervals, Period period) + { + this.seq = seq; + this.period = period; + this.intervals = intervals; + } + + /* (non-Javadoc) + * @see java.lang.Iterable#iterator() + */ + @Override + public Iterator iterator() + { + return new RowIterator(seq, intervals, period); + } + + static class RowIterator implements Iterator + { + private Yielder yielder; + private boolean done = false; + private DateTime endTime; + private DateTime expectedBucket; + private Period period; + private int intervalIndex = 0; + private List intervals; + private boolean processedLastRow = false; + private boolean processedExtraRow = false; + + public RowIterator(Sequence rows, List intervals, Period period) + { + this.period = period; + this.intervals = intervals; + expectedBucket = intervals.get(intervalIndex).getStart(); + endTime = intervals.get(intervals.size() - 1).getEnd(); + yielder = rows.toYielder(null, new BucketingAccumulator()); + } + + /* (non-Javadoc) + * @see java.util.Iterator#hasNext() + */ + @Override + public boolean hasNext() + { + // expectedBucket < endTime + if (expectedBucket.compareTo(endTime) < 0) { + return true; + } + return false; + } + + /* (non-Javadoc) + * @see java.util.Iterator#next() + */ + @Override + public RowBucket next() + { + RowBucket currentBucket = yielder.get(); + + if (expectedBucket.compareTo(intervals.get(intervalIndex).getEnd()) >= 0) { + intervalIndex++; + if (intervalIndex <= intervals.size()) { + expectedBucket = intervals.get(intervalIndex).getStart(); + } + } + // currentBucket > expectedBucket + if (currentBucket != null && currentBucket.getDateTime().compareTo(expectedBucket) > 0) { + currentBucket = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } + + if (!yielder.isDone()) { + // standard case. return regular row + yielder = yielder.next(currentBucket); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else if (!processedLastRow && yielder.get() != null && yielder.get().getNextBucket() == null) { + // yielder.isDone, processing last row + processedLastRow = true; + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else if (!processedExtraRow && yielder.get() != null && yielder.get().getNextBucket() != null) { + RowBucket lastRow = yielder.get().getNextBucket(); + + if (lastRow.getDateTime().compareTo(expectedBucket) > 0) { + lastRow = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return lastRow; + } + + // yielder is done, processing newBucket + processedExtraRow = true; + expectedBucket = expectedBucket.plus(period); + return lastRow; + } else if (expectedBucket.compareTo(endTime) < 0) { + // add any trailing blank rows + currentBucket = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else { + // we should never get here + throw new NoSuchElementException(); + } + + } + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java new file mode 100644 index 000000000000..20da12ca3fca --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * Interface for an averager + * + * @param The return type of the averager + */ +public interface Averager +{ + /** + * Add a row to the window being operated on + * + * @param e The row to add + * @param aggMap The Map of AggregatorFactory used to determine if the metric should to be finalized + */ + void addElement(Map e, Map aggMap); + + /** + * There is a missing row, so record a missing entry in the window + */ + void skip(); + + /** + * Compute the resulting "average" over the collected window + * + * @return the "average" over the window of buckets + */ + R getResult(); + + /** + * @return the name + */ + String getName(); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java new file mode 100644 index 000000000000..e8f2ad4fe7d5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Interface representing Averager in the movingAverage query. + * + * @param Type returned by the underlying averager. + * @param Type of finalized value. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "constant", value = ConstantAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMean", value = DoubleMeanAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMeanNoNulls", value = DoubleMeanNoNullAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMax", value = DoubleMaxAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMin", value = DoubleMinAveragerFactory.class), + @JsonSubTypes.Type(name = "longMean", value = LongMeanAveragerFactory.class), + @JsonSubTypes.Type(name = "longMeanNoNulls", value = LongMeanNoNullAveragerFactory.class), + @JsonSubTypes.Type(name = "longMax", value = LongMaxAveragerFactory.class), + @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class), + @JsonSubTypes.Type(name = "sketchUnion", value = SketchUnionAveragerFactory.class) +}) +public interface AveragerFactory +{ + int DEFAULT_PERIOD = 1; + + /** + * Gets the column name that will be populated by the Averager + * + * @return The column name + */ + String getName(); + + /** + * Returns the window size over which the averaging calculations will be + * performed. Size is computed in terms of buckets rather than absolute time. + * + * @return The window size + */ + int getNumBuckets(); + + /** + * Returns the cycle size (number of periods to skip during averaging calculations). + * + * @return The cycle size + */ + int getCycleSize(); + + /** + * Create an Averager for a specific dimension combination. + * + * @return The {@link Averager} + */ + Averager createAverager(); + + /** + * Gets the list of dependent fields that will be used by this Averager. Most + * {@link Averager}s depend on only a single field from the underlying query, but + * that is not required. This method allow the required fields to be communicated + * back to the main query so that validation to enforce the fields presence can + * be accomplished. + * + * @return A list of field names + */ + List getDependentFields(); + + /** + * Returns a {@link Comparator} that can be used to compare result values for + * purposes of sorting the end result of the query. + * + * @return A {@link Comparator} + */ + Comparator getComparator(); + + /** + * Finalize result value. + * + * @param val the value to finalize. + * + * @return The finalized value. + */ + F finalizeComputation(R val); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java new file mode 100644 index 000000000000..e525deb223e4 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.lang.reflect.Array; +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * Common base class available for use by averagers. The base class implements methods that + * capture incoming and skipped rows and store them in an array, to be used later for + * calculating the actual value. + * + * @param The type of intermediate value to be retrieved from the row and stored + * @param The type of result the averager is expected to produce + */ +public abstract class BaseAverager implements Averager +{ + + private final int numBuckets; + private final int cycleSize; + private final String name; + private final String fieldName; + private final I[] buckets; + private int index; + + /* This flag checks if the intermediate sketches have to be finalized or not + for further computation involving sketches */ + private boolean shouldFinalize; + + /** + * @param storageType The class to use for storing intermediate values + * @param numBuckets The number of buckets to include in the window being aggregated + * @param name The name of the resulting metric + * @param fieldName The field to extra from incoming rows and stored in the window cache + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + * @param shouldFinalize Whether the intermediate values need to be finalized prior to averaging + */ + public BaseAverager( + Class storageType, int numBuckets, String name, String fieldName, int cycleSize, + boolean shouldFinalize + ) + { + this.numBuckets = numBuckets; + this.name = name; + this.fieldName = fieldName; + this.index = 0; + @SuppressWarnings("unchecked") + final I[] array = (I[]) Array.newInstance(storageType, numBuckets); + this.buckets = array; + this.shouldFinalize = shouldFinalize; + this.cycleSize = cycleSize; + } + + public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize) + { + this(storageType, numBuckets, name, fieldName, cycleSize, true); + } + + + /* (non-Javadoc) + * @see Averager#addElement(java.util.Map, java.util.Map) + */ + @SuppressWarnings("unchecked") + @Override + public void addElement(Map e, Map a) + { + Object metric = e.get(fieldName); + I finalMetric; + if (a.containsKey(fieldName) && shouldFinalize) { + AggregatorFactory af = a.get(fieldName); + finalMetric = metric != null ? (I) af.finalizeComputation(metric) : null; + } else { + finalMetric = (I) metric; + } + buckets[index++] = finalMetric; + index %= numBuckets; + } + + /* (non-Javadoc) + * @see Averager#skip() + */ + @Override + public void skip() + { + buckets[index++] = null; + index %= numBuckets; + } + + /* (non-Javadoc) + * @see Averager#getResult() + */ + @Override + public R getResult() + { + if (!hasData()) { + return null; + } + return computeResult(); + } + + /** + * Compute the result value to be returned by getResult. + * + *

This routine will only be called when there is valid data within the window + * and doesn't need to worry about detecting the case where no data should be returned. + * + *

+ * The method typically should use {@link #getBuckets()} to retrieve the set of buckets + * within the window and then compute a value based on those. It should expect nulls within + * the array, indicating buckets where no row was found for the dimension combination. It is + * up to the actual implementation to determin how to evaluate those nulls. + * + *

+ * The type returned is NOT required to be the same type as the intermediary value. For example, + * the intermediate value could be a Sketch, but the result a Long. + * + * @return the computed result + */ + protected abstract R computeResult(); + + /* (non-Javadoc) + * @see Averager#getName() + */ + @Override + public String getName() + { + return name; + } + + /** + * Returns the fieldname to be extracted from any event rows passed in and stored + * for use computing the windowed function. + * + * @return the fieldName + */ + public String getFieldName() + { + return fieldName; + } + + /** + * @return the numBuckets + */ + public int getNumBuckets() + { + return numBuckets; + } + + /** + * @return the cycleSize + */ + public int getCycleSize() + { + return cycleSize; + } + + /** + * @return the array of buckets + */ + protected I[] getBuckets() + { + return (I[]) buckets; + } + + /** + * Determines wheter any data is present. If all the buckets are empty (not "0"), then + * no value should be returned from the Averager, as there were not valid rows within the window. + * + * @return true if any non-null values available + */ + protected boolean hasData() + { + for (Object b : buckets) { + if (b != null) { + return true; + } + } + return false; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java new file mode 100644 index 000000000000..d91e74f687b7 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Collections; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +/** + * Common base class for AveragerFactories + * + * @param Base type that the averager should return as a result + * @param Type that that is returned from finalization + */ +public abstract class BaseAveragerFactory implements AveragerFactory +{ + + protected String name; + protected String fieldName; + protected int numBuckets; + protected int cycleSize; + + /** + * Constructor. + * + * @param name Name of the Averager + * @param numBuckets Number of buckets in the analysis window + * @param fieldName Field from incoming events to include in the analysis + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + */ + public BaseAveragerFactory(String name, int numBuckets, String fieldName, Integer cycleSize) + { + this.name = name; + this.numBuckets = numBuckets; + this.fieldName = fieldName; + this.cycleSize = (cycleSize != null) ? cycleSize : DEFAULT_PERIOD; + Preconditions.checkNotNull(name, "Must have a valid, non-null averager name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null field name"); + Preconditions.checkArgument(this.cycleSize > 0, "Cycle size must be greater than zero"); + Preconditions.checkArgument(numBuckets > 0, "Bucket size must be greater than zero"); + Preconditions.checkArgument(this.cycleSize < numBuckets, "Cycle size must be less than the bucket size"); + Preconditions.checkArgument(numBuckets % this.cycleSize == 0, "cycleSize must devide numBuckets without a remainder"); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty("buckets") + public int getNumBuckets() + { + return numBuckets; + } + + @Override + @JsonProperty("cycleSize") + public int getCycleSize() + { + return cycleSize; + } + + @Override + public List getDependentFields() + { + return Collections.singletonList(fieldName); + } + + @SuppressWarnings("unchecked") + @Override + public F finalizeComputation(R val) + { + return (F) val; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java new file mode 100644 index 000000000000..ca76b505ae45 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ +/** + * Copyright 2018 Oath Inc. All rights reserved. + */ +package org.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; + +/** + * Base averager factory that adds a default comparable method. + * + * @param return type + * @param finalized type + */ +public abstract class ComparableAveragerFactory, F> extends BaseAveragerFactory +{ + /** + * Constructor. + * + * @param name Name of the Averager + * @param numBuckets Number of buckets in the analysis window + * @param fieldName Field from incoming events to include in the analysis + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + */ + public ComparableAveragerFactory(String name, int numBuckets, String fieldName, Integer cycleSize) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Comparator getComparator() + { + return Comparator.naturalOrder(); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java new file mode 100644 index 000000000000..806fa6b21242 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * The constant averager.Created soley for incremental development and wiring things up. + */ +public class ConstantAverager implements Averager +{ + + private String name; + private float retval; + + /** + * @param n + * @param name + * @param retval + */ + public ConstantAverager(int n, String name, float retval) + { + this.name = name; + this.retval = retval; + } + + /* (non-Javadoc) + * @see Averager#getResult() + */ + @Override + public Float getResult() + { + return retval; + } + + /* (non-Javadoc) + * @see Averager#getName() + */ + @Override + public String getName() + { + return name; + } + + /* (non-Javadoc) + * @see Averager#addElement(java.util.Map, java.util.Map) + */ + @Override + public void addElement(Map e, Map a) + { + // since we return a constant, no need to read from the event + } + + /* (non-Javadoc) + * @see Averager#skip() + */ + @Override + public void skip() + { + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java new file mode 100644 index 000000000000..cffc876fa399 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Implementation of AveragerFacvtory created solely for incremental development + */ + +public class ConstantAveragerFactory implements AveragerFactory +{ + + private String name; + private int numBuckets; + private float retval; + + @JsonCreator + public ConstantAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("retval") float retval + ) + { + this.name = name; + this.numBuckets = numBuckets; + this.retval = retval; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + @JsonProperty("buckets") + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public float getRetval() + { + return retval; + } + + @Override + public Averager createAverager() + { + return new ConstantAverager(numBuckets, name, retval); + } + + @Override + public List getDependentFields() + { + return Collections.emptyList(); + } + + @Override + public Comparator getComparator() + { + return Comparator.naturalOrder(); + } + + @Override + public int getCycleSize() + { + return 1; + } + + @Override + public Float finalizeComputation(Float val) + { + return val; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java new file mode 100644 index 000000000000..c9db3f2cd069 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMaxAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = Double.NEGATIVE_INFINITY; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Double.max(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java new file mode 100644 index 000000000000..9c708428fff9 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMaxAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public DoubleMaxAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMaxAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java new file mode 100644 index 000000000000..ece43f105992 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMeanAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = 0.0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + } else { + result += 0.0; + } + validBuckets++; + } + + startFrom++; + return result / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java new file mode 100644 index 000000000000..233b7c521bad --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMeanAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public DoubleMeanAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMeanAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java new file mode 100644 index 000000000000..5f98c25ea9bd --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMeanNoNullAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = 0.0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + validBuckets++; + } + } + + startFrom++; + return result / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java new file mode 100644 index 000000000000..9abf4fcce7d1 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMeanNoNullAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public DoubleMeanNoNullAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMeanNoNullAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java new file mode 100644 index 000000000000..0ded39e82f23 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMinAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMinAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = Double.POSITIVE_INFINITY; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Double.min(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java new file mode 100644 index 000000000000..d3d81f1be13b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMinAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public DoubleMinAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMinAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java new file mode 100644 index 000000000000..343a1c8f9e3a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMaxAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Long computeResult() + { + long result = Long.MIN_VALUE; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Long.max(result, (obj[(i + startFrom) % numBuckets]).longValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java new file mode 100644 index 000000000000..473026fd7180 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMaxAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public LongMaxAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMaxAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java new file mode 100644 index 000000000000..8c6d97d53863 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMeanAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + long result = 0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).longValue(); + } else { + result += 0; + } + validBuckets++; + } + + startFrom++; + return ((double) result) / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java new file mode 100644 index 000000000000..2c413b976a75 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMeanAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMeanAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMeanAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java new file mode 100644 index 000000000000..5cf6548e3605 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMeanNoNullAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + long result = 0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).longValue(); + validBuckets++; + } + } + + startFrom++; + return ((double) result) / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java new file mode 100644 index 000000000000..5761e61ae4d1 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMeanNoNullAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMeanNoNullAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMeanNoNullAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java new file mode 100644 index 000000000000..0a275988395d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMinAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMinAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Long computeResult() + { + long result = Long.MAX_VALUE; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Long.min(result, (obj[(i + startFrom) % numBuckets]).longValue()); + } + } + + startFrom++; + return result; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java new file mode 100644 index 000000000000..2d9f342414ea --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMinAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMinAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") int cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMinAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java new file mode 100644 index 000000000000..0756204fa5e6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.yahoo.sketches.Family; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.SetOperationBuilder; +import com.yahoo.sketches.theta.Union; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; + +public class SketchUnionAverager extends BaseAverager +{ + + private int startFrom = 0; + private final SetOperationBuilder builder; + + public SketchUnionAverager(int numBuckets, String name, String fieldName, int cycleSize, int size) + { + super(SketchHolder.class, numBuckets, name, fieldName, cycleSize, false); + this.builder = SetOperation.builder().setNominalEntries(size); + } + + @Override + protected SketchHolder computeResult() + { + int cycleSize = getCycleSize(); + Object[] obj = getBuckets(); + int numBuckets = getNumBuckets(); + + Union union = (Union) builder.build(Family.UNION); + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + ((SketchHolder) obj[(i + startFrom) % numBuckets]).updateUnion(union); + } + } + + startFrom++; + return SketchHolder.of(union.getResult()); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java new file mode 100644 index 000000000000..5a67de968365 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.yahoo.sketches.Util; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; + +public class SketchUnionAveragerFactory extends BaseAveragerFactory +{ + + private final int size; + + @JsonCreator + public SketchUnionAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size + ) + { + super(name, numBuckets, fieldName, cycleSize); + this.size = size == null ? Util.DEFAULT_NOMINAL_ENTRIES : size; + } + + public SketchUnionAveragerFactory( + String name, + int numBuckets, + Integer cycleSize, + String fieldName + ) + { + this(name, numBuckets, cycleSize, fieldName, null); + } + + @Override + public Averager createAverager() + { + return new SketchUnionAverager(numBuckets, name, fieldName, cycleSize, size); + } + + /* (non-Javadoc) + * @see AveragerFactory#finalize(java.lang.Object) + */ + @Override + public Double finalizeComputation(SketchHolder val) + { + return val.getSketch().getEstimate(); + } + + /* (non-Javadoc) + * @see BaseAveragerFactory#getComparator() + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + @Override + public Comparator getComparator() + { + return SketchHolder.COMPARATOR; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..72876559838d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1 @@ +org.apache.druid.query.movingaverage.MovingAverageQueryModule diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java new file mode 100644 index 000000000000..bc052b3b9854 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.junit.Test; + +import java.util.Arrays; + +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test IdentityYieldingAccumulator + */ +public class IdentityYieldingAccumulatorTest +{ + @Test + public void testAccumulator() + { + Sequence seq = Sequences.simple(Arrays.asList(1, 2, 3, 4, 5)); + + Yielder y = seq.toYielder(null, new IdentityYieldingAccumulator<>()); + + assertEquals(Integer.valueOf(1), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(2), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(3), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(4), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(5), y.get()); + y = y.next(null); + assertTrue(y.isDone()); + + assertNull(y.get()); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java new file mode 100644 index 000000000000..9e89b69bfb18 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -0,0 +1,802 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; +import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; + +import org.joda.time.DateTime; +import org.joda.time.chrono.ISOChronology; +import org.junit.Test; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * + */ +public class MovingAverageIterableTest +{ + private static final DateTime JAN_1 = new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_2 = new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_3 = new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + + private static final String GENDER = "gender"; + private static final String AGE = "age"; + private static final String COUNTRY = "country"; + + private static final Map dims1 = new HashMap<>(); + private static final Map dims2 = new HashMap<>(); + private static final Map dims3 = new HashMap<>(); + + static { + dims1.put(GENDER, "m"); + dims1.put(AGE, "10"); + dims1.put(COUNTRY, "US"); + + dims2.put(GENDER, "f"); + dims2.put(AGE, "8"); + dims2.put(COUNTRY, "US"); + + dims3.put(GENDER, "u"); + dims3.put(AGE, "5"); + dims3.put(COUNTRY, "UK"); + } + + @Test + public void testNext() + { + + Collection dims = Arrays.asList( + new DefaultDimensionSpec(GENDER, GENDER), + new DefaultDimensionSpec(AGE, AGE), + new DefaultDimensionSpec(COUNTRY, COUNTRY) + ); + + Sequence dayBuckets = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList( + new MapBasedRow(JAN_1, dims1), + new MapBasedRow(JAN_1, dims2) + )), + new RowBucket(JAN_2, Arrays.asList( + new MapBasedRow(JAN_2, dims1) + )), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Arrays.asList( + new MapBasedRow(JAN_4, dims2), + new MapBasedRow(JAN_4, dims3) + )) + )); + + Iterable iterable = new MovingAverageIterable( + dayBuckets, + dims, + Collections.singletonList(new ConstantAveragerFactory("noop", 1, 1.1f)), + Collections.emptyList(), + Collections.emptyList() + ); + + Iterator iter = iterable.iterator(); + + assertTrue(iter.hasNext()); + Row r = iter.next(); + assertEquals(JAN_1, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_1, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_2, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_2, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + Row r2 = r; + assertEquals(JAN_3, r.getTimestamp()); + assertEquals("US", r.getRaw(COUNTRY)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_3, r.getTimestamp()); + assertEquals("US", r.getRaw(COUNTRY)); + assertThat(r.getRaw(AGE), not(equalTo(r2.getRaw(AGE)))); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("u", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertFalse(iter.hasNext()); + } + + @Test + public void testAveraging() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + Map event4 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_2, event2); + + event3.put("gender", "m"); + event3.put("pageViews", 30L); + Row row3 = new MapBasedRow(JAN_3, event3); + + event4.put("gender", "f"); + event4.put("pageViews", 40L); + Row row4 = new MapBasedRow(JAN_2, event4); + + float retval = 14.5f; + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.singletonList(row2)), + new RowBucket(JAN_3, Arrays.asList(row3, row4)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new ConstantAveragerFactory("costPageViews", 7, retval), + new LongMeanAveragerFactory("movingAvgPageViews", 7, 1, "pageViews") + ), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row caResult = iter.next(); + + assertEquals(JAN_1, caResult.getTimestamp()); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(retval, caResult.getMetric("costPageViews").floatValue(), 0.0f); + assertEquals(1.4285715f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(4.285714f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(8.571428f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("f", (caResult.getDimension("gender")).get(0)); + assertEquals(5.714285850f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + + } + + + @Test + public void testCompleteData() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_2_row2 = new MapBasedRow(JAN_2, event2); + Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + + } + + // no injection if the data missing at the begining + @Test + public void testMissingDataAtBeginning() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_2_row2 = new MapBasedRow(JAN_2, event2); + Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + // test injection when the data is missing at the end + @Test + public void testMissingDataAtTheEnd() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + // test injection when the data is missing in the middle + @Test + public void testMissingDataAtMiddle() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + Map event4 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_3_row1 = new MapBasedRow(JAN_3, event1); + Row jan_3_row2 = new MapBasedRow(JAN_3, event2); + Row jan_3_row3 = new MapBasedRow(JAN_3, event3); + Row jan_4_row1 = new MapBasedRow(JAN_4, event1); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1)), + new RowBucket(JAN_3, Arrays.asList(jan_3_row1, jan_3_row2, jan_3_row3)), + new RowBucket(JAN_4, Arrays.asList(jan_4_row1)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 3, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysAtBegining() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_3, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.emptyList()), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.singletonList(row1)), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysInMiddle() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testWithFilteredAggregation() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + AveragerFactory averagerfactory = new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews"); + AggregatorFactory aggregatorFactory = new LongSumAggregatorFactory("pageViews", "pageViews"); + DimFilter filter = new SelectorDimFilter("gender", "m", null); + FilteredAggregatorFactory filteredAggregatorFactory = new FilteredAggregatorFactory(aggregatorFactory, filter); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + averagerfactory), + Collections.emptyList(), + Collections.singletonList( + filteredAggregatorFactory) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysAtEnd() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_2, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.singletonList(row2)), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.emptyList()), + new RowBucket(JAN_5, Collections.emptyList()), + new RowBucket(JAN_6, Collections.emptyList()) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + + assertEquals(JAN_1, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_2, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_3, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_4, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_5, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(5.0f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_6, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(0.0f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java new file mode 100644 index 000000000000..aaadd2344ec8 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -0,0 +1,442 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.google.inject.util.Providers; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.ForegroundCachePopulator; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.guice.DruidProcessingModule; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.RetryQueryRunnerConfig; +import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.timeline.TimelineLookup; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import org.apache.druid.query.movingaverage.test.TestConfig; +import org.apache.druid.client.CachingClusteredClient; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.initialization.Initialization; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; + +import mockit.Mock; +import mockit.MockUp; + +import org.apache.druid.java.util.emitter.service.ServiceEmitter; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; + +/** + * Base class for implementing MovingAverageQuery tests + */ +@RunWith(Parameterized.class) +public class MovingAverageQueryTest +{ + private final ObjectMapper jsonMapper; + private final Injector injector; + private final QueryToolChestWarehouse warehouse; + private final RetryQueryRunnerConfig retryConfig; + private final ServerConfig serverConfig; + + private final List groupByResults = new ArrayList<>(); + private final List> timeseriesResults = new ArrayList<>(); + + private final TestConfig config; + + @Parameters(name = "{0}") + public static Iterable data() throws IOException + { + BufferedReader testReader = new BufferedReader( + new InputStreamReader(MovingAverageQueryTest.class.getResourceAsStream("/queryTests"), StandardCharsets.UTF_8)); + List tests = new ArrayList<>(); + + for (String line = testReader.readLine(); line != null; line = testReader.readLine()) { + tests.add(new String[] {line}); + } + + return tests; + } + + public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMappingException, IOException + { + List modules = getRequiredModules(); + modules.add( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("queryTest"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(1); + binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); + } + ); + + + Injector baseInjector = GuiceInjectors.makeStartupInjector(); + injector = Initialization.makeInjectorWithModules(baseInjector, modules); + + jsonMapper = injector.getInstance(ObjectMapper.class); + warehouse = injector.getInstance(QueryToolChestWarehouse.class); + retryConfig = injector.getInstance(RetryQueryRunnerConfig.class); + serverConfig = injector.getInstance(ServerConfig.class); + + InputStream is = getClass().getResourceAsStream("/queryTests/" + yamlFile); + ObjectMapper reader = new ObjectMapper(new YAMLFactory()); + config = reader.readValue(is, TestConfig.class); + } + + /** + * Returns the JSON query that should be used in the test. + * + * @return The JSON query + */ + protected String getQueryString() + { + return config.query.toString(); + } + + /** + * Returns the JSON result that should be expected from the query. + * + * @return The JSON result + */ + protected String getExpectedResultString() + { + return config.expectedOutput.toString(); + } + + /** + * Returns the JSON result that the nested groupby query should produce. + * Either this method or {@link #getTimeseriesResultJson()} must be defined + * by the subclass. + * + * @return The JSON result from the groupby query + */ + protected String getGroupByResultJson() + { + ArrayNode node = config.intermediateResults.get("groupBy"); + return node == null ? null : node.toString(); + } + + /** + * Returns the JSON result that the nested timeseries query should produce. + * Either this method or {@link #getGroupByResultJson()} must be defined + * by the subclass. + * + * @return The JSON result from the timeseries query + */ + protected String getTimeseriesResultJson() + { + ArrayNode node = config.intermediateResults.get("timeseries"); + return node == null ? null : node.toString(); + } + + /** + * Returns the expected query type. + * + * @return The Query type + */ + protected Class getExpectedQueryType() + { + return MovingAverageQuery.class; + } + + protected TypeReference getExpectedResultType() + { + return new TypeReference>() + { + }; + } + + /** + * Returns a list of any additional Druid Modules necessary to run the test. + * + * @return List of Druid Modules + */ + protected List getRequiredModules() + { + List list = new ArrayList<>(); + + list.add(new SketchModule()); + list.add(new QueryRunnerFactoryModule()); + list.add(new QueryableModule()); + list.add(new DruidProcessingModule()); + + return list; + } + + /** + * Set up any needed mocks to stub out backend query behavior. + * + * @param query + * + * @throws IOException + * @throws JsonMappingException + * @throws JsonParseException + */ + protected void defineMocks(Query query) throws JsonParseException, JsonMappingException, IOException + { + groupByResults.clear(); + timeseriesResults.clear(); + List aggs; + + if (query instanceof GroupByQuery) { + aggs = ((GroupByQuery) query).getAggregatorSpecs(); + } else if (query instanceof TimeseriesQuery) { + aggs = ((TimeseriesQuery) query).getAggregatorSpecs(); + } else if (query instanceof MovingAverageQuery) { + aggs = ((MovingAverageQuery) query).getAggregatorSpecs(); + } else { + // unrecognized query type + aggs = Collections.emptyList(); + + } + + if (getGroupByResultJson() != null) { + groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference>() + { + })); + for (Row r : groupByResults) { + Map map = ((MapBasedRow) r).getEvent(); + for (AggregatorFactory agg : aggs) { + Object serializedVal = map.get(agg.getName()); + if (serializedVal != null) { + map.put(agg.getName(), agg.deserialize(serializedVal)); + } + } + } + } + + if (getTimeseriesResultJson() != null) { + timeseriesResults.addAll(jsonMapper.readValue( + getTimeseriesResultJson(), + new TypeReference>>() + { + } + )); + for (Result r : timeseriesResults) { + Map map = r.getValue().getBaseObject(); + for (AggregatorFactory agg : aggs) { + Object serializedVal = map.get(agg.getName()); + if (serializedVal != null) { + map.put(agg.getName(), agg.deserialize(serializedVal)); + } + } + } + } + } + + /** + * converts Int to Long, Float to Double in the actual and expected result + * + * @param result + */ + protected void consistentTypeCasting(List result) + { + for (MapBasedRow row : result) { + Map event = row.getEvent(); + event.forEach((key, value) -> { + if (Integer.class.isInstance(value)) { + event.put(key, ((Integer) value).longValue()); + } + if (Float.class.isInstance(value)) { + event.put(key, ((Float) value).doubleValue()); + } + }); + + } + } + + /** + * Validate that the specified query behaves correctly. + * + * @throws IOException + * @throws JsonMappingException + * @throws JsonParseException + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + @Test + public void testQuery() throws JsonParseException, JsonMappingException, IOException + { + + + // create mocks for nested queries + @SuppressWarnings("unused") + + MockUp groupByQuery = new MockUp() + { + @Mock + public QueryRunner getRunner(QuerySegmentWalker walker) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + return Sequences.simple(groupByResults); + } + }; + } + }; + + + @SuppressWarnings("unused") + MockUp timeseriesQuery = new MockUp() + { + @Mock + public QueryRunner getRunner(QuerySegmentWalker walker) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + return Sequences.simple(timeseriesResults); + } + }; + } + }; + + + Query query = jsonMapper.readValue(getQueryString(), Query.class); + assertThat(query, IsInstanceOf.instanceOf(getExpectedQueryType())); + + List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); + assertNotNull(expectedResults); + assertThat(expectedResults, IsInstanceOf.instanceOf(List.class)); + + CachingClusteredClient baseClient = new CachingClusteredClient( + warehouse, + new TimelineServerView() + { + @Override + public TimelineLookup getTimeline(DataSource dataSource) + { + return null; + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + return null; + } + + @Override + public void registerTimelineCallback(Executor exec, TimelineCallback callback) + { + + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + + } + + @Override + public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback) + { + + } + }, + MapCache.create(100000), + jsonMapper, + new ForegroundCachePopulator(jsonMapper, new CachePopulatorStats(), -1), + new CacheConfig(), + new DruidHttpClientConfig() + { + @Override + public long getMaxQueuedBytes() + { + return 0L; + } + } + ); + + ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( + new ServiceEmitter("", "", null) + { + public void emit(org.apache.druid.java.util.emitter.core.Event event) {} + }, + baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() + ); + final Map responseContext = new ConcurrentHashMap<>(); + + defineMocks(query); + + QueryPlus queryPlus = QueryPlus.wrap(query); + final Sequence res = query.getRunner(walker).run(queryPlus, responseContext); + + List actualResults = new ArrayList(); + actualResults = (List) res.accumulate(actualResults, Accumulators.list()); + + consistentTypeCasting(expectedResults); + consistentTypeCasting(actualResults); + + assertEquals(expectedResults, actualResults); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java new file mode 100644 index 000000000000..dbd127fdf7e6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.movingaverage.averagers.DoubleMeanAveragerFactory; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + + +/** + * Unit tests for PostAveragerCalcualtor + */ +public class PostAveragerCalculatorTest +{ + + private MovingAverageQuery query; + private PostAveragerCalculator pac; + private Map event; + private MapBasedRow row; + + @Before + public void setup() + { + query = new MovingAverageQuery( + new TableDataSource("d"), + new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval("2017-01-01/2017-01-01", ISOChronology.getInstanceUTC()))), + null, + Granularities.DAY, + null, + Collections.singletonList(new CountAggregatorFactory("count")), + Collections.emptyList(), + null, + Collections.singletonList(new DoubleMeanAveragerFactory("avgCount", 7, 1, "count")), + Collections.singletonList(new ArithmeticPostAggregator( + "avgCountRatio", + "/", + Arrays.asList( + new FieldAccessPostAggregator("count", "count"), + new FieldAccessPostAggregator("avgCount", "avgCount") + ) + )), + null, + null + ); + + pac = new PostAveragerCalculator(query); + event = new HashMap<>(); + row = new MapBasedRow(new DateTime(ISOChronology.getInstanceUTC()), event); + } + + @Test + public void testApply() + { + event.put("count", new Double(10.0)); + event.put("avgCount", new Double(12.0)); + + Row result = pac.apply(row); + + assertEquals(result.getMetric("avgCountRatio").floatValue(), 10.0f / 12.0f, 0.0); + } + + @Test + public void testApplyMissingColumn() + { + event.put("count", new Double(10.0)); + + Row result = pac.apply(row); + + assertEquals(result.getMetric("avgCountRatio").floatValue(), 0.0, 0.0); + assertNull(result.getRaw("avgCountRatio")); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java new file mode 100644 index 000000000000..912b91ad0294 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -0,0 +1,672 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.joda.time.chrono.ISOChronology; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class RowBucketIterableTest +{ + + private static final DateTime JAN_1 = new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_2 = new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_3 = new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_8 = new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_9 = new DateTime(2017, 1, 9, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + + private static final Map EVENT_M_10 = new HashMap<>(); + private static final Map EVENT_F_20 = new HashMap<>(); + private static final Map EVENT_U_30 = new HashMap<>(); + + private static final Row JAN_1_M_10 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_1_F_20 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_1_U_30 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + private static final Row JAN_2_M_10 = new MapBasedRow(new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_3_M_10 = new MapBasedRow(new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_3_F_20 = new MapBasedRow(new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_4_M_10 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_4_F_20 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_4_U_30 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + private static final Row JAN_5_M_10 = new MapBasedRow(new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_6_M_10 = new MapBasedRow(new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_7_F_20 = new MapBasedRow(new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_8_U_30 = new MapBasedRow(new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + + private static final Interval INTERVAL_JAN_1_1 = new Interval(JAN_1, JAN_2); + private static final Interval INTERVAL_JAN_1_2 = new Interval(JAN_1, JAN_3); + private static final Interval INTERVAL_JAN_1_4 = new Interval(JAN_1, JAN_5); + private static final Interval INTERVAL_JAN_1_5 = new Interval(JAN_1, JAN_6); + private static final Interval INTERVAL_JAN_6_8 = new Interval(JAN_6, JAN_9); + private static final Period ONE_DAY = Period.days(1); + + private List rows = null; + private List intervals = new ArrayList<>(); + + @BeforeClass + public static void setupClass() + { + EVENT_M_10.put("gender", "m"); + EVENT_M_10.put("pageViews", 10L); + EVENT_F_20.put("gender", "f"); + EVENT_F_20.put("pageViews", 20L); + EVENT_U_30.put("gender", "u"); + EVENT_U_30.put("pageViews", 30L); + } + + // normal case. data for all the days present + @Test + public void testCompleteData() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // all days present and last day only has one row + @Test + public void testApplyLastDaySingleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + } + + // all days present and last day has multiple rows + @Test + public void testApplyLastDayMultipleRows() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + rows.add(JAN_4_F_20); + rows.add(JAN_4_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + } + + // test single day with single row + @Test + public void testSingleDaySingleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_1); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + + List expected_day1 = Arrays.asList(JAN_1_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + assertEquals(JAN_1, actual.getDateTime()); + + } + + // test single day with multiple rows + @Test + public void testSingleDayMultipleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_1); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_1_U_30); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20, JAN_1_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + } + + // missing day at the beginning followed by single row + @Test + public void testMissingDaysAtBegining() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_2); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + } + + // missing day at the beginning followed by multiple row + @Test + public void testMissingDaysAtBeginingFollowedByMultipleRow() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // missing day at the beginning and at the end + @Test + public void testMissingDaysAtBeginingAndAtTheEnd() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Collections.emptyList(); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // multiple missing days in an interval + @Test + public void testMultipleMissingDays() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // multiple missing days in an interval followed by multiple row at the end + @Test + public void testMultipleMissingDaysMultipleRowAtTheEnd() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day5 = Arrays.asList(JAN_5_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_5); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + rows.add(JAN_5_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_5, actual.getDateTime()); + assertEquals(expected_day5, actual.getRows()); + } + + + // missing day in the middle followed by single row + @Test + public void testMissingDaysInMiddleOneRow() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + + } + + // missing day in the middle followed by multiple rows + @Test + public void testMissingDaysInMiddleMultipleRow() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Collections.emptyList(); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + + } + + // data missing for last day . + @Test + public void testApplyLastDayNoRows() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Collections.emptyList(); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // data missing for last two days + @Test + public void testApplyLastTwoDayNoRows() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Collections.emptyList(); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + + @Test + public void testApplyMultipleInterval() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + intervals.add(INTERVAL_JAN_6_8); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + List expected_day6 = Arrays.asList(JAN_6_M_10); + List expected_day7 = Arrays.asList(JAN_7_F_20); + List expected_day8 = Arrays.asList(JAN_8_U_30); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + rows.add(JAN_4_F_20); + rows.add(JAN_4_U_30); + rows.add(JAN_6_M_10); + rows.add(JAN_7_F_20); + rows.add(JAN_8_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day6, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day7, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day8, actual.getRows()); + } + + @Test + public void testNodata() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + intervals.add(INTERVAL_JAN_6_8); + + rows = new ArrayList(); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + assertTrue(iter.hasNext()); + RowBucket actual = iter.next(); + assertEquals(Collections.emptyList(), actual.getRows()); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java new file mode 100644 index 000000000000..8b16d29dcc44 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Comparator; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class BaseAveragerFactoryTest +{ + + private AveragerFactory fac; + + @Before + public void setup() + { + fac = new BaseAveragerFactory("test", 5, "field", 1) + { + @Override + public Averager createAverager() + { + return null; + } + + @Override + public Comparator getComparator() + { + return null; + } + }; + } + + @Test + public void testGetDependentFields() throws Exception + { + List dependentFields = fac.getDependentFields(); + assertEquals(1, dependentFields.size()); + assertEquals("field", dependentFields.get(0)); + } + + @Test + public void testFinalization() + { + Long input = Long.valueOf(5L); + assertEquals(input, fac.finalizeComputation(input)); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java new file mode 100644 index 000000000000..302cb4c0b25e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test class for BaseAverager + */ +public class BaseAveragerTest +{ + + public static class TestAverager extends BaseAverager + { + public TestAverager(Class clazz, int b, String name, String field, int cycleSize) + { + super(clazz, b, name, field, cycleSize); + } + + @Override + protected Integer computeResult() + { + return 1; + } + } + + @Test + public void testBaseAverager() + { + BaseAverager avg = new TestAverager(Integer.class, 5, "test", "field", 1); + + assertEquals("test", avg.getName()); + assertEquals(5, avg.getNumBuckets()); + assertEquals(5, avg.getBuckets().length); + assertTrue(avg.getBuckets().getClass().isArray()); + } + + @Test + public void testAddElement() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + Object[] buckets = avg.getBuckets(); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + + avg.addElement(Collections.singletonMap("field", 2), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertNull(buckets[2]); + + avg.addElement(Collections.singletonMap("field", 3), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertEquals(Integer.valueOf(3), buckets[2]); + + avg.addElement(Collections.singletonMap("field", 4), Collections.emptyMap()); + assertEquals(Integer.valueOf(4), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertEquals(Integer.valueOf(3), buckets[2]); + } + + @Test + public void testSkip() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + Object[] buckets = avg.getBuckets(); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(1), buckets[1]); + assertEquals(Integer.valueOf(1), buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNotNull(buckets[1]); + assertNotNull(buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNotNull(buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + + // poke some test data into the array + buckets[0] = Integer.valueOf(1); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + } + + @Test + public void testHasData() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + + assertFalse(avg.hasData()); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertTrue(avg.hasData()); + + avg.skip(); + avg.skip(); + avg.skip(); + + assertFalse(avg.hasData()); + } + + @Test + public void testGetResult() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + + assertNull(avg.getResult()); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), avg.getResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java new file mode 100644 index 000000000000..acea22bc09f5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertThat; + + +public class DoubleMaxAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMaxAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMaxAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java new file mode 100644 index 000000000000..989c38ab7ab5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMaxAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = new DoubleMaxAverager(3, "test", "field", 1); + + assertEquals(Double.NEGATIVE_INFINITY, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", -1.1e100), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(5.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(3.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java new file mode 100644 index 000000000000..df023e58338c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMeanAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMeanAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMeanAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java new file mode 100644 index 000000000000..c0a8b39e254b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = new DoubleMeanAverager(3, "test", "field", 1); + + assertEquals(0.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(4.0 / 3, avg.computeResult(), 0.0); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java new file mode 100644 index 000000000000..929485574885 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanAveragerWithPeriodTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager averager = new DoubleMeanAverager(14, "test", "field", 7); + + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + + assertEquals(7, averager.computeResult(), 0.0); // (7+7)/2 + + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(1, averager.computeResult(), 0.0); // (1+1)/2 + + BaseAverager averager1 = new DoubleMeanAverager(14, "test", "field", 3); + + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + + assertEquals(1, averager1.computeResult(), 0.0); // (1+1+1+1+1)/5 + + assertEquals(2, averager1.computeResult(), 0.0); // (2+2+2+2+2)/5 + + assertEquals(13.0 / 5, averager1.computeResult(), 0.0); // (3+3+3+3+1)/5 + + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java new file mode 100644 index 000000000000..070f44914b91 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMeanNoNullAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMeanNoNullAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMeanNoNullAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java new file mode 100644 index 000000000000..46a6efa89deb --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanNoNullAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new DoubleMeanNoNullAverager(3, "test", "field", 1); + + assertEquals(Double.NaN, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(2.0, avg.computeResult(), 0.0); + + // testing cycleSize functionality + BaseAverager averager = new DoubleMeanNoNullAverager(14, "test", "field", 7); + + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, averager.computeResult(), 0.0); + + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 8.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 9.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", null), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 11.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 12.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 13.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 14.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 15.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 16.0), new HashMap<>()); + + assertEquals(7.5, averager.computeResult(), 0.0); + + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(8.5, averager.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java new file mode 100644 index 000000000000..88aa4d35d894 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMinAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMinAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMinAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java new file mode 100644 index 000000000000..f5ccab25b981 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMinAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new DoubleMinAverager(3, "test", "field", 1); + + assertEquals(Double.POSITIVE_INFINITY, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", -1.1e100), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + avg.skip(); + assertEquals(3.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java new file mode 100644 index 000000000000..ce90efd439f6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMaxAveragerFactoryTest +{ + + @Test + public void testCreateAverager() throws Exception + { + AveragerFactory fac = new LongMaxAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMaxAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java new file mode 100644 index 000000000000..24cc7bbb7349 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMaxAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMaxAverager(3, "test", "field", 1); + + assertEquals(Long.MIN_VALUE, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", -1000000L), new HashMap<>()); + assertEquals(-1000000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 1L), new HashMap<>()); + assertEquals(1, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(1, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 5L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(5, (long) avg.computeResult()); + + avg.skip(); + assertEquals(3, (long) avg.computeResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java new file mode 100644 index 000000000000..e164383f6778 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMeanAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMeanAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMeanAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java new file mode 100644 index 000000000000..7187c876fb20 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMeanAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMeanAverager(3, "test", "field", 1); + + assertEquals(0.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(3)), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(4.0 / 3, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java new file mode 100644 index 000000000000..0dcf9e5c274c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMeanNoNullAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMeanNoNullAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMeanNoNullAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java new file mode 100644 index 000000000000..34b9696eda3d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMeanNoNullAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMeanNoNullAverager(3, "test", "field", 1); + + assertEquals(Double.NaN, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(2.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java new file mode 100644 index 000000000000..48e75371c704 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMinAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMinAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMinAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java new file mode 100644 index 000000000000..1b0f874db824 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMinAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMinAverager(3, "test", "field", 1); + + assertEquals(Long.MAX_VALUE, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", -10000L), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 1L), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", new Integer(1000)), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 5L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(2, (long) avg.computeResult()); + + avg.skip(); + avg.skip(); + assertEquals(3, (long) avg.computeResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java new file mode 100644 index 000000000000..68cae5d19d54 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class SketchUnionAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new SketchUnionAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(SketchUnionAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java new file mode 100644 index 000000000000..0ff62f2ae409 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.yahoo.sketches.Family; +import com.yahoo.sketches.Util; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Union; +import com.yahoo.sketches.theta.UpdateSketch; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class SketchUnionAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = + new SketchUnionAverager(2, "test", "field", 1, Util.DEFAULT_NOMINAL_ENTRIES); + + UpdateSketch sketch1 = UpdateSketch.builder().build(); + for (int key = 0; key < 16; key++) { + sketch1.update(key); + } + + UpdateSketch sketch2 = UpdateSketch.builder().build(); + for (int key = 16; key < 32; key++) { + sketch2.update(key); + } + + UpdateSketch sketch3 = UpdateSketch.builder().build(); + for (int key = 32; key < 48; key++) { + sketch2.update(key); + } + + SketchHolder holder1 = SketchHolder.of(sketch1); + SketchHolder holder2 = SketchHolder.of(sketch2); + SketchHolder holder3 = SketchHolder.of(sketch3); + + Union union1 = (Union) SetOperation.builder().build(Family.UNION); + union1.update(sketch1); + union1.update(sketch2); + + avg.addElement(Collections.singletonMap("field", holder1), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", holder2), new HashMap<>()); + + assertEquals(avg.computeResult().getEstimate(), union1.getResult().getEstimate(), 0); + + avg.addElement(Collections.singletonMap("field", holder3), new HashMap<>()); + + Union union2 = (Union) SetOperation.builder().build(Family.UNION); + union2.update(sketch2); + union2.update(sketch3); + + assertEquals(avg.computeResult().getEstimate(), union2.getResult().getEstimate(), 0); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java new file mode 100644 index 000000000000..92c5715b954f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.test; + +import java.util.Map; + +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + +/** + * Configuration for a unit test. + */ +public class TestConfig +{ + public ObjectNode query; + public ArrayNode expectedOutput; + public Map intermediateResults; +} diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml new file mode 100644 index 000000000000..a3d2d164d2dc --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml @@ -0,0 +1,57 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml new file mode 100644 index 000000000000..a3d2d164d2dc --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml @@ -0,0 +1,57 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml new file mode 100644 index 000000000000..1458ed8c2af6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml @@ -0,0 +1,51 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: [] + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + timeseries: + - timestamp: 2017-01-01T00:00Z + result: + timespent_secs: 120.0 + timeSpent: 2.0 + - timestamp: 2017-01-02T00:00Z + result: + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml new file mode 100644 index 000000000000..626b94c23c7f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml @@ -0,0 +1,79 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + - name: regBcookies + fieldName: bcookies_with_yuid_sketch + size: 16384 + type: thetaSketch + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 + regBcookies: 0.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 0 + timeSpent: 0.0 + trailing7DayAvgTimeSpent: 1.0 + regBcookies: 0.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240 + timeSpent: 4.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml new file mode 100644 index 000000000000..460de3f4cfd5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml @@ -0,0 +1,70 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pr_pc_pl_us_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-01T00:00Z/2017-01-02T00:00Z + dimensions: + - gender + averagers: + - name: regBcookiesAvg + fieldName: regBcookies + size: 16384 + buckets: 2 + type: sketchUnion + aggregations: + - name: regBcookies + fieldName: bcookie_with_yuid_sketch + size: 16384 + type: thetaSketch + postAveragers: [ + ] + postAggregations: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: 0.0 + regBcookiesAvg: 0.0 +- version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: 0.0 + regBcookiesAvg: 0.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2016-12-31T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml new file mode 100644 index 000000000000..ba685ff8411a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml @@ -0,0 +1,81 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: trailing7DayAvgTimeSpent +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + trailing7DayAvgTimeSpent: 6.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml new file mode 100644 index 000000000000..59f75bc04807 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml @@ -0,0 +1,82 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: trailing7DayAvgTimeSpent + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + trailing7DayAvgTimeSpent: 6.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml new file mode 100644 index 000000000000..c7d7ddc05110 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml @@ -0,0 +1,84 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_uc_ud + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 7 + name: trailing7DayAvgTotalPageViews + fieldName: totalPageViews + type: doubleMean + aggregations: + - name: addPageViews + fieldName: additive_page_views + type: longSum + - name: pageViews + fieldName: other_page_views + type: longSum + postAggregations: + - type: arithmetic + name: totalPageViews + fn: + + fields: + - type: fieldAccess + fieldName: addPageViews + - type: fieldAccess + fieldName: pageViews + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: addPageViews + direction: DESC + dimension: trailing7DayAvgTotalPageViews + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 + trailing7DayAvgTotalPageViews: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + addPageViews: 0 + pageViews: 0 + totalPageViews: 0.0 + trailing7DayAvgTotalPageViews: 2.142857142857143 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + addPageViews: 5.0 + pageViews: 10.0 + totalPageViews: 15.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + addPageViews: 6.0 + pageViews: 12.0 + totalPageViews: 18.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml new file mode 100644 index 000000000000..89ae94155e97 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml @@ -0,0 +1,82 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_uc_ud + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 7 + name: trailing7DayAvgTotalPageViews + fieldName: totalPageViews + type: doubleMean + aggregations: + - name: addPageViews + fieldName: additive_page_views + type: longSum + - name: pageViews + fieldName: other_page_views + type: longSum + postAggregations: + - type: arithmetic + name: totalPageViews + fn: + + fields: + - type: fieldAccess + fieldName: addPageViews + - type: fieldAccess + fieldName: pageViews + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: addPageViews + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 + trailing7DayAvgTotalPageViews: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + addPageViews: 0 + pageViews: 0 + totalPageViews: 0.0 + trailing7DayAvgTotalPageViews: 2.142857142857143 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + addPageViews: 5.0 + pageViews: 10.0 + totalPageViews: 15.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + addPageViews: 6.0 + pageViews: 12.0 + totalPageViews: 18.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties new file mode 100644 index 000000000000..cf60cde85fe5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -0,0 +1,57 @@ +druid.broker.cache.unCacheable=["groupBy","segmentMetadata"] +druid.broker.http.numConnections=50 +druid.server.http.numThreads=1 +druid.announcer.maxBytesPerNode=524288 +druid.announcer.segmentsPerNode=50 +druid.announcer.type=batch +druid.broker.balancer.type=random +druid.broker.cache.initialSize=500000 +druid.broker.cache.logEvictionCount=0 +druid.broker.cache.sizeInBytes=0 +druid.broker.cache.type=local +druid.broker.http.readTimeOut=PT15M +druid.broker.select.tier=highestPriority +druid.broker.select.tier.custom.priorities= +druid.coordinator.conversion.on=false +druid.coordinator.load.timeout=PT15M +druid.coordinator.merge.on=false +druid.coordinator.period=PT30S +druid.coordinator.period.indexingPeriod=PT1800S +druid.coordinator.startDelay=PT60S +druid.curator.compress=false +druid.emitter.logging.loggerClass=LoggingEmitter +druid.emitter.logging.logLevel=info +druid.extensions.coordinates=[] +druid.manager.config.pollDuration=PT1M +druid.manager.rules.defaultTier=tier1 +druid.manager.rules.pollDuration=PT1M +druid.manager.segment.pollDuration=PT1M +druid.monitoring.emissionPeriod=PT1M +druid.monitoring.monitors=[\"com.metamx.metrics.JvmMonitor\"] +druid.processing.buffer.sizeBytes=655360 +druid.processing.numThreads=1 +druid.processing.columnCache.sizeBytes=0 +druid.processing.formatString='processing-%s' +druid.publish.type=metadata +druid.query.chunkPeriod=P1M +druid.query.groupBy.defaultStrategy=v1 +druid.query.groupBy.maxIntermediateRows=50000 +druid.query.groupBy.maxResults=5000000 +druid.query.groupBy.singleThreaded=false +druid.query.search.maxSearchLimit=1000 +druid.request.logging.feed= +druid.request.logging.dir= +druid.request.logging.type=noop +druid.segmentCache.announceIntervalMillis=5000 +druid.segmentCache.deleteOnRemove=true +druid.segmentCache.dropSegmentDelayMillis=30000 +druid.segmentCache.numLoadingThreads=1 +druid.selectors.indexing.serviceName=overlord +druid.server.http.maxIdleTime=PT5m +druid.storage.type=hdfs +druid.worker.capacity=30 +druid.server.maxSize=1073741824 +druid.server.priority=0 +druid.server.tier=_default_tier +druid.worker.ip=localhost +druid.worker.version=0 diff --git a/pom.xml b/pom.xml index 7b54762bf91e..9861023dbdd9 100644 --- a/pom.xml +++ b/pom.xml @@ -155,6 +155,7 @@ extensions-contrib/opentsdb-emitter extensions-contrib/materialized-view-maintenance extensions-contrib/materialized-view-selection + extensions-contrib/moving-average-query distribution From ef7e0e6012b40ee0b955f077d67f080f1dab611a Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 8 Oct 2018 23:16:21 -0700 Subject: [PATCH 02/45] Fix failing code inspections. --- .../movingaverage/MovingAverageQuery.java | 12 ++-- .../MovingAverageQueryModule.java | 9 ++- .../MovingAverageQueryRunner.java | 5 +- .../movingaverage/RowBucketIterable.java | 2 +- .../MovingAverageIterableTest.java | 60 ++++++++-------- .../movingaverage/MovingAverageQueryTest.java | 6 +- .../movingaverage/RowBucketIterableTest.java | 70 +++++++++---------- 7 files changed, 86 insertions(+), 78 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 52ddd83a230c..f10efd0626c3 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -94,14 +94,14 @@ public MovingAverageQuery( this.dimFilter = dimFilter; this.granularity = granularity; - this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; for (DimensionSpec spec : this.dimensions) { Preconditions.checkArgument(spec != null, "dimensions has null DimensionSpec"); } - this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; - this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; - this.averagerSpecs = averagerSpecs == null ? ImmutableList.>of() : averagerSpecs; - this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; + this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; + this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; + this.averagerSpecs = averagerSpecs == null ? ImmutableList.of() : averagerSpecs; + this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; this.havingSpec = havingSpec; this.limitSpec = (limitSpec == null) ? NoopLimitSpec.INSTANCE : limitSpec; @@ -232,12 +232,14 @@ public boolean getContextSortByDimsFirst() return getContextBoolean(CTX_KEY_SORT_BY_DIMS_FIRST, false); } + @Override @JsonProperty public DimFilter getFilter() { return dimFilter; } + @Override @JsonProperty public Granularity getGranularity() { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java index a7efa358e666..05ae2560d6f5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -32,7 +32,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryToolChest; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class MovingAverageQueryModule implements DruidModule @@ -53,8 +53,11 @@ public void configure(Binder binder) @Override public List getJacksonModules() { - return Arrays.asList(new SimpleModule("MovingAverageQueryModule") - .registerSubtypes(new NamedType(MovingAverageQuery.class, "movingAverage"))); + return Collections.singletonList(new SimpleModule("MovingAverageQueryModule") + .registerSubtypes(new NamedType( + MovingAverageQuery.class, + "movingAverage" + ))); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 1bf421ee1899..cb0c1b4c55ac 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -141,7 +141,7 @@ public Sequence run(QueryPlus query, Map responseConte "127.0.0.1", gbq, new QueryStats( - ImmutableMap.of( + ImmutableMap.of( "query/time", 0, "query/bytes", 0, "success", true @@ -179,7 +179,7 @@ public Sequence run(QueryPlus query, Map responseConte "127.0.0.1", tsq, new QueryStats( - ImmutableMap.of( + ImmutableMap.of( "query/time", 0, "query/bytes", 0, "success", true @@ -228,6 +228,7 @@ public Sequence run(QueryPlus query, Map responseConte static class TimeseriesResultToRow implements Function, Row> { + @Override public Row apply(Result lookbackResult) { Map event = lookbackResult.getValue().getBaseObject(); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 25dfe246ffca..f0201fbac784 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -76,7 +76,7 @@ public RowIterator(Sequence rows, List intervals, Period period) this.intervals = intervals; expectedBucket = intervals.get(intervalIndex).getStart(); endTime = intervals.get(intervals.size() - 1).getEnd(); - yielder = rows.toYielder(null, new BucketingAccumulator()); + yielder = rows.toYielder(null, new BucketingAccumulator()); } /* (non-Javadoc) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 9e89b69bfb18..415d13862911 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -106,7 +106,7 @@ public void testNext() new MapBasedRow(JAN_1, dims1), new MapBasedRow(JAN_1, dims2) )), - new RowBucket(JAN_2, Arrays.asList( + new RowBucket(JAN_2, Collections.singletonList( new MapBasedRow(JAN_2, dims1) )), new RowBucket(JAN_3, Collections.emptyList()), @@ -217,8 +217,8 @@ public void testAveraging() new LongMeanAveragerFactory("movingAvgPageViews", 7, 1, "pageViews") ), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -281,11 +281,11 @@ public void testCompleteData() new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -349,15 +349,15 @@ public void testMissingDataAtBeginning() Row jan_2_row3 = new MapBasedRow(JAN_2, event3); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan_1_row1)), + new RowBucket(JAN_1, Collections.singletonList(jan_1_row1)), new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -410,14 +410,14 @@ public void testMissingDataAtTheEnd() Sequence seq = Sequences.simple(Arrays.asList( new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1)) + new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -485,16 +485,16 @@ public void testMissingDataAtMiddle() Sequence seq = Sequences.simple(Arrays.asList( new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1)), + new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)), new RowBucket(JAN_3, Arrays.asList(jan_3_row1, jan_3_row2, jan_3_row3)), - new RowBucket(JAN_4, Arrays.asList(jan_4_row1)) + new RowBucket(JAN_4, Collections.singletonList(jan_4_row1)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 3, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -586,11 +586,11 @@ public void testMissingDaysAtBegining() new RowBucket(JAN_4, Collections.singletonList(row2)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -632,11 +632,11 @@ public void testMissingDaysInMiddle() new RowBucket(JAN_4, Collections.singletonList(row2)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -693,10 +693,10 @@ public void testWithFilteredAggregation() DimFilter filter = new SelectorDimFilter("gender", "m", null); FilteredAggregatorFactory filteredAggregatorFactory = new FilteredAggregatorFactory(aggregatorFactory, filter); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( averagerfactory), Collections.emptyList(), - Collections.singletonList( + Collections.singletonList( filteredAggregatorFactory) ).iterator(); @@ -750,11 +750,11 @@ public void testMissingDaysAtEnd() new RowBucket(JAN_6, Collections.emptyList()) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index aaadd2344ec8..3e1313d7983e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -131,7 +132,7 @@ public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMa binder.bindConstant().annotatedWith(Names.named("serviceName")).to("queryTest"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(1); - binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); + binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); } ); @@ -420,7 +421,8 @@ public long getMaxQueuedBytes() ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( new ServiceEmitter("", "", null) { - public void emit(org.apache.druid.java.util.emitter.core.Event event) {} + @Override + public void emit(Event event) {} }, baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() ); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 912b91ad0294..6746a1e31e74 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -107,10 +107,10 @@ public void testCompleteData() rows.add(JAN_3_M_10); rows.add(JAN_4_M_10); - List expected_day1 = Arrays.asList(JAN_1_M_10); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -142,9 +142,9 @@ public void testApplyLastDaySingleRow() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); + List expected_day4 = Collections.singletonList(JAN_4_M_10); rows = new ArrayList(); rows.add(JAN_1_M_10); @@ -179,8 +179,8 @@ public void testApplyLastDayMultipleRows() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); rows = new ArrayList(); @@ -220,7 +220,7 @@ public void testSingleDaySingleRow() rows = new ArrayList(); rows.add(JAN_1_M_10); - List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -263,7 +263,7 @@ public void testMissingDaysAtBegining() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_2); @@ -291,9 +291,9 @@ public void testMissingDaysAtBeginingFollowedByMultipleRow() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -330,8 +330,8 @@ public void testMissingDaysAtBeginingAndAtTheEnd() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); List expected_day4 = Collections.emptyList(); intervals = new ArrayList<>(); @@ -368,9 +368,9 @@ public void testMultipleMissingDays() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -406,10 +406,10 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); - List expected_day5 = Arrays.asList(JAN_5_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expected_day5 = Collections.singletonList(JAN_5_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_5); @@ -450,10 +450,10 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() public void testMissingDaysInMiddleOneRow() { - List expected_day1 = Arrays.asList(JAN_1_M_10); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); rows = new ArrayList(); rows.add(JAN_1_M_10); @@ -487,10 +487,10 @@ public void testMissingDaysInMiddleOneRow() public void testMissingDaysInMiddleMultipleRow() { - List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); List expected_day2 = Collections.emptyList(); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -531,8 +531,8 @@ public void testApplyLastDayNoRows() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Collections.emptyList(); rows = new ArrayList(); @@ -565,7 +565,7 @@ public void testApplyLastTwoDayNoRows() { List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); List expected_day4 = Collections.emptyList(); @@ -606,12 +606,12 @@ public void testApplyMultipleInterval() intervals.add(INTERVAL_JAN_6_8); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); - List expected_day6 = Arrays.asList(JAN_6_M_10); - List expected_day7 = Arrays.asList(JAN_7_F_20); - List expected_day8 = Arrays.asList(JAN_8_U_30); + List expected_day6 = Collections.singletonList(JAN_6_M_10); + List expected_day7 = Collections.singletonList(JAN_7_F_20); + List expected_day8 = Collections.singletonList(JAN_8_U_30); rows = new ArrayList(); rows.add(JAN_1_M_10); From 136eb939effa0fdc1c6a13a4d4ad92c4cee68a85 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Tue, 9 Oct 2018 12:29:15 -0700 Subject: [PATCH 03/45] See if explicit types will invoke the correct comparison function. --- .../druid/query/movingaverage/MovingAverageQueryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 3e1313d7983e..43dae733a793 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -366,7 +366,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) Query query = jsonMapper.readValue(getQueryString(), Query.class); assertThat(query, IsInstanceOf.instanceOf(getExpectedQueryType())); - List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); + List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); assertNotNull(expectedResults); assertThat(expectedResults, IsInstanceOf.instanceOf(List.class)); @@ -434,7 +434,7 @@ baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheCon final Sequence res = query.getRunner(walker).run(queryPlus, responseContext); List actualResults = new ArrayList(); - actualResults = (List) res.accumulate(actualResults, Accumulators.list()); + actualResults = (List) res.accumulate(actualResults, Accumulators.list()); consistentTypeCasting(expectedResults); consistentTypeCasting(actualResults); From 1d9884792cb162433fc2d3b9c6728e132d9b5853 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 10 Oct 2018 13:29:54 -0700 Subject: [PATCH 04/45] Explicitly remove support for druid.generic.useDefaultValueForNull configuration parameter. --- .../development/extensions-contrib/moving-average-query.md | 7 +++++-- .../druid/query/movingaverage/MovingAverageQuery.java | 4 ++++ .../query/movingaverage/MovingAverageIterableTest.java | 2 ++ .../druid/query/movingaverage/MovingAverageQueryTest.java | 2 +- .../query/movingaverage/PostAveragerCalculatorTest.java | 1 + 5 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 387b397ae991..37aaa99842c9 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -35,10 +35,13 @@ To use this extension, make sure to [load](../../operations/including-extensions ##Configuration There are currently no configuration properties specific to Moving Average. +##Limitations +* movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. +* movingAverage is missing support for the following timeseries properties: `descending`. +* movingAverage is missing support for [SQL-compatible null handling](https://github.com/apache/incubator-druid/issues/4349) (So setting druid.generic.useDefaultValueForNull in configuration will give an error). + ##Query spec: * Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. -* Note: movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. -* Note: movingAverage is missing support for the following timeseries properties: `descending`. |property|description|required?| |--------|-----------|---------| diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index f10efd0626c3..6d5fdde5c0fd 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -30,6 +30,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; @@ -92,6 +93,9 @@ public MovingAverageQuery( { super(dataSource, querySegmentSpec, false, context); + //TBD: Implement null awareness to respect the contract of this flag. + Preconditions.checkArgument(NullHandling.replaceWithDefault(), "movingAverage does not support druid.generic.useDefaultValueForNull=false"); + this.dimFilter = dimFilter; this.granularity = granularity; this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 415d13862911..27c6a3148abb 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -610,6 +610,7 @@ public void testMissingDaysAtBegining() @Test public void testMissingDaysInMiddle() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Map event1 = new HashMap<>(); Map event2 = new HashMap<>(); @@ -726,6 +727,7 @@ public void testWithFilteredAggregation() @Test public void testMissingDaysAtEnd() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Map event1 = new HashMap<>(); Map event2 = new HashMap<>(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 43dae733a793..491af896545c 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -136,7 +136,7 @@ public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMa } ); - + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Injector baseInjector = GuiceInjectors.makeStartupInjector(); injector = Initialization.makeInjectorWithModules(baseInjector, modules); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java index dbd127fdf7e6..4073302c7b0a 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -57,6 +57,7 @@ public class PostAveragerCalculatorTest @Before public void setup() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); query = new MovingAverageQuery( new TableDataSource("d"), new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval("2017-01-01/2017-01-01", ISOChronology.getInstanceUTC()))), From 7b71a261b5ef7d16d026cb94e5abd7283bf76345 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 10:46:32 -0800 Subject: [PATCH 05/45] Update styling and headers for complience. --- .../moving-average-query.md | 19 +++++++ .../moving-average-query/pom.xml | 20 +++---- .../movingaverage/AveragerFactoryWrapper.java | 9 +-- .../movingaverage/BucketingAccumulator.java | 7 ++- .../DefaultMovingAverageQueryMetrics.java | 1 + ...faultMovingAverageQueryMetricsFactory.java | 1 + .../IdentityYieldingAccumulator.java | 1 + .../movingaverage/MovingAverageHelper.java | 8 +-- .../movingaverage/MovingAverageIterable.java | 6 +- .../movingaverage/MovingAverageQuery.java | 2 +- .../MovingAverageQueryMetrics.java | 1 + .../MovingAverageQueryMetricsFactory.java | 1 + .../MovingAverageQueryModule.java | 2 - .../MovingAverageQueryRunner.java | 26 ++++----- .../MovingAverageQueryToolChest.java | 9 ++- .../movingaverage/PostAveragerCalculator.java | 7 ++- .../druid/query/movingaverage/RowBucket.java | 4 +- .../movingaverage/RowBucketIterable.java | 8 +-- .../movingaverage/averagers/Averager.java | 4 +- .../averagers/AveragerFactory.java | 7 ++- .../movingaverage/averagers/BaseAverager.java | 11 ++-- .../averagers/BaseAveragerFactory.java | 7 ++- .../averagers/ComparableAveragerFactory.java | 4 +- .../averagers/ConstantAverager.java | 5 +- .../averagers/ConstantAveragerFactory.java | 6 +- .../averagers/DoubleMaxAverager.java | 1 + .../averagers/DoubleMaxAveragerFactory.java | 1 + .../averagers/DoubleMeanAverager.java | 1 + .../averagers/DoubleMeanAveragerFactory.java | 1 + .../averagers/DoubleMeanNoNullAverager.java | 1 + .../DoubleMeanNoNullAveragerFactory.java | 1 + .../averagers/DoubleMinAverager.java | 1 + .../averagers/DoubleMinAveragerFactory.java | 1 + .../averagers/LongMaxAverager.java | 1 + .../averagers/LongMaxAveragerFactory.java | 1 + .../averagers/LongMeanAverager.java | 1 + .../averagers/LongMeanAveragerFactory.java | 1 + .../averagers/LongMeanNoNullAverager.java | 1 + .../LongMeanNoNullAveragerFactory.java | 1 + .../averagers/LongMinAverager.java | 1 + .../averagers/LongMinAveragerFactory.java | 1 + .../averagers/SketchUnionAverager.java | 1 + .../averagers/SketchUnionAveragerFactory.java | 5 +- .../IdentityYieldingAccumulatorTest.java | 8 +-- .../MovingAverageIterableTest.java | 33 ++++++----- .../movingaverage/MovingAverageQueryTest.java | 55 +++++++++---------- .../PostAveragerCalculatorTest.java | 18 +++--- .../movingaverage/RowBucketIterableTest.java | 8 +-- .../averagers/BaseAveragerFactoryTest.java | 1 + .../averagers/BaseAveragerTest.java | 1 + .../DoubleMaxAveragerFactoryTest.java | 1 + .../averagers/DoubleMaxAveragerTest.java | 1 + .../DoubleMeanAveragerFactoryTest.java | 1 + .../averagers/DoubleMeanAveragerTest.java | 1 + .../DoubleMeanAveragerWithPeriodTest.java | 1 + .../DoubleMeanNoNullAveragerFactoryTest.java | 1 + .../DoubleMeanNoNullAveragerTest.java | 1 + .../DoubleMinAveragerFactoryTest.java | 1 + .../averagers/DoubleMinAveragerTest.java | 1 + .../averagers/LongMaxAveragerFactoryTest.java | 1 + .../averagers/LongMaxAveragerTest.java | 1 + .../LongMeanAveragerFactoryTest.java | 1 + .../averagers/LongMeanAveragerTest.java | 1 + .../LongMeanNoNullAveragerFactoryTest.java | 1 + .../averagers/LongMeanNoNullAveragerTest.java | 1 + .../averagers/LongMinAveragerFactoryTest.java | 1 + .../averagers/LongMinAveragerTest.java | 1 + .../SketchUnionAveragerFactoryTest.java | 1 + .../averagers/SketchUnionAveragerTest.java | 1 + .../query/movingaverage/test/TestConfig.java | 5 +- 70 files changed, 203 insertions(+), 143 deletions(-) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 37aaa99842c9..804b4fdb6866 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -2,6 +2,25 @@ layout: doc_page --- + + # Moving Average Queries ## Overview diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index e9c4e77671c0..a7e76d1bca97 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -21,11 +21,11 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 - + - druid org.apache.druid - 0.13.0-SNAPSHOT + druid + 0.13.0-incubating-SNAPSHOT ../../pom.xml @@ -63,7 +63,7 @@ org.apache.druid - druid-api + druid-core ${project.parent.version} provided @@ -73,17 +73,17 @@ ${project.parent.version} provided - - org.apache.druid - druid-server - ${project.parent.version} - provided - org.apache.druid.extensions druid-datasketches ${project.parent.version} provided + + org.apache.druid + druid-server + 0.13.0-incubating-SNAPSHOT + compile + diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 00b5982db946..198f898d9f18 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -16,17 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.Comparator; -import java.util.List; +package org.apache.druid.query.movingaverage; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.segment.ColumnSelectorFactory; +import java.util.Comparator; +import java.util.List; + /** * A wrapper around averagers that makes them appear to be aggregators. * This is necessary purely to allow existing common druid code that only knows diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java index 34ec4353e65a..1aa8782b242f 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.ArrayList; -import java.util.Collection; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import java.util.ArrayList; +import java.util.Collection; + /** * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java index 857e48201f2e..8dced39ccc97 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java index d494a2e48b9b..64b7c5b6910b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java index b0250b0a5c7e..4171a72781cb 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.java.util.common.guava.YieldingAccumulator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java index 4188af4cba7c..f0cfb0bd15bf 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java @@ -19,14 +19,14 @@ package org.apache.druid.query.movingaverage; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.dimension.DimensionSpec; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + public class MovingAverageHelper { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 1cf6aa25a51a..c01aaa4f5a65 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -16,10 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import org.apache.druid.query.movingaverage.averagers.Averager; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -29,6 +27,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.movingaverage.averagers.Averager; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 6d5fdde5c0fd..7f12e5e1ab57 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -31,7 +31,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularity; @@ -47,6 +46,7 @@ import org.apache.druid.query.groupby.having.HavingSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.spec.QuerySegmentSpec; import java.util.ArrayList; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java index 48a8e564a2c3..6b9f39ad0f2e 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.query.QueryMetrics; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java index 3857d323b7fd..db344a0f0ec5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; /** diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java index 05ae2560d6f5..9655678680da 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -22,10 +22,8 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; - import com.google.inject.Binder; import com.google.inject.multibindings.MapBinder; - import org.apache.druid.guice.DruidBinders; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.DruidModule; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index cb0c1b4c55ac..d64ba18222d2 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -19,46 +19,44 @@ package org.apache.druid.query.movingaverage; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.query.QueryContexts; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.QueryStats; import org.apache.druid.server.RequestLogLine; import org.apache.druid.server.log.RequestLogger; +import org.joda.time.Interval; +import org.joda.time.Period; import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** * The QueryRunner for MovingAverage query. diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index e35b22df32d0..1bc9a8a37273 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -19,16 +19,11 @@ package org.apache.druid.query.movingaverage; -import java.util.Map; - -import javax.annotation.Nullable; - import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.Maps; import com.google.inject.Inject; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.QueryMetrics; @@ -38,8 +33,12 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.server.log.RequestLogger; +import javax.annotation.Nullable; +import java.util.Map; + /** * The QueryToolChest for MovingAverage Query */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java index 39e988b4bb76..3490f3e4192c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java @@ -16,16 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.List; -import java.util.Map; +package org.apache.druid.query.movingaverage; import com.google.common.base.Function; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.aggregation.PostAggregator; +import java.util.List; +import java.util.Map; + /** * Function that can be applied to a Sequence to calculate PostAverager results */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java index 8bf058fdf68f..2c96530a209a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -16,14 +16,14 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.Row; import org.joda.time.DateTime; import java.util.Collection; -import org.apache.druid.data.input.Row; - /** * Represents a set of rows for a specific date */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index f0201fbac784..1e1fda717215 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -16,8 +16,12 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -27,10 +31,6 @@ import java.util.List; import java.util.NoSuchElementException; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; - /** * It is the iterable used to bucket data into days, * doing appropriate lookahead to see if the next row is in the same day or a new day. diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java index 20da12ca3fca..506380cac1bb 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java @@ -19,10 +19,10 @@ package org.apache.druid.query.movingaverage.averagers; -import java.util.Map; - import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Map; + /** * Interface for an averager * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java index e8f2ad4fe7d5..e2041b0db40a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Comparator; -import java.util.List; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.util.Comparator; +import java.util.List; + /** * Interface representing Averager in the movingAverage query. * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index e525deb223e4..37497c695ead 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -16,13 +16,14 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; +import org.apache.druid.query.aggregation.AggregatorFactory; + import java.lang.reflect.Array; import java.util.Map; -import org.apache.druid.query.aggregation.AggregatorFactory; - /** * Common base class available for use by averagers. The base class implements methods that * capture incoming and skipped rows and store them in an array, to be used later for @@ -53,10 +54,8 @@ public abstract class BaseAverager implements Averager * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). * @param shouldFinalize Whether the intermediate values need to be finalized prior to averaging */ - public BaseAverager( - Class storageType, int numBuckets, String name, String fieldName, int cycleSize, - boolean shouldFinalize - ) + public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize, + boolean shouldFinalize) { this.numBuckets = numBuckets; this.name = name; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java index d91e74f687b7..dda08c04be08 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Collections; -import java.util.List; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; + /** * Common base class for AveragerFactories * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java index ca76b505ae45..0463d55b97da 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java @@ -16,9 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -/** - * Copyright 2018 Oath Inc. All rights reserved. - */ + package org.apache.druid.query.movingaverage.averagers; import java.util.Comparator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java index 806fa6b21242..bc76c99610f9 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java @@ -16,12 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Map; +package org.apache.druid.query.movingaverage.averagers; import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Map; + /** * The constant averager.Created soley for incremental development and wiring things up. */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java index cffc876fa399..45339c37058b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java @@ -19,13 +19,13 @@ package org.apache.druid.query.movingaverage.averagers; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Comparator; import java.util.List; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - /** * Implementation of AveragerFacvtory created solely for incremental development */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java index c9db3f2cd069..2c6ac9cdf5ab 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMaxAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java index 9c708428fff9..1e82f09e9e95 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java index ece43f105992..9b0459df0db8 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMeanAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java index 233b7c521bad..58f544671a96 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java index 5f98c25ea9bd..c3b8adcd519d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMeanNoNullAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java index 9abf4fcce7d1..d6e11893a5e7 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java index 0ded39e82f23..1c6a4cb58d74 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMinAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java index d3d81f1be13b..35a783b2235d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java index 343a1c8f9e3a..b2c04d12aecd 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMaxAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java index 473026fd7180..847bbcb9e341 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java index 8c6d97d53863..8d7892d7c22c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMeanAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java index 2c413b976a75..d02e06d96173 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java index 5cf6548e3605..0e59af32d628 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMeanNoNullAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java index 5761e61ae4d1..03ad7d1e654c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java index 0a275988395d..fedf055ce91e 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMinAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java index 2d9f342414ea..ff2562541172 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java index 0756204fa5e6..12ce0bca3d9a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.yahoo.sketches.Family; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java index 5a67de968365..5e687ba3a2e7 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java @@ -16,15 +16,16 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Comparator; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.Util; import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; +import java.util.Comparator; + public class SketchUnionAveragerFactory extends BaseAveragerFactory { diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java index bc052b3b9854..dac2d693a13a 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java @@ -16,15 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; - -import org.junit.Test; -import java.util.Arrays; +package org.apache.druid.query.movingaverage; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; +import org.junit.Test; + +import java.util.Arrays; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 27c6a3148abb..a1c7e9acc0f5 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -16,24 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import static org.hamcrest.CoreMatchers.anyOf; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; -import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; -import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; - -import org.joda.time.DateTime; -import org.joda.time.chrono.ISOChronology; -import org.junit.Test; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -46,6 +30,12 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; +import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; +import org.joda.time.DateTime; +import org.joda.time.chrono.ISOChronology; +import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; @@ -56,6 +46,15 @@ import java.util.List; import java.util.Map; +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + /** * */ diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 491af896545c..ff5746449f99 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -16,23 +16,43 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; import com.google.inject.util.Providers; +import mockit.Mock; +import mockit.MockUp; +import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.client.cache.MapCache; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -41,6 +61,12 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.RetryQueryRunnerConfig; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.movingaverage.test.TestConfig; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.timeline.TimelineLookup; @@ -62,33 +88,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; -import com.fasterxml.jackson.core.JsonParseException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import org.apache.druid.query.movingaverage.test.TestConfig; -import org.apache.druid.client.CachingClusteredClient; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.initialization.Initialization; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; -import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.timeseries.TimeseriesQuery; -import org.apache.druid.query.timeseries.TimeseriesResultValue; - -import mockit.Mock; -import mockit.MockUp; - -import org.apache.druid.java.util.emitter.service.ServiceEmitter; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java index 4073302c7b0a..d961bc47059e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -16,9 +16,18 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.movingaverage.averagers.DoubleMeanAveragerFactory; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; @@ -30,15 +39,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; -import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 6746a1e31e74..6633d0068a85 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -16,16 +16,13 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -41,6 +38,9 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class RowBucketIterableTest { diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java index 8b16d29dcc44..9956a7065f3d 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Before; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java index 302cb4c0b25e..c6e960d84b89 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java index acea22bc09f5..773cae4843ba 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java index 989c38ab7ab5..496063355743 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java index df023e58338c..68d9b67024f8 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java index c0a8b39e254b..a0840a8d33f7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java index 929485574885..49c73b34c5ca 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java index 070f44914b91..9359fc2cefee 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java index 46a6efa89deb..6d946e4835c7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java index 88aa4d35d894..ef2bb6f0882c 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java index f5ccab25b981..02fd2c2cec8b 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java index ce90efd439f6..7246e8bd006e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java index 24cc7bbb7349..c799a1ad60b7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java index e164383f6778..763d9c75321f 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java index 7187c876fb20..c34d1ec2e612 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java index 0dcf9e5c274c..f3c4dac4902e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java index 34b9696eda3d..0681db77ffcd 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java index 48e75371c704..067f6b223a10 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java index 1b0f874db824..4cbcdaed4c0d 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java index 68cae5d19d54..540dc5ffccd9 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java index 0ff62f2ae409..b70b7a9539ad 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.yahoo.sketches.Family; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java index 92c5715b954f..792394e2723e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java @@ -16,13 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.test; -import java.util.Map; +package org.apache.druid.query.movingaverage.test; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.Map; + /** * Configuration for a unit test. */ From 6635751805e4ac6208d5bcd18d13f751d7d6f3f6 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 14:06:09 -0800 Subject: [PATCH 06/45] Refresh code with latest master changes: * Remove NullDimensionSelector. * Apply changes of RequestLogger. * Apply changes of TimelineServerView. --- .../DefaultMovingAverageQueryMetricsFactory.java | 4 ++++ .../druid/query/movingaverage/MovingAverageIterable.java | 3 +-- .../query/movingaverage/MovingAverageQueryRunner.java | 8 ++++---- .../druid/query/movingaverage/MovingAverageQueryTest.java | 6 ++++++ .../src/test/resources/runtime.properties | 1 + 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java index 64b7c5b6910b..c9efb8bc25d3 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -33,6 +33,10 @@ public class DefaultMovingAverageQueryMetricsFactory implements MovingAverageQue private static final MovingAverageQueryMetricsFactory INSTANCE = new DefaultMovingAverageQueryMetricsFactory(new DefaultObjectMapper()); + /** + * Should be used only in tests, directly or indirectly (via {@link + * MovingAverageQueryToolChest#MovingAverageQueryToolChest}). + */ @VisibleForTesting public static MovingAverageQueryMetricsFactory instance() { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index c01aaa4f5a65..463634a93e98 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -32,7 +32,6 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.NullDimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; import java.util.Collection; @@ -84,7 +83,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) { // Generating empty records while aggregating on Filtered aggregators requires a dimension selector // for initialization. This dimension selector is not actually used for generating values - return NullDimensionSelector.instance(); + return DimensionSelector.constant(null); } @Override diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index d64ba18222d2..0328f4b029f5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -134,10 +134,10 @@ public Sequence run(QueryPlus query, Map responseConte Sequence results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponse); try { // use localhost for remote address - requestLogger.log(new RequestLogLine( + requestLogger.logNativeQuery(RequestLogLine.forNative( + gbq, DateTimes.nowUtc(), "127.0.0.1", - gbq, new QueryStats( ImmutableMap.of( "query/time", 0, @@ -172,10 +172,10 @@ public Sequence run(QueryPlus query, Map responseConte Sequence> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponse); try { // use localhost for remote address - requestLogger.log(new RequestLogLine( + requestLogger.logNativeQuery(RequestLogLine.forNative( + tsq, DateTimes.nowUtc(), "127.0.0.1", - tsq, new QueryStats( ImmutableMap.of( "query/time", 0, diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index ff5746449f99..baf887063c61 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -379,6 +379,12 @@ public TimelineLookup getTimeline(DataSource dataSource) return null; } + @Override + public List getDruidServers() + { + return null; + } + @Override public QueryRunner getQueryRunner(DruidServer server) { diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index cf60cde85fe5..2877ef660145 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -55,3 +55,4 @@ druid.server.priority=0 druid.server.tier=_default_tier druid.worker.ip=localhost druid.worker.version=0 +druid.request.logging.type=slf4j \ No newline at end of file From b7317805679bb7be644964131ad5a6bfd89bd1ce Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 15:46:20 -0800 Subject: [PATCH 07/45] Small checkstyle fix. --- .../moving-average-query/src/test/resources/runtime.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index 2877ef660145..1c6b09b91b36 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -55,4 +55,4 @@ druid.server.priority=0 druid.server.tier=_default_tier druid.worker.ip=localhost druid.worker.version=0 -druid.request.logging.type=slf4j \ No newline at end of file +druid.request.logging.type=slf4j From 95b803a4f06ae5dfbe3c2b6461ee12cb885443e7 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 16:58:52 -0800 Subject: [PATCH 08/45] Checkstyle fixes. --- .../apache/druid/query/movingaverage/MovingAverageQuery.java | 4 ++-- .../query/movingaverage/MovingAverageQueryToolChest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 7f12e5e1ab57..38fc1ebcc124 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -29,7 +29,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; @@ -50,6 +49,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -164,7 +164,7 @@ private static void verifyOutputNames( ) { - final Set outputNames = Sets.newHashSet(); + final Set outputNames = new HashSet<>(); for (DimensionSpec dimension : dimensions) { if (!outputNames.add(dimension.getOutputName())) { throw new IAE("Duplicate output name[%s]", dimension.getOutputName()); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index 1bc9a8a37273..ae293f440a8b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Maps; import com.google.inject.Inject; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -37,6 +36,7 @@ import org.apache.druid.server.log.RequestLogger; import javax.annotation.Nullable; +import java.util.HashMap; import java.util.Map; /** @@ -100,7 +100,7 @@ public Function makePostComputeManipulatorFn(MovingAverageQuery query, public Row apply(Row result) { MapBasedRow mRow = (MapBasedRow) result; - final Map values = Maps.newHashMap(mRow.getEvent()); + final Map values = new HashMap(mRow.getEvent()); for (AggregatorFactory agg : query.getAggregatorSpecs()) { Object aggVal = values.get(agg.getName()); From a21a3ceab8f23d4791f7ed188cd9d34f05858b30 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Thu, 31 Jan 2019 00:15:16 -0800 Subject: [PATCH 09/45] Fixing rat errors; Teamcity errors. --- .../moving-average-query/README.md | 19 +++++++++++++++++++ .../movingaverage/MovingAverageIterable.java | 8 ++++---- ...rg.apache.druid.initialization.DruidModule | 15 +++++++++++++++ .../movingaverage/MovingAverageQueryTest.java | 6 +++--- 4 files changed, 41 insertions(+), 7 deletions(-) diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md index b92a6dbce303..502f1e68304b 100644 --- a/extensions-contrib/moving-average-query/README.md +++ b/extensions-contrib/moving-average-query/README.md @@ -1,3 +1,22 @@ + + druid-moving-average-query ============= diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 463634a93e98..936a9525f84b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -104,7 +104,7 @@ public ColumnCapabilities getColumnCapabilities(String s) Aggregator aggFactorized = agg.factorize(colFact); fakeEvents.put(agg.getName(), aggFactorized.get()); }); - postAggMap.values().stream().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); + postAggMap.values().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); } /* (non-Javadoc) @@ -280,12 +280,12 @@ private Row computeMovingAverage(Map key, Row r, boolean skip) } if (!skip) { - avg.stream().forEach(af -> af.addElement(event, aggMap)); + avg.forEach(af -> af.addElement(event, aggMap)); } else { - avg.stream().forEach(af -> af.skip()); + avg.forEach(af -> af.skip()); } - avg.stream().forEach(af -> result.put(af.getName(), af.getResult())); + avg.forEach(af -> result.put(af.getName(), af.getResult())); // At least one non-dimension value must be in the record for it to be valid. if (result.entrySet().stream().anyMatch(e -> !key.containsKey(e.getKey()) && e.getValue() != null)) { diff --git a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 72876559838d..ec70e7d9c464 100644 --- a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -1 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF 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. + org.apache.druid.query.movingaverage.MovingAverageQueryModule diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index baf887063c61..d2ee24547160 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -123,7 +123,7 @@ public static Iterable data() throws IOException return tests; } - public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMappingException, IOException + public MovingAverageQueryTest(String yamlFile) throws IOException { List modules = getRequiredModules(); modules.add( @@ -238,7 +238,7 @@ protected List getRequiredModules() * @throws JsonMappingException * @throws JsonParseException */ - protected void defineMocks(Query query) throws JsonParseException, JsonMappingException, IOException + protected void defineMocks(Query query) throws IOException { groupByResults.clear(); timeseriesResults.clear(); @@ -320,7 +320,7 @@ protected void consistentTypeCasting(List result) */ @SuppressWarnings({"unchecked", "rawtypes"}) @Test - public void testQuery() throws JsonParseException, JsonMappingException, IOException + public void testQuery() throws IOException { From 9591a9dea471010dd52365ceea8bfc6743dcf4ec Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 25 Feb 2019 12:00:20 -0800 Subject: [PATCH 10/45] Removing support theta sketches. Will be added back in this pr or a following once DI conflicts with datasketches are resolved. --- .../moving-average-query.md | 20 ----- .../moving-average-query/pom.xml | 6 -- .../averagers/AveragerFactory.java | 3 +- .../averagers/SketchUnionAverager.java | 57 ------------- .../averagers/SketchUnionAveragerFactory.java | 81 ------------------ .../movingaverage/MovingAverageQueryTest.java | 2 - .../SketchUnionAveragerFactoryTest.java | 37 --------- .../averagers/SketchUnionAveragerTest.java | 82 ------------------- .../queryTests/missingGroupByValues.yaml | 17 ++-- .../queryTests/movingAverageWithSketches.yaml | 70 ---------------- 10 files changed, 9 insertions(+), 366 deletions(-) delete mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java delete mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java delete mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java delete mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java delete mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 804b4fdb6866..e8b4294154fb 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -106,8 +106,6 @@ These are properties which are common to all Averagers: * longMeanNoNulls * longMax * longMin -* [Sketch averagers](#sketch-averagers): - * sketchUnion #### Standard averagers @@ -127,24 +125,6 @@ Example of usage: { "type" : "doubleMean", "name" : , "fieldName": } ``` -#### Sketch averagers - -Sketch averager are meant to perform merge operations on [DataSketches](../extensions-core/datasketches-extension.html) (When using a Sketch averager, please include the [DataSketches](../extensions-core/datasketches-extension.html) extension as well). - -Extra properties for Sketch averagers: - -|property|description|required?| -|--------|-----------|---------| -|size|Sketch size; See [DataSketches aggregator](../extensions-core/datasketches-aggregators.html)|no, defaults to 4096| - -Available functions: -* sketchUnion - -Example of usage: -```json -{ "type" : "sketchUnion", "name" : , "fieldName": } -``` - ### Cycle size (Day of Week) This optional parameter is used to calculate over a single bucket within each cycle instead of all buckets. A prime example would be weekly buckets, resulting in a Day of Week calculation. (Other examples: Month of year, Hour of day). diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index a7e76d1bca97..416e34c6174b 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -73,12 +73,6 @@ ${project.parent.version} provided - - org.apache.druid.extensions - druid-datasketches - ${project.parent.version} - provided - org.apache.druid druid-server diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java index e2041b0db40a..a3c82781b233 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -41,8 +41,7 @@ @JsonSubTypes.Type(name = "longMean", value = LongMeanAveragerFactory.class), @JsonSubTypes.Type(name = "longMeanNoNulls", value = LongMeanNoNullAveragerFactory.class), @JsonSubTypes.Type(name = "longMax", value = LongMaxAveragerFactory.class), - @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class), - @JsonSubTypes.Type(name = "sketchUnion", value = SketchUnionAveragerFactory.class) + @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class) }) public interface AveragerFactory { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java deleted file mode 100644 index 12ce0bca3d9a..000000000000 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.yahoo.sketches.Family; -import com.yahoo.sketches.theta.SetOperation; -import com.yahoo.sketches.theta.SetOperationBuilder; -import com.yahoo.sketches.theta.Union; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; - -public class SketchUnionAverager extends BaseAverager -{ - - private int startFrom = 0; - private final SetOperationBuilder builder; - - public SketchUnionAverager(int numBuckets, String name, String fieldName, int cycleSize, int size) - { - super(SketchHolder.class, numBuckets, name, fieldName, cycleSize, false); - this.builder = SetOperation.builder().setNominalEntries(size); - } - - @Override - protected SketchHolder computeResult() - { - int cycleSize = getCycleSize(); - Object[] obj = getBuckets(); - int numBuckets = getNumBuckets(); - - Union union = (Union) builder.build(Family.UNION); - for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - ((SketchHolder) obj[(i + startFrom) % numBuckets]).updateUnion(union); - } - } - - startFrom++; - return SketchHolder.of(union.getResult()); - } -} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java deleted file mode 100644 index 5e687ba3a2e7..000000000000 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.yahoo.sketches.Util; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; - -import java.util.Comparator; - -public class SketchUnionAveragerFactory extends BaseAveragerFactory -{ - - private final int size; - - @JsonCreator - public SketchUnionAveragerFactory( - @JsonProperty("name") String name, - @JsonProperty("buckets") int numBuckets, - @JsonProperty("cycleSize") Integer cycleSize, - @JsonProperty("fieldName") String fieldName, - @JsonProperty(value = "size", required = false) Integer size - ) - { - super(name, numBuckets, fieldName, cycleSize); - this.size = size == null ? Util.DEFAULT_NOMINAL_ENTRIES : size; - } - - public SketchUnionAveragerFactory( - String name, - int numBuckets, - Integer cycleSize, - String fieldName - ) - { - this(name, numBuckets, cycleSize, fieldName, null); - } - - @Override - public Averager createAverager() - { - return new SketchUnionAverager(numBuckets, name, fieldName, cycleSize, size); - } - - /* (non-Javadoc) - * @see AveragerFactory#finalize(java.lang.Object) - */ - @Override - public Double finalizeComputation(SketchHolder val) - { - return val.getSketch().getEstimate(); - } - - /* (non-Javadoc) - * @see BaseAveragerFactory#getComparator() - */ - @SuppressWarnings({"rawtypes", "unchecked"}) - @Override - public Comparator getComparator() - { - return SketchHolder.COMPARATOR; - } -} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index d2ee24547160..9ee1d28ea693 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -62,7 +62,6 @@ import org.apache.druid.query.Result; import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.movingaverage.test.TestConfig; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -221,7 +220,6 @@ protected List getRequiredModules() { List list = new ArrayList<>(); - list.add(new SketchModule()); list.add(new QueryRunnerFactoryModule()); list.add(new QueryableModule()); list.add(new DruidProcessingModule()); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java deleted file mode 100644 index 540dc5ffccd9..000000000000 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import org.junit.Test; - -import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.junit.Assert.assertThat; - -public class SketchUnionAveragerFactoryTest -{ - - @Test - public void testCreateAverager() - { - AveragerFactory fac = new SketchUnionAveragerFactory("test", 5, 1, "field"); - assertThat(fac.createAverager(), instanceOf(SketchUnionAverager.class)); - } - -} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java deleted file mode 100644 index b70b7a9539ad..000000000000 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.yahoo.sketches.Family; -import com.yahoo.sketches.Util; -import com.yahoo.sketches.theta.SetOperation; -import com.yahoo.sketches.theta.Union; -import com.yahoo.sketches.theta.UpdateSketch; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; -import org.junit.Test; - -import java.util.Collections; -import java.util.HashMap; - -import static org.junit.Assert.assertEquals; - -public class SketchUnionAveragerTest -{ - - @Test - public void testComputeResult() throws Exception - { - BaseAverager avg = - new SketchUnionAverager(2, "test", "field", 1, Util.DEFAULT_NOMINAL_ENTRIES); - - UpdateSketch sketch1 = UpdateSketch.builder().build(); - for (int key = 0; key < 16; key++) { - sketch1.update(key); - } - - UpdateSketch sketch2 = UpdateSketch.builder().build(); - for (int key = 16; key < 32; key++) { - sketch2.update(key); - } - - UpdateSketch sketch3 = UpdateSketch.builder().build(); - for (int key = 32; key < 48; key++) { - sketch2.update(key); - } - - SketchHolder holder1 = SketchHolder.of(sketch1); - SketchHolder holder2 = SketchHolder.of(sketch2); - SketchHolder holder3 = SketchHolder.of(sketch3); - - Union union1 = (Union) SetOperation.builder().build(Family.UNION); - union1.update(sketch1); - union1.update(sketch2); - - avg.addElement(Collections.singletonMap("field", holder1), new HashMap<>()); - avg.addElement(Collections.singletonMap("field", holder2), new HashMap<>()); - - assertEquals(avg.computeResult().getEstimate(), union1.getResult().getEstimate(), 0); - - avg.addElement(Collections.singletonMap("field", holder3), new HashMap<>()); - - Union union2 = (Union) SetOperation.builder().build(Family.UNION); - union2.update(sketch2); - union2.update(sketch3); - - assertEquals(avg.computeResult().getEstimate(), union2.getResult().getEstimate(), 0); - - } - -} diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml index 626b94c23c7f..c4ab5a4fa8a3 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml @@ -21,10 +21,9 @@ query: - name: timespent_secs fieldName: timespent type: longSum - - name: regBcookies - fieldName: bcookies_with_yuid_sketch - size: 16384 - type: thetaSketch + - name: someSum + fieldName: someSum_field + type: doubleSum postAggregations: - type: arithmetic name: timeSpent @@ -45,7 +44,7 @@ expectedOutput: timespent_secs: 240 timeSpent: 4.0 trailing7DayAvgTimeSpent: 3.0 - regBcookies: 0.0 + someSum: 3.0 - version: v1 timestamp: 2017-01-02T00:00Z event: @@ -53,7 +52,7 @@ expectedOutput: timespent_secs: 0 timeSpent: 0.0 trailing7DayAvgTimeSpent: 1.0 - regBcookies: 0.0 + someSum: 0.0 intermediateResults: groupBy: - version: v1 @@ -62,18 +61,18 @@ intermediateResults: gender: m timespent_secs: 120 timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 5.0 - version: v1 timestamp: 2017-01-01T00:00Z event: gender: f timespent_secs: 120 timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 2.0 - version: v1 timestamp: 2017-01-02T00:00Z event: gender: m timespent_secs: 240 timeSpent: 4.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 3.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml deleted file mode 100644 index 460de3f4cfd5..000000000000 --- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml +++ /dev/null @@ -1,70 +0,0 @@ -query: - queryType: movingAverage - dataSource: - type: table - name: slice_pr_pc_pl_us_os - context: { - } - granularity: - type: period - period: P1D - intervals: - - 2017-01-01T00:00Z/2017-01-02T00:00Z - dimensions: - - gender - averagers: - - name: regBcookiesAvg - fieldName: regBcookies - size: 16384 - buckets: 2 - type: sketchUnion - aggregations: - - name: regBcookies - fieldName: bcookie_with_yuid_sketch - size: 16384 - type: thetaSketch - postAveragers: [ - ] - postAggregations: [ - ] -expectedOutput: -- version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: 0.0 - regBcookiesAvg: 0.0 -- version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: f - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: 0.0 - regBcookiesAvg: 0.0 -intermediateResults: - groupBy: - - version: v1 - timestamp: 2016-12-31T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - - version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - - version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: f - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - From 84e48dda437690c4fc3f7041a9a05ac001352283 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 25 Feb 2019 23:11:32 -0800 Subject: [PATCH 11/45] Implements some of the review fixes. --- .../moving-average-query.md | 24 ++++---- .../moving-average-query/README.md | 6 -- .../moving-average-query/pom.xml | 6 +- .../movingaverage/AveragerFactoryWrapper.java | 48 ++++++++------- .../movingaverage/MovingAverageIterable.java | 2 +- .../MovingAverageQueryRunner.java | 2 +- .../MovingAverageQueryToolChest.java | 6 +- ... => PostAveragerAggregatorCalculator.java} | 4 +- .../movingaverage/RowBucketIterable.java | 15 ++--- .../movingaverage/averagers/BaseAverager.java | 2 +- .../MovingAverageIterableTest.java | 1 - .../movingaverage/MovingAverageQueryTest.java | 3 +- ...PostAveragerAggregatorCalculatorTest.java} | 6 +- .../movingaverage/RowBucketIterableTest.java | 2 - .../averagers/BaseAveragerFactoryTest.java | 2 +- .../averagers/DoubleMaxAveragerTest.java | 2 +- .../averagers/DoubleMeanAveragerTest.java | 2 +- .../DoubleMeanAveragerWithPeriodTest.java | 2 +- .../averagers/LongMaxAveragerFactoryTest.java | 2 +- .../averagers/LongMeanAveragerTest.java | 2 +- .../src/test/resources/runtime.properties | 58 +------------------ 21 files changed, 65 insertions(+), 132 deletions(-) rename extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/{PostAveragerCalculator.java => PostAveragerAggregatorCalculator.java} (92%) rename extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/{PostAveragerCalculatorTest.java => PostAveragerAggregatorCalculatorTest.java} (95%) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index e8b4294154fb..bf5a0da48412 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -51,34 +51,34 @@ It runs the query in two main phases: ## Operations To use this extension, make sure to [load](../../operations/including-extensions.html) `druid-moving-average-query` only to the Broker. -##Configuration +## Configuration There are currently no configuration properties specific to Moving Average. -##Limitations +## Limitations * movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. * movingAverage is missing support for the following timeseries properties: `descending`. * movingAverage is missing support for [SQL-compatible null handling](https://github.com/apache/incubator-druid/issues/4349) (So setting druid.generic.useDefaultValueForNull in configuration will give an error). ##Query spec: -* Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. +* Most properties in the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. |property|description|required?| |--------|-----------|---------| -|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query|yes| -|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information |yes| -|dimensions|A JSON list of dimensions to do the groupBy over; can be ommitted for using timeseries back-engine; or see [DimensionSpec](../../querying/dimensionspecs.html) for ways to extract dimensions |no| -|limitSpec|See [LimitSpec](../../querying/limitspec.html).|no| -|having|See [Having](../../querying/having.html).|no| -|granularity|Granularity type must be:`period`; See [Granularities](../../querying/granularities.html)|yes| +|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query.|yes| +|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information.|yes| +|dimensions|A JSON list of [DimensionSpec](../../querying/dimensionspecs.html)|no| +|limitSpec|See [LimitSpec](../../querying/limitspec.html)|no| +|having|See [Having](../../querying/having.html)|no| +|granularity|A period granilarity; See [Period Granularities](../../querying/granularities.html#period-granularities)|yes| |filter|See [Filters](../../querying/filters.html)|no| |aggregations|Aggregations forms the input to Averagers; See [Aggregations](../../querying/aggregations.html)|yes| |postAggregations|Supports only aggregations as input; See [Post Aggregations](../../querying/post-aggregations.html)|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| -|averagers|Defines the moving average function; See [Aggregations](../../querying/aggregations.html)|yes| +|averagers|Defines the moving average function; See [Averagers](#averagers)|yes| |postAveragers|Support input of both averagers and aggregations; Syntax is identical to postAggregations (See [Post Aggregations](../../querying/post-aggregations.html))|no| -##Averagers +## Averagers Averagers are used to define the Moving-Average function. Averagers are not limited to an average - they can also provide other types of window functions such as MAX()/MIN(). @@ -95,7 +95,7 @@ These are properties which are common to all Averagers: |cycleSize|Cycle size; Used to calculate day-of-week option; See [Cycle size (Day of Week)](#cycle-size-day-of-week)|no, defaults to 1| -###Averager types: +### Averager types: * [Standard averagers](#standard-averagers): * doubleMean diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md index 502f1e68304b..33156e74a055 100644 --- a/extensions-contrib/moving-average-query/README.md +++ b/extensions-contrib/moving-average-query/README.md @@ -27,9 +27,3 @@ Overview Documentation ============= See the druid.io website or under [Druid Github Repo](https://github.com/apache/incubator-druid/tree/master/docs/content/development/extensions-contrib/moving-average-query.md). - -Thanks -=========== - -This module was created by [Will Lauer](https://github.com/will-lauer) and [Mangesh Pardeshi](https://github.com/mangesh-pardeshi), with additional contributions by: [Deepak Babu](https://github.com/deepakb91), [Asif Mansoor](https://github.com/asifmansoora), [Ke Wang](https://github.com/kenuiuc), [Eyal Yurman](https://github.com/yurmix). -Thanks to all contributors! diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index 416e34c6174b..ae6f68a3b881 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.15.0-incubating-SNAPSHOT ../../pom.xml @@ -76,8 +76,8 @@ org.apache.druid druid-server - 0.13.0-incubating-SNAPSHOT - compile + ${project.parent.version} + provided diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 198f898d9f18..f6f1d90b4f65 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -33,6 +33,10 @@ * This is necessary purely to allow existing common druid code that only knows * about aggregators to work with the MovingAverageQuery query as well. * + * NOTE: The {@link AggregatorFactory} abstract class is only partially extended. + * Most methods are not implemented and throw {@link UnsupportedOperationException} if called. + * This is becsuse these methods are invalid for the AveragerFactoryWrapper. + * * @param Result type * @param Finalized Result type */ @@ -55,21 +59,21 @@ public AveragerFactoryWrapper(AveragerFactory af, String prefix) } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(ColumnSelectorFactory metricFactory) throws UnsupportedOperationException { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /* (non-Javadoc) @@ -82,43 +86,43 @@ public Comparator getComparator() } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public Object combine(Object lhs, Object rhs) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public AggregatorFactory getCombiningFactory() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public List getRequiredColumns() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public Object deserialize(Object object) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @SuppressWarnings("unchecked") @Override @@ -137,39 +141,39 @@ public String getName() } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public List requiredFields() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public byte[] getCacheKey() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public String getTypeName() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public int getMaxIntermediateSize() { - return 0; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 936a9525f84b..be919dd4dd44 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -195,7 +195,7 @@ private Row internalNext() cacheIter = cache.getRows().iterator(); } - Row r = null; + Row r; // return rows from the cached RowBucket if (cacheIter != null) { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 0328f4b029f5..64a503cfd97d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -207,7 +207,7 @@ public Sequence run(QueryPlus query, Map responseConte // Apply any postAveragers Sequence movingAvgResultsWithPostAveragers = - Sequences.map(movingAvgResults, new PostAveragerCalculator(maq)); + Sequences.map(movingAvgResults, new PostAveragerAggregatorCalculator(maq)); // remove rows outside the reporting window List reportingIntervals = maq.getIntervals(); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index ae293f440a8b..9164c30a039b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -49,8 +49,6 @@ public class MovingAverageQueryToolChest extends QueryToolChest makePostComputeManipulatorFn(MovingAverageQuery query, public Row apply(Row result) { MapBasedRow mRow = (MapBasedRow) result; - final Map values = new HashMap(mRow.getEvent()); + final Map values = new HashMap<>(mRow.getEvent()); for (AggregatorFactory agg : query.getAggregatorSpecs()) { Object aggVal = values.get(agg.getName()); @@ -114,7 +112,7 @@ public Row apply(Row result) for (AveragerFactory avg : query.getAveragerSpecs()) { Object aggVal = values.get(avg.getName()); if (aggVal != null) { - values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper(avg, avg.getName() + "_"), aggVal)); + values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper<>(avg, avg.getName() + "_"), aggVal)); } else { values.put(avg.getName(), null); } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java similarity index 92% rename from extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java rename to extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java index 3490f3e4192c..5031330badaa 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java @@ -30,12 +30,12 @@ /** * Function that can be applied to a Sequence to calculate PostAverager results */ -public class PostAveragerCalculator implements Function +public class PostAveragerAggregatorCalculator implements Function { private final List postAveragers; - public PostAveragerCalculator(MovingAverageQuery maq) + public PostAveragerAggregatorCalculator(MovingAverageQuery maq) { this.postAveragers = maq.getPostAveragerSpecs(); } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 1e1fda717215..688eb1943302 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -55,13 +55,12 @@ public RowBucketIterable(Sequence seq, List intervals, Period per @Override public Iterator iterator() { - return new RowIterator(seq, intervals, period); + return new RowBucketIterator(seq, intervals, period); } - static class RowIterator implements Iterator + static class RowBucketIterator implements Iterator { private Yielder yielder; - private boolean done = false; private DateTime endTime; private DateTime expectedBucket; private Period period; @@ -70,7 +69,7 @@ static class RowIterator implements Iterator private boolean processedLastRow = false; private boolean processedExtraRow = false; - public RowIterator(Sequence rows, List intervals, Period period) + public RowBucketIterator(Sequence rows, List intervals, Period period) { this.period = period; this.intervals = intervals; @@ -85,11 +84,7 @@ public RowIterator(Sequence rows, List intervals, Period period) @Override public boolean hasNext() { - // expectedBucket < endTime - if (expectedBucket.compareTo(endTime) < 0) { - return true; - } - return false; + return expectedBucket.compareTo(endTime) < 0; } /* (non-Javadoc) @@ -102,7 +97,7 @@ public RowBucket next() if (expectedBucket.compareTo(intervals.get(intervalIndex).getEnd()) >= 0) { intervalIndex++; - if (intervalIndex <= intervals.size()) { + if (intervalIndex < intervals.size()) { expectedBucket = intervals.get(intervalIndex).getStart(); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index 37497c695ead..00fee22216ea 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -176,7 +176,7 @@ public int getCycleSize() */ protected I[] getBuckets() { - return (I[]) buckets; + return buckets; } /** diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index a1c7e9acc0f5..39c13604e373 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -66,7 +66,6 @@ public class MovingAverageIterableTest private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final String GENDER = "gender"; private static final String AGE = "age"; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 9ee1d28ea693..1169653dd1b9 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -82,6 +82,7 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -429,7 +430,7 @@ public void emit(Event event) {} }, baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() ); - final Map responseContext = new ConcurrentHashMap<>(); + final Map responseContext = new HashMap<>(); defineMocks(query); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java similarity index 95% rename from extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java rename to extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java index d961bc47059e..51c707749f18 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java @@ -46,11 +46,11 @@ /** * Unit tests for PostAveragerCalcualtor */ -public class PostAveragerCalculatorTest +public class PostAveragerAggregatorCalculatorTest { private MovingAverageQuery query; - private PostAveragerCalculator pac; + private PostAveragerAggregatorCalculator pac; private Map event; private MapBasedRow row; @@ -80,7 +80,7 @@ public void setup() null ); - pac = new PostAveragerCalculator(query); + pac = new PostAveragerAggregatorCalculator(query); event = new HashMap<>(); row = new MapBasedRow(new DateTime(ISOChronology.getInstanceUTC()), event); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 6633d0068a85..0dcfe1c6e4d1 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -50,8 +50,6 @@ public class RowBucketIterableTest private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_8 = new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_9 = new DateTime(2017, 1, 9, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final Map EVENT_M_10 = new HashMap<>(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java index 9956a7065f3d..98104e675b04 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -52,7 +52,7 @@ public Comparator getComparator() } @Test - public void testGetDependentFields() throws Exception + public void testGetDependentFields() { List dependentFields = fac.getDependentFields(); assertEquals(1, dependentFields.size()); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java index 496063355743..e1ba10fad193 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -30,7 +30,7 @@ public class DoubleMaxAveragerTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager avg = new DoubleMaxAverager(3, "test", "field", 1); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java index a0840a8d33f7..0d5f2c7cc8f5 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -30,7 +30,7 @@ public class DoubleMeanAveragerTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager avg = new DoubleMeanAverager(3, "test", "field", 1); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java index 49c73b34c5ca..8cde307c88bf 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -30,7 +30,7 @@ public class DoubleMeanAveragerWithPeriodTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager averager = new DoubleMeanAverager(14, "test", "field", 7); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java index 7246e8bd006e..7601a5d49156 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -28,7 +28,7 @@ public class LongMaxAveragerFactoryTest { @Test - public void testCreateAverager() throws Exception + public void testCreateAverager() { AveragerFactory fac = new LongMaxAveragerFactory("test", 5, 1, "field"); assertThat(fac.createAverager(), instanceOf(LongMaxAverager.class)); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java index c34d1ec2e612..cb037a233148 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -42,7 +42,7 @@ public void testComputeResult() avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); assertEquals(2.0, avg.computeResult(), 0.0); - avg.addElement(Collections.singletonMap("field", new Integer(3)), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3), new HashMap<>()); assertEquals(3.0, avg.computeResult(), 0.0); avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index 1c6b09b91b36..1e395f3164e0 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -1,58 +1,2 @@ -druid.broker.cache.unCacheable=["groupBy","segmentMetadata"] -druid.broker.http.numConnections=50 -druid.server.http.numThreads=1 -druid.announcer.maxBytesPerNode=524288 -druid.announcer.segmentsPerNode=50 -druid.announcer.type=batch -druid.broker.balancer.type=random -druid.broker.cache.initialSize=500000 -druid.broker.cache.logEvictionCount=0 -druid.broker.cache.sizeInBytes=0 -druid.broker.cache.type=local -druid.broker.http.readTimeOut=PT15M -druid.broker.select.tier=highestPriority -druid.broker.select.tier.custom.priorities= -druid.coordinator.conversion.on=false -druid.coordinator.load.timeout=PT15M -druid.coordinator.merge.on=false -druid.coordinator.period=PT30S -druid.coordinator.period.indexingPeriod=PT1800S -druid.coordinator.startDelay=PT60S -druid.curator.compress=false -druid.emitter.logging.loggerClass=LoggingEmitter -druid.emitter.logging.logLevel=info -druid.extensions.coordinates=[] -druid.manager.config.pollDuration=PT1M -druid.manager.rules.defaultTier=tier1 -druid.manager.rules.pollDuration=PT1M -druid.manager.segment.pollDuration=PT1M -druid.monitoring.emissionPeriod=PT1M -druid.monitoring.monitors=[\"com.metamx.metrics.JvmMonitor\"] druid.processing.buffer.sizeBytes=655360 -druid.processing.numThreads=1 -druid.processing.columnCache.sizeBytes=0 -druid.processing.formatString='processing-%s' -druid.publish.type=metadata -druid.query.chunkPeriod=P1M -druid.query.groupBy.defaultStrategy=v1 -druid.query.groupBy.maxIntermediateRows=50000 -druid.query.groupBy.maxResults=5000000 -druid.query.groupBy.singleThreaded=false -druid.query.search.maxSearchLimit=1000 -druid.request.logging.feed= -druid.request.logging.dir= -druid.request.logging.type=noop -druid.segmentCache.announceIntervalMillis=5000 -druid.segmentCache.deleteOnRemove=true -druid.segmentCache.dropSegmentDelayMillis=30000 -druid.segmentCache.numLoadingThreads=1 -druid.selectors.indexing.serviceName=overlord -druid.server.http.maxIdleTime=PT5m -druid.storage.type=hdfs -druid.worker.capacity=30 -druid.server.maxSize=1073741824 -druid.server.priority=0 -druid.server.tier=_default_tier -druid.worker.ip=localhost -druid.worker.version=0 -druid.request.logging.type=slf4j +druid.request.logging.type=slf4j \ No newline at end of file From 0d24960097853a38fe9d2045f66b357f6b752e38 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 8 Oct 2018 14:22:50 -0700 Subject: [PATCH 12/45] Contributing Moving-Average Query to open source. --- distribution/pom.xml | 3 +- .../moving-average-query.md | 335 ++++++++ docs/content/development/extensions.md | 1 + .../moving-average-query/README.md | 16 + .../moving-average-query/pom.xml | 89 ++ .../movingaverage/AveragerFactoryWrapper.java | 174 ++++ .../movingaverage/BucketingAccumulator.java | 65 ++ .../DefaultMovingAverageQueryMetrics.java | 61 ++ ...faultMovingAverageQueryMetricsFactory.java | 54 ++ .../IdentityYieldingAccumulator.java | 39 + .../movingaverage/MovingAverageHelper.java | 53 ++ .../movingaverage/MovingAverageIterable.java | 302 +++++++ .../movingaverage/MovingAverageQuery.java | 372 ++++++++ .../MovingAverageQueryMetrics.java | 41 + .../MovingAverageQueryMetricsFactory.java | 35 + .../MovingAverageQueryModule.java | 60 ++ .../MovingAverageQueryRunner.java | 238 ++++++ .../MovingAverageQueryToolChest.java | 146 ++++ .../movingaverage/PostAveragerCalculator.java | 59 ++ .../druid/query/movingaverage/RowBucket.java | 61 ++ .../movingaverage/RowBucketIterable.java | 152 ++++ .../movingaverage/averagers/Averager.java | 57 ++ .../averagers/AveragerFactory.java | 106 +++ .../movingaverage/averagers/BaseAverager.java | 199 +++++ .../averagers/BaseAveragerFactory.java | 102 +++ .../averagers/ComparableAveragerFactory.java | 53 ++ .../averagers/ConstantAverager.java | 80 ++ .../averagers/ConstantAveragerFactory.java | 101 +++ .../averagers/DoubleMaxAverager.java | 48 ++ .../averagers/DoubleMaxAveragerFactory.java | 43 + .../averagers/DoubleMeanAverager.java | 52 ++ .../averagers/DoubleMeanAveragerFactory.java | 43 + .../averagers/DoubleMeanNoNullAverager.java | 50 ++ .../DoubleMeanNoNullAveragerFactory.java | 42 + .../averagers/DoubleMinAverager.java | 48 ++ .../averagers/DoubleMinAveragerFactory.java | 42 + .../averagers/LongMaxAverager.java | 48 ++ .../averagers/LongMaxAveragerFactory.java | 42 + .../averagers/LongMeanAverager.java | 52 ++ .../averagers/LongMeanAveragerFactory.java | 43 + .../averagers/LongMeanNoNullAverager.java | 50 ++ .../LongMeanNoNullAveragerFactory.java | 43 + .../averagers/LongMinAverager.java | 49 ++ .../averagers/LongMinAveragerFactory.java | 43 + .../averagers/SketchUnionAverager.java | 56 ++ .../averagers/SketchUnionAveragerFactory.java | 80 ++ ...rg.apache.druid.initialization.DruidModule | 1 + .../IdentityYieldingAccumulatorTest.java | 59 ++ .../MovingAverageIterableTest.java | 802 ++++++++++++++++++ .../movingaverage/MovingAverageQueryTest.java | 442 ++++++++++ .../PostAveragerCalculatorTest.java | 108 +++ .../movingaverage/RowBucketIterableTest.java | 672 +++++++++++++++ .../averagers/BaseAveragerFactoryTest.java | 67 ++ .../averagers/BaseAveragerTest.java | 155 ++++ .../DoubleMaxAveragerFactoryTest.java | 37 + .../averagers/DoubleMaxAveragerTest.java | 56 ++ .../DoubleMeanAveragerFactoryTest.java | 36 + .../averagers/DoubleMeanAveragerTest.java | 57 ++ .../DoubleMeanAveragerWithPeriodTest.java | 80 ++ .../DoubleMeanNoNullAveragerFactoryTest.java | 36 + .../DoubleMeanNoNullAveragerTest.java | 81 ++ .../DoubleMinAveragerFactoryTest.java | 36 + .../averagers/DoubleMinAveragerTest.java | 57 ++ .../averagers/LongMaxAveragerFactoryTest.java | 36 + .../averagers/LongMaxAveragerTest.java | 56 ++ .../LongMeanAveragerFactoryTest.java | 36 + .../averagers/LongMeanAveragerTest.java | 56 ++ .../LongMeanNoNullAveragerFactoryTest.java | 36 + .../averagers/LongMeanNoNullAveragerTest.java | 56 ++ .../averagers/LongMinAveragerFactoryTest.java | 36 + .../averagers/LongMinAveragerTest.java | 57 ++ .../SketchUnionAveragerFactoryTest.java | 36 + .../averagers/SketchUnionAveragerTest.java | 81 ++ .../query/movingaverage/test/TestConfig.java | 34 + .../queryTests/basicGroupByMovingAverage.yaml | 57 ++ .../basicGroupByMovingAverage2.yaml | 57 ++ .../basicTimeseriesMovingAverage.yaml | 51 ++ .../queryTests/missingGroupByValues.yaml | 79 ++ .../queryTests/movingAverageWithSketches.yaml | 70 ++ .../queryTests/sortingAveragersAsc.yaml | 81 ++ .../queryTests/sortingAveragersDesc.yaml | 82 ++ ...ortingWithNonMovingAndMovingAvgMetric.yaml | 84 ++ .../sortingWithNonMovingAvgMetric.yaml | 82 ++ .../src/test/resources/runtime.properties | 57 ++ pom.xml | 3 +- 85 files changed, 7931 insertions(+), 2 deletions(-) create mode 100644 docs/content/development/extensions-contrib/moving-average-query.md create mode 100644 extensions-contrib/moving-average-query/README.md create mode 100644 extensions-contrib/moving-average-query/pom.xml create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java create mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java create mode 100644 extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java create mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml create mode 100644 extensions-contrib/moving-average-query/src/test/resources/runtime.properties diff --git a/distribution/pom.xml b/distribution/pom.xml index 89df39f69b3b..0a5f34c59ce1 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -310,7 +310,8 @@ org.apache.druid.extensions.contrib:druid-time-min-max -c org.apache.druid.extensions.contrib:druid-virtual-columns - + -c + org.apache.druid.extensions.contrib:druid-moving-average-query diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md new file mode 100644 index 000000000000..387b397ae991 --- /dev/null +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -0,0 +1,335 @@ +--- +layout: doc_page +--- + +# Moving Average Queries + +## Overview +**Moving Average Query** is an extension which provides support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries. + +These Aggregate Window Functions consume standard Druid Aggregators and outputs additional windowed aggregates called [Averagers](#averagers). + +#### High level algorithm + +Moving Average encapsulates the [groupBy query](../../querying/groupbyquery.html) (Or [timeseries](../../querying/timeseriesquery.html) in case of no dimensions) in order to rely on the maturity of these query types. + +It runs the query in two main phases: +1. Runs an inner [groupBy](../../querying/groupbyquery.html) or [timeseries](../../querying/timeseriesquery.html) query to compute Aggregators (i.e. daily count of events). +2. Passes over aggregated results in Broker, in order to compute Averagers (i.e. moving 7 day average of the daily count). + +#### Main enhancements provided by this extension: +1. Functionality: Extending druid query functionality (i.e. initial introduction of Window Functions). +2. Performance: Improving performance of such moving aggregations by eliminating multiple segment scans. + +#### Further reading +[Moving Average](https://en.wikipedia.org/wiki/Moving_average) + +[Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) + +[Analytic Functions](https://cloud.google.com/bigquery/docs/reference/standard-sql/analytic-function-concepts) + + +## Operations +To use this extension, make sure to [load](../../operations/including-extensions.html) `druid-moving-average-query` only to the Broker. + +##Configuration +There are currently no configuration properties specific to Moving Average. + +##Query spec: +* Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. +* Note: movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. +* Note: movingAverage is missing support for the following timeseries properties: `descending`. + +|property|description|required?| +|--------|-----------|---------| +|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query|yes| +|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information |yes| +|dimensions|A JSON list of dimensions to do the groupBy over; can be ommitted for using timeseries back-engine; or see [DimensionSpec](../../querying/dimensionspecs.html) for ways to extract dimensions |no| +|limitSpec|See [LimitSpec](../../querying/limitspec.html).|no| +|having|See [Having](../../querying/having.html).|no| +|granularity|Granularity type must be:`period`; See [Granularities](../../querying/granularities.html)|yes| +|filter|See [Filters](../../querying/filters.html)|no| +|aggregations|Aggregations forms the input to Averagers; See [Aggregations](../../querying/aggregations.html)|yes| +|postAggregations|Supports only aggregations as input; See [Post Aggregations](../../querying/post-aggregations.html)|no| +|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| +|context|An additional JSON Object which can be used to specify certain flags.|no| +|averagers|Defines the moving average function; See [Aggregations](../../querying/aggregations.html)|yes| +|postAveragers|Support input of both averagers and aggregations; Syntax is identical to postAggregations (See [Post Aggregations](../../querying/post-aggregations.html))|no| + +##Averagers + +Averagers are used to define the Moving-Average function. Averagers are not limited to an average - they can also provide other types of window functions such as MAX()/MIN(). + +### Properties + +These are properties which are common to all Averagers: + +|property|description|required?| +|--------|-----------|---------| +|type|Averager type; See [Averager types](#averager-types)|yes| +|name|Averager name|yes| +|fieldName|Input name (An aggregation name)|yes| +|buckets|Number of lookback buckets (time periods), including current one. Must be >0|yes| +|cycleSize|Cycle size; Used to calculate day-of-week option; See [Cycle size (Day of Week)](#cycle-size-day-of-week)|no, defaults to 1| + + +###Averager types: + +* [Standard averagers](#standard-averagers): + * doubleMean + * doubleMeanNoNulls + * doubleMax + * doubleMin + * longMean + * longMeanNoNulls + * longMax + * longMin +* [Sketch averagers](#sketch-averagers): + * sketchUnion + +#### Standard averagers + +These averagers offer four functions: +* Mean (Average) +* MeanNoNulls (Ignores empty buckets). +* Max +* Min + +**Ignoring nulls**: +Using a MeanNoNulls averager is useful when the interval starts at the dataset beginning time. +In that case, the first records will ignore missing buckets and average won't be artificially low. +However, this also means that empty days in a sparse dataset will also be ignored. + +Example of usage: +```json +{ "type" : "doubleMean", "name" : , "fieldName": } +``` + +#### Sketch averagers + +Sketch averager are meant to perform merge operations on [DataSketches](../extensions-core/datasketches-extension.html) (When using a Sketch averager, please include the [DataSketches](../extensions-core/datasketches-extension.html) extension as well). + +Extra properties for Sketch averagers: + +|property|description|required?| +|--------|-----------|---------| +|size|Sketch size; See [DataSketches aggregator](../extensions-core/datasketches-aggregators.html)|no, defaults to 4096| + +Available functions: +* sketchUnion + +Example of usage: +```json +{ "type" : "sketchUnion", "name" : , "fieldName": } +``` + +### Cycle size (Day of Week) +This optional parameter is used to calculate over a single bucket within each cycle instead of all buckets. +A prime example would be weekly buckets, resulting in a Day of Week calculation. (Other examples: Month of year, Hour of day). + +I.e. when using these parameters: +* *granularity*: period=P1D (daily) +* *buckets*: 28 +* *cycleSize*: 7 + +Within each output record, the averager will compute the result over the following buckets: current (#0), #7, #14, #21. +Whereas without specifying cycleSize it would have computed over all 28 buckets. + +## Examples + +All examples are based on the Wikipedia dataset provided in the Druid [tutorials](../../tutorials/index.html). + +### Basic example + +Calculating a 7-buckets moving average for Wikipedia edit deltas. + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT30M" + }, + "intervals": [ + "2015-09-12T00:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta30Min", + "fieldName": "delta", + "type": "longSum" + } + ], + "averagers": [ + { + "name": "trailing30MinChanges", + "fieldName": "delta30Min", + "type": "longMean", + "buckets": 7 + } + ] +} +``` + +Result: +```json +[ { + "version" : "v1", + "timestamp" : "2015-09-12T00:30:00.000Z", + "event" : { + "delta30Min" : 30490, + "trailing30MinChanges" : 4355.714285714285 + } + }, { + "version" : "v1", + "timestamp" : "2015-09-12T01:00:00.000Z", + "event" : { + "delta30Min" : 96526, + "trailing30MinChanges" : 18145.14285714286 + } + }, { +... +... +... +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:00:00.000Z", + "event" : { + "delta30Min" : 119100, + "trailing30MinChanges" : 198697.2857142857 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:30:00.000Z", + "event" : { + "delta30Min" : 177882, + "trailing30MinChanges" : 193890.0 + } +} +``` + +### Post averager example + +Calculating a 7-buckets moving average for Wikipedia edit deltas, plus a ratio between the current period and the moving average. + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT30M" + }, + "intervals": [ + "2015-09-12T22:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta30Min", + "fieldName": "delta", + "type": "longSum" + } + ], + "averagers": [ + { + "name": "trailing30MinChanges", + "fieldName": "delta30Min", + "type": "longMean", + "buckets": 7 + } + ], + "postAveragers" : [ + { + "name": "ratioTrailing30MinChanges", + "type": "arithmetic", + "fn": "/", + "fields": [ + { + "type": "fieldAccess", + "fieldName": "delta30Min" + }, + { + "type": "fieldAccess", + "fieldName": "trailing30MinChanges" + } + ] + } + ] +} +``` + +Result: +```json +[ { + "version" : "v1", + "timestamp" : "2015-09-12T22:00:00.000Z", + "event" : { + "delta30Min" : 144269, + "trailing30MinChanges" : 204088.14285714287, + "ratioTrailing30MinChanges" : 0.7068955500319539 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T22:30:00.000Z", + "event" : { + "delta30Min" : 242860, + "trailing30MinChanges" : 214031.57142857142, + "ratioTrailing30MinChanges" : 1.134692411867141 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:00:00.000Z", + "event" : { + "delta30Min" : 119100, + "trailing30MinChanges" : 198697.2857142857, + "ratioTrailing30MinChanges" : 0.5994042624782422 + } +}, { + "version" : "v1", + "timestamp" : "2015-09-12T23:30:00.000Z", + "event" : { + "delta30Min" : 177882, + "trailing30MinChanges" : 193890.0, + "ratioTrailing30MinChanges" : 0.9174377224199288 + } +} ] +``` + + +### Cycle size example + +Calculating an average of every first 10-minutes of the last 3 hours: + +Query syntax: +```json +{ + "queryType": "movingAverage", + "dataSource": "wikipedia", + "granularity": { + "type": "period", + "period": "PT10M" + }, + "intervals": [ + "2015-09-12T00:00:00Z/2015-09-13T00:00:00Z" + ], + "aggregations": [ + { + "name": "delta10Min", + "fieldName": "delta", + "type": "doubleSum" + } + ], + "averagers": [ + { + "name": "trailing10MinPerHourChanges", + "fieldName": "delta10Min", + "type": "doubleMeanNoNulls", + "buckets": 18, + "cycleSize": 6 + } + ] +} +``` diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index a6d3a7b3913d..206dacee709c 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -94,6 +94,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c |kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)| |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| |druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)| +|druid-moving-average-query|Support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries.|[link](../development/extensions-contrib/moving-average-query.html)| ## Promoting Community Extension to Core Extension diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md new file mode 100644 index 000000000000..b92a6dbce303 --- /dev/null +++ b/extensions-contrib/moving-average-query/README.md @@ -0,0 +1,16 @@ +druid-moving-average-query +============= + +Overview +============= +**Moving Average Query** is an extension which provides support for [Moving Average](https://en.wikipedia.org/wiki/Moving_average) and other Aggregate [Window Functions](https://en.wikibooks.org/wiki/Structured_Query_Language/Window_functions) in Druid queries. + +Documentation +============= +See the druid.io website or under [Druid Github Repo](https://github.com/apache/incubator-druid/tree/master/docs/content/development/extensions-contrib/moving-average-query.md). + +Thanks +=========== + +This module was created by [Will Lauer](https://github.com/will-lauer) and [Mangesh Pardeshi](https://github.com/mangesh-pardeshi), with additional contributions by: [Deepak Babu](https://github.com/deepakb91), [Asif Mansoor](https://github.com/asifmansoora), [Ke Wang](https://github.com/kenuiuc), [Eyal Yurman](https://github.com/yurmix). +Thanks to all contributors! diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml new file mode 100644 index 000000000000..e9c4e77671c0 --- /dev/null +++ b/extensions-contrib/moving-average-query/pom.xml @@ -0,0 +1,89 @@ + + + + 4.0.0 + + + + druid + org.apache.druid + 0.13.0-SNAPSHOT + ../../pom.xml + + + org.apache.druid.extensions.contrib + druid-moving-average-query + druid-moving-average-query + + + UTF-8 + + + + + org.jmockit + jmockit + 1.25 + test + + + junit + junit + test + + + org.hamcrest + hamcrest-library + 1.3 + test + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + 2.8.3 + test + + + org.apache.druid + druid-api + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid.extensions + druid-datasketches + ${project.parent.version} + provided + + + diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java new file mode 100644 index 000000000000..00b5982db946 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Comparator; +import java.util.List; + +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; + +/** + * A wrapper around averagers that makes them appear to be aggregators. + * This is necessary purely to allow existing common druid code that only knows + * about aggregators to work with the MovingAverageQuery query as well. + * + * @param Result type + * @param Finalized Result type + */ +public class AveragerFactoryWrapper extends AggregatorFactory +{ + + private final AveragerFactory af; + private final String prefix; + + /** + * Simple constructor + * + * @param af + * @param prefix + */ + public AveragerFactoryWrapper(AveragerFactory af, String prefix) + { + this.af = af; + this.prefix = prefix; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return null; + } + + /* (non-Javadoc) + * @see org.apache.druid.query.aggregation.AggregatorFactory#getComparator() + */ + @Override + public Comparator getComparator() + { + return af.getComparator(); + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Object combine(Object lhs, Object rhs) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public AggregatorFactory getCombiningFactory() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public List getRequiredColumns() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public Object deserialize(Object object) + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @SuppressWarnings("unchecked") + @Override + public Object finalizeComputation(Object object) + { + return af.finalizeComputation((T) object); + } + + /* (non-Javadoc) + * @see org.apache.druid.query.aggregation.AggregatorFactory#getName() + */ + @Override + public String getName() + { + return prefix + af.getName(); + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public List requiredFields() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public byte[] getCacheKey() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public String getTypeName() + { + return null; + } + + /** + * Returns null because Averagers aren't actually Aggregators + */ + @Override + public int getMaxIntermediateSize() + { + return 0; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java new file mode 100644 index 000000000000..34ec4353e65a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.YieldingAccumulator; + +/** + * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. + */ +public class BucketingAccumulator extends YieldingAccumulator +{ + + /* (non-Javadoc) + * @see YieldingAccumulator#accumulate(java.lang.Object, java.lang.Object) + */ + @Override + public RowBucket accumulate(RowBucket accumulated, Row in) + { + Collection rows; + + if (accumulated == null) { + // first row, initializing + rows = new ArrayList<>(); + accumulated = new RowBucket(in.getTimestamp(), rows); + } else if (accumulated.getNextBucket() != null) { + accumulated = accumulated.getNextBucket(); + } + + if (!accumulated.getDateTime().equals(in.getTimestamp())) { + // day change detected + rows = new ArrayList<>(); + rows.add(in); + RowBucket nextBucket = new RowBucket(in.getTimestamp(), rows); + accumulated.setNextBucket(nextBucket); + yield(); + } else { + // still on the same day + rows = accumulated.getRows(); + rows.add(in); + } + + return accumulated; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java new file mode 100644 index 000000000000..857e48201f2e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.query.DefaultQueryMetrics; +import org.apache.druid.query.DruidMetrics; + +public class DefaultMovingAverageQueryMetrics extends DefaultQueryMetrics implements + MovingAverageQueryMetrics +{ + + public DefaultMovingAverageQueryMetrics(ObjectMapper jsonMapper) + { + super(jsonMapper); + } + + @Override + public void query(MovingAverageQuery query) + { + super.query(query); + numDimensions(query); + numMetrics(query); + numComplexMetrics(query); + } + + @Override + public void numDimensions(MovingAverageQuery query) + { + setDimension("numDimensions", String.valueOf(query.getDimensions().size())); + } + + @Override + public void numMetrics(MovingAverageQuery query) + { + setDimension("numMetrics", String.valueOf(query.getAggregatorSpecs().size())); + } + + @Override + public void numComplexMetrics(MovingAverageQuery query) + { + int numComplexAggs = DruidMetrics.findNumComplexAggs(query.getAggregatorSpecs()); + setDimension("numComplexMetrics", String.valueOf(numComplexAggs)); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java new file mode 100644 index 000000000000..d494a2e48b9b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Inject; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.jackson.DefaultObjectMapper; + +@LazySingleton +public class DefaultMovingAverageQueryMetricsFactory implements MovingAverageQueryMetricsFactory +{ + + private static final MovingAverageQueryMetricsFactory INSTANCE = + new DefaultMovingAverageQueryMetricsFactory(new DefaultObjectMapper()); + + @VisibleForTesting + public static MovingAverageQueryMetricsFactory instance() + { + return INSTANCE; + } + + private final ObjectMapper jsonMapper; + + @Inject + public DefaultMovingAverageQueryMetricsFactory(@Json ObjectMapper jsonMapper) + { + this.jsonMapper = jsonMapper; + } + + @Override + public MovingAverageQueryMetrics makeMetrics() + { + return new DefaultMovingAverageQueryMetrics(jsonMapper); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java new file mode 100644 index 000000000000..b0250b0a5c7e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.java.util.common.guava.YieldingAccumulator; + +/** + * Simple yielding accumulator that yields each row. + */ +public class IdentityYieldingAccumulator extends YieldingAccumulator +{ + + /* (non-Javadoc) + * @see YieldingAccumulator#accumulate(java.lang.Object, java.lang.Object) + */ + @Override + public T accumulate(T accumulated, T in) + { + yield(); + return in; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java new file mode 100644 index 000000000000..4188af4cba7c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.dimension.DimensionSpec; + +public class MovingAverageHelper +{ + + /** + * @param dimensions A list of DimensionSpec in the specified in the query + * @param row The Row to be used for looking up dimension values + * + * @return A Map of dimension/value from the row + */ + + public static Map getDimKeyFromRow(Collection dimensions, Row row) + { + + Map key = new HashMap<>(); + Map event = ((MapBasedRow) row).getEvent(); + + for (DimensionSpec dimension : dimensions) { + key.put(dimension.getOutputName(), event.get(dimension.getOutputName())); + } + + return key; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java new file mode 100644 index 000000000000..1cf6aa25a51a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.movingaverage.averagers.Averager; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.NullDimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; + +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * {@link MovingAverageIterable} iterates over days {@link RowBucket}, producing rows for each dimension combination, + * filling in missing entries with "empty" rows so that the averaging buckets have enough data to operate on. + * It then computes the moving average on the buckets and returns the row. + * See {@link MovingAverageIterator#computeMovingAverage(Map, Row, boolean)} for more details. + */ +public class MovingAverageIterable implements Iterable +{ + + private final Sequence seq; + private final Collection dims; + private final Collection> factories; + private final Map postAggMap; + private final Map aggMap; + private final Map fakeEvents; + + public MovingAverageIterable( + Sequence buckets, + Collection dims, + Collection> factories, + List postAggList, + List aggList + ) + { + this.dims = dims; + this.factories = factories; + this.seq = buckets; + + postAggMap = postAggList.stream().collect(Collectors.toMap(postAgg -> postAgg.getName(), postAgg -> postAgg)); + aggMap = aggList.stream().collect(Collectors.toMap(agg -> agg.getName(), agg -> agg)); + + ColumnSelectorFactory colFact = new ColumnSelectorFactory() + { + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + // Generating empty records while aggregating on Filtered aggregators requires a dimension selector + // for initialization. This dimension selector is not actually used for generating values + return NullDimensionSelector.instance(); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String s) + { + return null; + } + + @Override + public ColumnCapabilities getColumnCapabilities(String s) + { + return null; + } + }; + // Fill in all the fake events + fakeEvents = new LinkedHashMap<>(); + aggMap.values().forEach(agg -> { + Aggregator aggFactorized = agg.factorize(colFact); + fakeEvents.put(agg.getName(), aggFactorized.get()); + }); + postAggMap.values().stream().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); + } + + /* (non-Javadoc) + * @see java.lang.Iterable#iterator() + */ + @Override + public Iterator iterator() + { + return new MovingAverageIterator(seq, dims, factories, fakeEvents, aggMap); + } + + static class MovingAverageIterator implements Iterator + { + + private final Collection dims; + private final Map, Collection>> averagers = new HashMap<>(); + private final Collection> factories; + + private Yielder yielder; + private RowBucket cache = null; + private Iterator cacheIter; + private Iterator> averagersKeysIter; + private Set> seenKeys = new HashSet<>(); + private Row saveNext; + private Map aggMap; + private Map fakeEvents; + + public MovingAverageIterator( + Sequence rows, + Collection dims, + Collection> factories, + Map fakeEvents, + Map aggMap + ) + { + this.dims = dims; + this.factories = factories; + this.fakeEvents = fakeEvents; + this.aggMap = aggMap; + + yielder = rows.toYielder(null, new IdentityYieldingAccumulator()); + } + + /* (non-Javadoc) + * @see java.util.Iterator#hasNext() + */ + @Override + public boolean hasNext() + { + + if (saveNext != null) { + return true; + } + + try { + saveNext = internalNext(); + return true; + } + catch (NoSuchElementException e) { + return false; + } + + + } + + /* (non-Javadoc) + * @see java.util.Iterator#next() + */ + @Override + public Row next() + { + + if (saveNext != null) { + Row retVal = saveNext; + saveNext = null; + return retVal; + } + + return internalNext(); + } + + private Row internalNext() + { + if (cache == null && !yielder.isDone()) { + cache = yielder.get(); + yielder = yielder.next(cache); + + cacheIter = cache.getRows().iterator(); + } + + Row r = null; + + // return rows from the cached RowBucket + if (cacheIter != null) { + if (cacheIter.hasNext()) { + r = cacheIter.next(); + Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); + seenKeys.add(key); + r = computeMovingAverage(key, r, false); + if (r != null) { + return r; + } else { + throw new NoSuchElementException(); + } + } else { + Set> averagerKeys = new HashSet<>(averagers.keySet()); + averagerKeys.removeAll(seenKeys); + averagersKeysIter = averagerKeys.iterator(); + cacheIter = null; + } + } + + // return fake rows for unseen dimension combinations + if (averagersKeysIter != null) { + while (averagersKeysIter.hasNext()) { + Map dims = averagersKeysIter.next(); + Map fakeEventsCopy = new HashMap<>(fakeEvents); + + dims.forEach((dim, value) -> { + fakeEventsCopy.put(dim, value); + }); + + r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), fakeEventsCopy), true); + if (r != null) { + return r; + } + } + + seenKeys.clear(); + averagersKeysIter = null; + cache = null; + } + + if (cacheIter == null && averagersKeysIter == null && yielder.isDone()) { + // we should never get here. For some reason, there is + // no more work to do, so continuing to iterate will infinite loop + throw new NoSuchElementException(); + } + + // nothing to do here, so move on to the next row + return internalNext(); + } + + /** + * Compute and add any moving average columns. + * + *

Normally, the row passed in will be added to all the {@link Averager}'s and then results pulled + * from each averager. If skip is true, then the incoming row is actually a dummy value due to + * no data being present for this dimension combination in the current bucket. When this happens, + * {@link Averager#skip()} should be called instead of {@link Averager#addElement(Map, Map)}()} to force proper + * decaying of the average values. + * + *

Usually, the contents of key will be contained by the row R being passed in, but in the case of a + * dummy row, its possible that the dimensions will be known but the row empty. Hence, the values are + * passed as two separate arguments. + * + * @param key The dimension set that this row applies to. + * @param r The Row to operate on + * @param skip Indicates whether skip or add should be called + * + * @return The updated row containing averager results, or null if no averagers computed a result + */ + private Row computeMovingAverage(Map key, Row r, boolean skip) + { + Map event = ((MapBasedRow) r).getEvent(); + Map result = new HashMap<>(event); + + Collection> avg = averagers.get(key); + + if (avg == null) { + avg = factories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); + averagers.put(key, avg); + } + + if (!skip) { + avg.stream().forEach(af -> af.addElement(event, aggMap)); + } else { + avg.stream().forEach(af -> af.skip()); + } + + avg.stream().forEach(af -> result.put(af.getName(), af.getResult())); + + // At least one non-dimension value must be in the record for it to be valid. + if (result.entrySet().stream().anyMatch(e -> !key.containsKey(e.getKey()) && e.getValue() != null)) { + result.putAll(event); + return new MapBasedRow(r.getTimestamp(), result); + } else { + // No averagers returned anything. All buckets must be empty. + // skip this row. + return null; + } + } + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java new file mode 100644 index 000000000000..52ddd83a230c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.groupby.having.HavingSpec; +import org.apache.druid.query.groupby.orderby.LimitSpec; +import org.apache.druid.query.groupby.orderby.NoopLimitSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Class that defines druid MovingAverage query fields + */ +@JsonTypeName("movingAverage") +public class MovingAverageQuery extends BaseQuery +{ + + public static final String MOVING_AVG_QUERY_TYPE = "movingAverage"; + public static final String CTX_KEY_SORT_BY_DIMS_FIRST = "sortByDimsFirst"; + + private final LimitSpec limitSpec; + private final HavingSpec havingSpec; + private final DimFilter dimFilter; + private final Function, Sequence> limitFn; + private final Granularity granularity; + private final List dimensions; + private final List aggregatorSpecs; + private final List postAggregatorSpecs; + private final List> averagerSpecs; + private final List postAveragerSpecs; + + @JsonCreator + public MovingAverageQuery( + @JsonProperty("dataSource") DataSource dataSource, + @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, + @JsonProperty("filter") DimFilter dimFilter, + @JsonProperty("granularity") Granularity granularity, + @JsonProperty("dimensions") List dimensions, + @JsonProperty("aggregations") List aggregatorSpecs, + @JsonProperty("postAggregations") List postAggregatorSpecs, + @JsonProperty("having") HavingSpec havingSpec, + @JsonProperty("averagers") List> averagerSpecs, + @JsonProperty("postAveragers") List postAveragerSpecs, + @JsonProperty("limitSpec") LimitSpec limitSpec, + @JsonProperty("context") Map context + ) + { + super(dataSource, querySegmentSpec, false, context); + + this.dimFilter = dimFilter; + this.granularity = granularity; + this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + for (DimensionSpec spec : this.dimensions) { + Preconditions.checkArgument(spec != null, "dimensions has null DimensionSpec"); + } + this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; + this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; + this.averagerSpecs = averagerSpecs == null ? ImmutableList.>of() : averagerSpecs; + this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; + this.havingSpec = havingSpec; + this.limitSpec = (limitSpec == null) ? NoopLimitSpec.INSTANCE : limitSpec; + + Preconditions.checkNotNull(this.granularity, "Must specify a granularity"); + + verifyOutputNames(this.dimensions, this.aggregatorSpecs, this.postAggregatorSpecs); + + // build combined list of aggregators and averagers so that limit spec building is happy + List combinedAggregatorSpecs = new ArrayList<>(); + combinedAggregatorSpecs.addAll(this.aggregatorSpecs); + for (AveragerFactory avg : this.averagerSpecs) { + combinedAggregatorSpecs.add(new AveragerFactoryWrapper(avg, "")); + } + + Function, Sequence> postProcFn = + this.limitSpec.build( + this.dimensions, + combinedAggregatorSpecs, + this.postAggregatorSpecs, + this.granularity, + getContextSortByDimsFirst() + ); + + if (havingSpec != null) { + postProcFn = Functions.compose( + postProcFn, + new Function, Sequence>() + { + @Override + public Sequence apply(Sequence input) + { + return Sequences.filter( + input, + new Predicate() + { + @Override + public boolean apply(Row input) + { + return MovingAverageQuery.this.havingSpec.eval(input); + } + } + ); + } + } + ); + } + + this.limitFn = postProcFn; + + } + + private static void verifyOutputNames( + List dimensions, + List aggregators, + List postAggregators + ) + { + + final Set outputNames = Sets.newHashSet(); + for (DimensionSpec dimension : dimensions) { + if (!outputNames.add(dimension.getOutputName())) { + throw new IAE("Duplicate output name[%s]", dimension.getOutputName()); + } + } + + for (AggregatorFactory aggregator : aggregators) { + if (!outputNames.add(aggregator.getName())) { + throw new IAE("Duplicate output name[%s]", aggregator.getName()); + } + } + + for (PostAggregator postAggregator : postAggregators) { + if (!outputNames.add(postAggregator.getName())) { + throw new IAE("Duplicate output name[%s]", postAggregator.getName()); + } + } + } + + /** + * A private constructor that avoids all of the various state checks. Used by the with*() methods where the checks + * have already passed in order for the object to exist. + */ + private MovingAverageQuery( + DataSource dataSource, + QuerySegmentSpec querySegmentSpec, + DimFilter dimFilter, + Granularity granularity, + List dimensions, + List aggregatorSpecs, + List> averagerSpecs, + List postAggregatorSpecs, + List postAveragerSpecs, + HavingSpec havingSpec, + LimitSpec orderBySpec, + Function, Sequence> limitFn, + Map context + ) + { + super(dataSource, querySegmentSpec, false, context); + + this.dimFilter = dimFilter; + this.granularity = granularity; + this.dimensions = dimensions; + this.aggregatorSpecs = aggregatorSpecs; + this.averagerSpecs = averagerSpecs; + this.postAggregatorSpecs = postAggregatorSpecs; + this.postAveragerSpecs = postAveragerSpecs; + this.havingSpec = havingSpec; + this.limitSpec = orderBySpec; + this.limitFn = limitFn; + } + + @Override + public boolean hasFilters() + { + return dimFilter != null; + } + + @Override + public String getType() + { + return MOVING_AVG_QUERY_TYPE; + } + + @JsonIgnore + public boolean getContextSortByDimsFirst() + { + return getContextBoolean(CTX_KEY_SORT_BY_DIMS_FIRST, false); + } + + @JsonProperty + public DimFilter getFilter() + { + return dimFilter; + } + + @JsonProperty + public Granularity getGranularity() + { + return granularity; + } + + @JsonProperty + public List getDimensions() + { + return dimensions; + } + + @JsonProperty("aggregations") + public List getAggregatorSpecs() + { + return aggregatorSpecs; + } + + @JsonProperty("averagers") + public List> getAveragerSpecs() + { + return averagerSpecs; + } + + @JsonProperty("postAggregations") + public List getPostAggregatorSpecs() + { + return postAggregatorSpecs; + } + + @JsonProperty("postAveragers") + public List getPostAveragerSpecs() + { + return postAveragerSpecs; + } + + @JsonProperty("having") + public HavingSpec getHavingSpec() + { + return havingSpec; + } + + @JsonProperty + public LimitSpec getLimitSpec() + { + return limitSpec; + } + + @Override + public MovingAverageQuery withOverriddenContext(Map contextOverride) + { + return new MovingAverageQuery( + getDataSource(), + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + computeOverridenContext(contextOverride) + ); + } + + @Override + public MovingAverageQuery withQuerySegmentSpec(QuerySegmentSpec spec) + { + return new MovingAverageQuery( + getDataSource(), + spec, + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + @Override + public Query withDataSource(DataSource dataSource) + { + return new MovingAverageQuery( + dataSource, + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + public Query withPostAveragers(List postAveragerSpecs) + { + return new MovingAverageQuery( + getDataSource(), + getQuerySegmentSpec(), + dimFilter, + granularity, + dimensions, + aggregatorSpecs, + averagerSpecs, + postAggregatorSpecs, + postAveragerSpecs, + havingSpec, + limitSpec, + limitFn, + getContext() + ); + } + + public Sequence applyLimit(Sequence results) + { + return limitFn.apply(results); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java new file mode 100644 index 000000000000..48a8e564a2c3 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.QueryMetrics; + +public interface MovingAverageQueryMetrics extends QueryMetrics +{ + /** + * Sets the size of {@link MovingAverageQuery#getDimensions()} of the given query as dimension. + */ + void numDimensions(MovingAverageQuery query); + + /** + * Sets the number of metrics of the given groupBy query as dimension. + */ + void numMetrics(MovingAverageQuery query); + + /** + * Sets the number of "complex" metrics of the given groupBy query as dimension. By default it is assumed that + * "complex" metric is a metric of not long or double type, but it could be redefined in the implementation of this + * method. + */ + void numComplexMetrics(MovingAverageQuery query); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java new file mode 100644 index 000000000000..3857d323b7fd --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +/** + * Implementations could be injected using + *

+ * PolyBind + * .optionBinder(binder, Key.get(MovingAverageQueryMetricsFactory.class)) + * .addBinding("myCustomMovingAverageQueryMetricsFactory") + * .to(MyCustomMovingAverageQueryMetricsFactory.class); + *

+ * And then setting property: + * druid.query.movingavgquery.queryMetricsFactory=myCustomMovingAverageQueryMetricsFactory + */ +public interface MovingAverageQueryMetricsFactory +{ + MovingAverageQueryMetrics makeMetrics(); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java new file mode 100644 index 000000000000..a7efa358e666 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; + +import com.google.inject.Binder; +import com.google.inject.multibindings.MapBinder; + +import org.apache.druid.guice.DruidBinders; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryToolChest; + +import java.util.Arrays; +import java.util.List; + +public class MovingAverageQueryModule implements DruidModule +{ + + @Override + public void configure(Binder binder) + { + MapBinder, QueryToolChest> toolChests = DruidBinders.queryToolChestBinder(binder); + + //Bind the query toolchest to the query class and add the binding to toolchest + toolChests.addBinding(MovingAverageQuery.class).to(MovingAverageQueryToolChest.class); + + //Bind the query toolchest to binder + binder.bind(MovingAverageQueryToolChest.class).in(LazySingleton.class); + } + + @Override + public List getJacksonModules() + { + return Arrays.asList(new SimpleModule("MovingAverageQueryModule") + .registerSubtypes(new NamedType(MovingAverageQuery.class, "movingAverage"))); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java new file mode 100644 index 000000000000..1bf421ee1899 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.QueryContexts; +import org.joda.time.Interval; +import org.joda.time.Period; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import com.google.common.base.Function; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnionDataSource; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.server.QueryStats; +import org.apache.druid.server.RequestLogLine; +import org.apache.druid.server.log.RequestLogger; + +import javax.annotation.Nullable; + +/** + * The QueryRunner for MovingAverage query. + * High level flow: + * 1. Invokes an inner groupBy query (Or timeseries for no dimensions scenario) to get Aggregations/PostAggregtions. + * 2. Result is passed to {@link RowBucketIterable}, which groups rows of all dimension combinations into period-based (e.g. daily) buckets of rows ({@link RowBucket}). + * 3. The sequence is passed to {@link MovingAverageIterable}, which performs the main part of the query of adding Averagers computation into the records. + * 4. Finishes up by applying post averagers, removing redundant dates, and applying post phases (having, sorting, limits). + */ +public class MovingAverageQueryRunner implements QueryRunner +{ + + public static final String QUERY_FAIL_TIME = "queryFailTime"; + public static final String QUERY_TOTAL_BYTES_GATHERED = "queryTotalBytesGathered"; + + private final QuerySegmentWalker walker; + private final RequestLogger requestLogger; + + public MovingAverageQueryRunner( + QueryToolChestWarehouse warehouse, + @Nullable QuerySegmentWalker walker, + RequestLogger requestLogger + ) + { + this.walker = walker; + this.requestLogger = requestLogger; + } + + @Override + public Sequence run(QueryPlus query, Map responseContext) + { + + MovingAverageQuery maq = (MovingAverageQuery) query.getQuery(); + List intervals; + final Period period; + + // Get the largest bucket from the list of averagers + Optional opt = + maq.getAveragerSpecs().stream().map(AveragerFactory::getNumBuckets).max(Integer::compare); + int buckets = opt.orElse(0); + + //Extend the interval beginning by specified bucket - 1 + if (maq.getGranularity() instanceof PeriodGranularity) { + period = ((PeriodGranularity) maq.getGranularity()).getPeriod(); + int offset = buckets <= 0 ? 0 : (1 - buckets); + intervals = maq.getIntervals() + .stream() + .map(i -> new Interval(i.getStart().withPeriodAdded(period, offset), i.getEnd())) + .collect(Collectors.toList()); + } else { + throw new ISE("Only PeriodGranulaity is supported for movingAverage queries"); + } + + Sequence resultsSeq; + DataSource dataSource = maq.getDataSource(); + if (maq.getDimensions() != null && !maq.getDimensions().isEmpty() && + (dataSource instanceof TableDataSource || dataSource instanceof UnionDataSource || + dataSource instanceof QueryDataSource)) { + // build groupBy query from movingAverage query + GroupByQuery.Builder builder = GroupByQuery.builder() + .setDataSource(dataSource) + .setInterval(intervals) + .setDimFilter(maq.getFilter()) + .setGranularity(maq.getGranularity()) + .setDimensions(maq.getDimensions()) + .setAggregatorSpecs(maq.getAggregatorSpecs()) + .setPostAggregatorSpecs(maq.getPostAggregatorSpecs()) + .setContext(maq.getContext()); + GroupByQuery gbq = builder.build(); + + HashMap gbqResponse = new HashMap<>(); + gbqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(gbq)); + gbqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); + + Sequence results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponse); + try { + // use localhost for remote address + requestLogger.log(new RequestLogLine( + DateTimes.nowUtc(), + "127.0.0.1", + gbq, + new QueryStats( + ImmutableMap.of( + "query/time", 0, + "query/bytes", 0, + "success", true + )) + )); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + resultsSeq = results; + } else { + // no dimensions, so optimize this as a TimeSeries + TimeseriesQuery tsq = new TimeseriesQuery( + dataSource, + new MultipleIntervalSegmentSpec(intervals), + false, + null, + maq.getFilter(), + maq.getGranularity(), + maq.getAggregatorSpecs(), + maq.getPostAggregatorSpecs(), + 0, + maq.getContext() + ); + HashMap tsqResponse = new HashMap<>(); + tsqResponse.put(QUERY_FAIL_TIME, System.currentTimeMillis() + QueryContexts.getTimeout(tsq)); + tsqResponse.put(QUERY_TOTAL_BYTES_GATHERED, new AtomicLong()); + + Sequence> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponse); + try { + // use localhost for remote address + requestLogger.log(new RequestLogLine( + DateTimes.nowUtc(), + "127.0.0.1", + tsq, + new QueryStats( + ImmutableMap.of( + "query/time", 0, + "query/bytes", 0, + "success", true + )) + )); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + resultsSeq = Sequences.map(results, new TimeseriesResultToRow()); + } + + // Process into day buckets + Sequence bucketedMovingAvgResults = + Sequences.simple(new RowBucketIterable(resultsSeq, intervals, period)); + + // Apply the windows analysis functions + Sequence movingAvgResults = Sequences.simple( + new MovingAverageIterable( + bucketedMovingAvgResults, + maq.getDimensions(), + maq.getAveragerSpecs(), + maq.getPostAggregatorSpecs(), + maq.getAggregatorSpecs() + )); + + // Apply any postAveragers + Sequence movingAvgResultsWithPostAveragers = + Sequences.map(movingAvgResults, new PostAveragerCalculator(maq)); + + // remove rows outside the reporting window + List reportingIntervals = maq.getIntervals(); + movingAvgResults = + Sequences.filter( + movingAvgResultsWithPostAveragers, + row -> reportingIntervals.stream().anyMatch(i -> i.contains(row.getTimestamp())) + ); + + // Apply any having, sorting, and limits + movingAvgResults = ((MovingAverageQuery) maq).applyLimit(movingAvgResults); + + return movingAvgResults; + + } + + static class TimeseriesResultToRow implements Function, Row> + { + public Row apply(Result lookbackResult) + { + Map event = lookbackResult.getValue().getBaseObject(); + MapBasedRow row = new MapBasedRow(lookbackResult.getTimestamp(), event); + return row; + } + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java new file mode 100644 index 000000000000..e35b22df32d0 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.Map; + +import javax.annotation.Nullable; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.base.Function; +import com.google.common.base.Functions; +import com.google.common.collect.Maps; +import com.google.inject.Inject; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.server.log.RequestLogger; + +/** + * The QueryToolChest for MovingAverage Query + */ +public class MovingAverageQueryToolChest extends QueryToolChest +{ + + private final QuerySegmentWalker walker; + private final RequestLogger requestLogger; + private QueryToolChestWarehouse warehouse; + + public static final String MOVING_AVERAGE_MERGE_KEY = "movingAverageMerge"; + + private final MovingAverageQueryMetricsFactory movingAverageQueryMetricsFactory; + + /** + * Construct a MovingAverageQueryToolChest for processing moving-average queries. + * MovingAverage queries are expected to be processed on broker nodes and never hit historical nodes. + * + * @param walker + * @param requestLogger + */ + @Inject + public MovingAverageQueryToolChest(@Nullable QuerySegmentWalker walker, RequestLogger requestLogger) + { + + this.walker = walker; + this.requestLogger = requestLogger; + this.movingAverageQueryMetricsFactory = DefaultMovingAverageQueryMetricsFactory.instance(); + } + + @Inject(optional = true) + public void setWarehouse(QueryToolChestWarehouse warehouse) + { + this.warehouse = warehouse; + } + + @Override + public QueryRunner mergeResults(QueryRunner runner) + { + return new MovingAverageQueryRunner(warehouse, walker, requestLogger); + } + + @Override + public QueryMetrics makeMetrics(MovingAverageQuery query) + { + MovingAverageQueryMetrics movingAverageQueryMetrics = movingAverageQueryMetricsFactory.makeMetrics(); + movingAverageQueryMetrics.query(query); + return movingAverageQueryMetrics; + } + + @Override + public Function makePostComputeManipulatorFn(MovingAverageQuery query, MetricManipulationFn fn) + { + + return new Function() + { + + @Override + public Row apply(Row result) + { + MapBasedRow mRow = (MapBasedRow) result; + final Map values = Maps.newHashMap(mRow.getEvent()); + + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + Object aggVal = values.get(agg.getName()); + if (aggVal != null) { + values.put(agg.getName(), fn.manipulate(agg, aggVal)); + } else { + values.put(agg.getName(), null); + } + } + + for (AveragerFactory avg : query.getAveragerSpecs()) { + Object aggVal = values.get(avg.getName()); + if (aggVal != null) { + values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper(avg, avg.getName() + "_"), aggVal)); + } else { + values.put(avg.getName(), null); + } + } + + return new MapBasedRow(result.getTimestamp(), values); + + } + }; + + } + + + @Override + public TypeReference getResultTypeReference() + { + return new TypeReference() + { + }; + } + + @Override + public Function makePreComputeManipulatorFn(MovingAverageQuery query, MetricManipulationFn fn) + { + return Functions.identity(); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java new file mode 100644 index 000000000000..39e988b4bb76 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import java.util.List; +import java.util.Map; + +import com.google.common.base.Function; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.query.aggregation.PostAggregator; + +/** + * Function that can be applied to a Sequence to calculate PostAverager results + */ +public class PostAveragerCalculator implements Function +{ + + private final List postAveragers; + + public PostAveragerCalculator(MovingAverageQuery maq) + { + this.postAveragers = maq.getPostAveragerSpecs(); + } + + /* (non-Javadoc) + * @see com.google.common.base.Function#apply(java.lang.Object) + */ + @Override + public Row apply(Row input) + { + MapBasedRow row = (MapBasedRow) input; + Map event = row.getEvent(); + + for (PostAggregator postAverager : postAveragers) { + boolean allColsPresent = postAverager.getDependentFields().stream().allMatch(c -> event.get(c) != null); + event.put(postAverager.getName(), allColsPresent ? postAverager.compute(event) : null); + } + + return input; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java new file mode 100644 index 000000000000..8bf058fdf68f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.joda.time.DateTime; + +import java.util.Collection; + +import org.apache.druid.data.input.Row; + +/** + * Represents a set of rows for a specific date + */ +public class RowBucket +{ + private final DateTime dateTime; + private final Collection rows; + private RowBucket nextBucket = null; + + public RowBucket(DateTime dateTime, Collection rows) + { + this.dateTime = dateTime; + this.rows = rows; + } + + public DateTime getDateTime() + { + return dateTime; + } + + public Collection getRows() + { + return rows; + } + + public RowBucket getNextBucket() + { + return nextBucket; + } + + public void setNextBucket(RowBucket nextRow) + { + this.nextBucket = nextRow; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java new file mode 100644 index 000000000000..25dfe246ffca --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; + +/** + * It is the iterable used to bucket data into days, + * doing appropriate lookahead to see if the next row is in the same day or a new day. + */ +public class RowBucketIterable implements Iterable +{ + + public final Sequence seq; + private List intervals; + private Period period; + + public RowBucketIterable(Sequence seq, List intervals, Period period) + { + this.seq = seq; + this.period = period; + this.intervals = intervals; + } + + /* (non-Javadoc) + * @see java.lang.Iterable#iterator() + */ + @Override + public Iterator iterator() + { + return new RowIterator(seq, intervals, period); + } + + static class RowIterator implements Iterator + { + private Yielder yielder; + private boolean done = false; + private DateTime endTime; + private DateTime expectedBucket; + private Period period; + private int intervalIndex = 0; + private List intervals; + private boolean processedLastRow = false; + private boolean processedExtraRow = false; + + public RowIterator(Sequence rows, List intervals, Period period) + { + this.period = period; + this.intervals = intervals; + expectedBucket = intervals.get(intervalIndex).getStart(); + endTime = intervals.get(intervals.size() - 1).getEnd(); + yielder = rows.toYielder(null, new BucketingAccumulator()); + } + + /* (non-Javadoc) + * @see java.util.Iterator#hasNext() + */ + @Override + public boolean hasNext() + { + // expectedBucket < endTime + if (expectedBucket.compareTo(endTime) < 0) { + return true; + } + return false; + } + + /* (non-Javadoc) + * @see java.util.Iterator#next() + */ + @Override + public RowBucket next() + { + RowBucket currentBucket = yielder.get(); + + if (expectedBucket.compareTo(intervals.get(intervalIndex).getEnd()) >= 0) { + intervalIndex++; + if (intervalIndex <= intervals.size()) { + expectedBucket = intervals.get(intervalIndex).getStart(); + } + } + // currentBucket > expectedBucket + if (currentBucket != null && currentBucket.getDateTime().compareTo(expectedBucket) > 0) { + currentBucket = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } + + if (!yielder.isDone()) { + // standard case. return regular row + yielder = yielder.next(currentBucket); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else if (!processedLastRow && yielder.get() != null && yielder.get().getNextBucket() == null) { + // yielder.isDone, processing last row + processedLastRow = true; + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else if (!processedExtraRow && yielder.get() != null && yielder.get().getNextBucket() != null) { + RowBucket lastRow = yielder.get().getNextBucket(); + + if (lastRow.getDateTime().compareTo(expectedBucket) > 0) { + lastRow = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return lastRow; + } + + // yielder is done, processing newBucket + processedExtraRow = true; + expectedBucket = expectedBucket.plus(period); + return lastRow; + } else if (expectedBucket.compareTo(endTime) < 0) { + // add any trailing blank rows + currentBucket = new RowBucket(expectedBucket, Collections.emptyList()); + expectedBucket = expectedBucket.plus(period); + return currentBucket; + } else { + // we should never get here + throw new NoSuchElementException(); + } + + } + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java new file mode 100644 index 000000000000..20da12ca3fca --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * Interface for an averager + * + * @param The return type of the averager + */ +public interface Averager +{ + /** + * Add a row to the window being operated on + * + * @param e The row to add + * @param aggMap The Map of AggregatorFactory used to determine if the metric should to be finalized + */ + void addElement(Map e, Map aggMap); + + /** + * There is a missing row, so record a missing entry in the window + */ + void skip(); + + /** + * Compute the resulting "average" over the collected window + * + * @return the "average" over the window of buckets + */ + R getResult(); + + /** + * @return the name + */ + String getName(); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java new file mode 100644 index 000000000000..e8f2ad4fe7d5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Interface representing Averager in the movingAverage query. + * + * @param Type returned by the underlying averager. + * @param Type of finalized value. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "constant", value = ConstantAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMean", value = DoubleMeanAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMeanNoNulls", value = DoubleMeanNoNullAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMax", value = DoubleMaxAveragerFactory.class), + @JsonSubTypes.Type(name = "doubleMin", value = DoubleMinAveragerFactory.class), + @JsonSubTypes.Type(name = "longMean", value = LongMeanAveragerFactory.class), + @JsonSubTypes.Type(name = "longMeanNoNulls", value = LongMeanNoNullAveragerFactory.class), + @JsonSubTypes.Type(name = "longMax", value = LongMaxAveragerFactory.class), + @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class), + @JsonSubTypes.Type(name = "sketchUnion", value = SketchUnionAveragerFactory.class) +}) +public interface AveragerFactory +{ + int DEFAULT_PERIOD = 1; + + /** + * Gets the column name that will be populated by the Averager + * + * @return The column name + */ + String getName(); + + /** + * Returns the window size over which the averaging calculations will be + * performed. Size is computed in terms of buckets rather than absolute time. + * + * @return The window size + */ + int getNumBuckets(); + + /** + * Returns the cycle size (number of periods to skip during averaging calculations). + * + * @return The cycle size + */ + int getCycleSize(); + + /** + * Create an Averager for a specific dimension combination. + * + * @return The {@link Averager} + */ + Averager createAverager(); + + /** + * Gets the list of dependent fields that will be used by this Averager. Most + * {@link Averager}s depend on only a single field from the underlying query, but + * that is not required. This method allow the required fields to be communicated + * back to the main query so that validation to enforce the fields presence can + * be accomplished. + * + * @return A list of field names + */ + List getDependentFields(); + + /** + * Returns a {@link Comparator} that can be used to compare result values for + * purposes of sorting the end result of the query. + * + * @return A {@link Comparator} + */ + Comparator getComparator(); + + /** + * Finalize result value. + * + * @param val the value to finalize. + * + * @return The finalized value. + */ + F finalizeComputation(R val); +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java new file mode 100644 index 000000000000..e525deb223e4 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.lang.reflect.Array; +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * Common base class available for use by averagers. The base class implements methods that + * capture incoming and skipped rows and store them in an array, to be used later for + * calculating the actual value. + * + * @param The type of intermediate value to be retrieved from the row and stored + * @param The type of result the averager is expected to produce + */ +public abstract class BaseAverager implements Averager +{ + + private final int numBuckets; + private final int cycleSize; + private final String name; + private final String fieldName; + private final I[] buckets; + private int index; + + /* This flag checks if the intermediate sketches have to be finalized or not + for further computation involving sketches */ + private boolean shouldFinalize; + + /** + * @param storageType The class to use for storing intermediate values + * @param numBuckets The number of buckets to include in the window being aggregated + * @param name The name of the resulting metric + * @param fieldName The field to extra from incoming rows and stored in the window cache + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + * @param shouldFinalize Whether the intermediate values need to be finalized prior to averaging + */ + public BaseAverager( + Class storageType, int numBuckets, String name, String fieldName, int cycleSize, + boolean shouldFinalize + ) + { + this.numBuckets = numBuckets; + this.name = name; + this.fieldName = fieldName; + this.index = 0; + @SuppressWarnings("unchecked") + final I[] array = (I[]) Array.newInstance(storageType, numBuckets); + this.buckets = array; + this.shouldFinalize = shouldFinalize; + this.cycleSize = cycleSize; + } + + public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize) + { + this(storageType, numBuckets, name, fieldName, cycleSize, true); + } + + + /* (non-Javadoc) + * @see Averager#addElement(java.util.Map, java.util.Map) + */ + @SuppressWarnings("unchecked") + @Override + public void addElement(Map e, Map a) + { + Object metric = e.get(fieldName); + I finalMetric; + if (a.containsKey(fieldName) && shouldFinalize) { + AggregatorFactory af = a.get(fieldName); + finalMetric = metric != null ? (I) af.finalizeComputation(metric) : null; + } else { + finalMetric = (I) metric; + } + buckets[index++] = finalMetric; + index %= numBuckets; + } + + /* (non-Javadoc) + * @see Averager#skip() + */ + @Override + public void skip() + { + buckets[index++] = null; + index %= numBuckets; + } + + /* (non-Javadoc) + * @see Averager#getResult() + */ + @Override + public R getResult() + { + if (!hasData()) { + return null; + } + return computeResult(); + } + + /** + * Compute the result value to be returned by getResult. + * + *

This routine will only be called when there is valid data within the window + * and doesn't need to worry about detecting the case where no data should be returned. + * + *

+ * The method typically should use {@link #getBuckets()} to retrieve the set of buckets + * within the window and then compute a value based on those. It should expect nulls within + * the array, indicating buckets where no row was found for the dimension combination. It is + * up to the actual implementation to determin how to evaluate those nulls. + * + *

+ * The type returned is NOT required to be the same type as the intermediary value. For example, + * the intermediate value could be a Sketch, but the result a Long. + * + * @return the computed result + */ + protected abstract R computeResult(); + + /* (non-Javadoc) + * @see Averager#getName() + */ + @Override + public String getName() + { + return name; + } + + /** + * Returns the fieldname to be extracted from any event rows passed in and stored + * for use computing the windowed function. + * + * @return the fieldName + */ + public String getFieldName() + { + return fieldName; + } + + /** + * @return the numBuckets + */ + public int getNumBuckets() + { + return numBuckets; + } + + /** + * @return the cycleSize + */ + public int getCycleSize() + { + return cycleSize; + } + + /** + * @return the array of buckets + */ + protected I[] getBuckets() + { + return (I[]) buckets; + } + + /** + * Determines wheter any data is present. If all the buckets are empty (not "0"), then + * no value should be returned from the Averager, as there were not valid rows within the window. + * + * @return true if any non-null values available + */ + protected boolean hasData() + { + for (Object b : buckets) { + if (b != null) { + return true; + } + } + return false; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java new file mode 100644 index 000000000000..d91e74f687b7 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Collections; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +/** + * Common base class for AveragerFactories + * + * @param Base type that the averager should return as a result + * @param Type that that is returned from finalization + */ +public abstract class BaseAveragerFactory implements AveragerFactory +{ + + protected String name; + protected String fieldName; + protected int numBuckets; + protected int cycleSize; + + /** + * Constructor. + * + * @param name Name of the Averager + * @param numBuckets Number of buckets in the analysis window + * @param fieldName Field from incoming events to include in the analysis + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + */ + public BaseAveragerFactory(String name, int numBuckets, String fieldName, Integer cycleSize) + { + this.name = name; + this.numBuckets = numBuckets; + this.fieldName = fieldName; + this.cycleSize = (cycleSize != null) ? cycleSize : DEFAULT_PERIOD; + Preconditions.checkNotNull(name, "Must have a valid, non-null averager name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null field name"); + Preconditions.checkArgument(this.cycleSize > 0, "Cycle size must be greater than zero"); + Preconditions.checkArgument(numBuckets > 0, "Bucket size must be greater than zero"); + Preconditions.checkArgument(this.cycleSize < numBuckets, "Cycle size must be less than the bucket size"); + Preconditions.checkArgument(numBuckets % this.cycleSize == 0, "cycleSize must devide numBuckets without a remainder"); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty("buckets") + public int getNumBuckets() + { + return numBuckets; + } + + @Override + @JsonProperty("cycleSize") + public int getCycleSize() + { + return cycleSize; + } + + @Override + public List getDependentFields() + { + return Collections.singletonList(fieldName); + } + + @SuppressWarnings("unchecked") + @Override + public F finalizeComputation(R val) + { + return (F) val; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java new file mode 100644 index 000000000000..ca76b505ae45 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ +/** + * Copyright 2018 Oath Inc. All rights reserved. + */ +package org.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; + +/** + * Base averager factory that adds a default comparable method. + * + * @param return type + * @param finalized type + */ +public abstract class ComparableAveragerFactory, F> extends BaseAveragerFactory +{ + /** + * Constructor. + * + * @param name Name of the Averager + * @param numBuckets Number of buckets in the analysis window + * @param fieldName Field from incoming events to include in the analysis + * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). + */ + public ComparableAveragerFactory(String name, int numBuckets, String fieldName, Integer cycleSize) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Comparator getComparator() + { + return Comparator.naturalOrder(); + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java new file mode 100644 index 000000000000..806fa6b21242 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Map; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +/** + * The constant averager.Created soley for incremental development and wiring things up. + */ +public class ConstantAverager implements Averager +{ + + private String name; + private float retval; + + /** + * @param n + * @param name + * @param retval + */ + public ConstantAverager(int n, String name, float retval) + { + this.name = name; + this.retval = retval; + } + + /* (non-Javadoc) + * @see Averager#getResult() + */ + @Override + public Float getResult() + { + return retval; + } + + /* (non-Javadoc) + * @see Averager#getName() + */ + @Override + public String getName() + { + return name; + } + + /* (non-Javadoc) + * @see Averager#addElement(java.util.Map, java.util.Map) + */ + @Override + public void addElement(Map e, Map a) + { + // since we return a constant, no need to read from the event + } + + /* (non-Javadoc) + * @see Averager#skip() + */ + @Override + public void skip() + { + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java new file mode 100644 index 000000000000..cffc876fa399 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Implementation of AveragerFacvtory created solely for incremental development + */ + +public class ConstantAveragerFactory implements AveragerFactory +{ + + private String name; + private int numBuckets; + private float retval; + + @JsonCreator + public ConstantAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("retval") float retval + ) + { + this.name = name; + this.numBuckets = numBuckets; + this.retval = retval; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + @JsonProperty("buckets") + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public float getRetval() + { + return retval; + } + + @Override + public Averager createAverager() + { + return new ConstantAverager(numBuckets, name, retval); + } + + @Override + public List getDependentFields() + { + return Collections.emptyList(); + } + + @Override + public Comparator getComparator() + { + return Comparator.naturalOrder(); + } + + @Override + public int getCycleSize() + { + return 1; + } + + @Override + public Float finalizeComputation(Float val) + { + return val; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java new file mode 100644 index 000000000000..c9db3f2cd069 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMaxAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = Double.NEGATIVE_INFINITY; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Double.max(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java new file mode 100644 index 000000000000..9c708428fff9 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMaxAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public DoubleMaxAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMaxAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java new file mode 100644 index 000000000000..ece43f105992 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMeanAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = 0.0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + } else { + result += 0.0; + } + validBuckets++; + } + + startFrom++; + return result / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java new file mode 100644 index 000000000000..233b7c521bad --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMeanAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public DoubleMeanAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMeanAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java new file mode 100644 index 000000000000..5f98c25ea9bd --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMeanNoNullAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = 0.0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + validBuckets++; + } + } + + startFrom++; + return result / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java new file mode 100644 index 000000000000..9abf4fcce7d1 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMeanNoNullAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public DoubleMeanNoNullAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMeanNoNullAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java new file mode 100644 index 000000000000..0ded39e82f23 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class DoubleMinAverager extends BaseAverager +{ + + private int startFrom = 0; + + public DoubleMinAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + double result = Double.POSITIVE_INFINITY; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Double.min(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java new file mode 100644 index 000000000000..d3d81f1be13b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DoubleMinAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public DoubleMinAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new DoubleMinAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java new file mode 100644 index 000000000000..343a1c8f9e3a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMaxAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Long computeResult() + { + long result = Long.MIN_VALUE; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Long.max(result, (obj[(i + startFrom) % numBuckets]).longValue()); + } + } + + startFrom++; + return result; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java new file mode 100644 index 000000000000..473026fd7180 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMaxAveragerFactory extends ComparableAveragerFactory +{ + @JsonCreator + public LongMaxAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMaxAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java new file mode 100644 index 000000000000..8c6d97d53863 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMeanAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + long result = 0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).longValue(); + } else { + result += 0; + } + validBuckets++; + } + + startFrom++; + return ((double) result) / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java new file mode 100644 index 000000000000..2c413b976a75 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMeanAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMeanAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMeanAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java new file mode 100644 index 000000000000..5cf6548e3605 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMeanNoNullAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Double computeResult() + { + long result = 0; + int validBuckets = 0; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result += (obj[(i + startFrom) % numBuckets]).longValue(); + validBuckets++; + } + } + + startFrom++; + return ((double) result) / validBuckets; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java new file mode 100644 index 000000000000..5761e61ae4d1 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMeanNoNullAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMeanNoNullAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMeanNoNullAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java new file mode 100644 index 000000000000..0a275988395d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +public class LongMinAverager extends BaseAverager +{ + + private int startFrom = 0; + + public LongMinAverager(int numBuckets, String name, String fieldName, int cycleSize) + { + super(Number.class, numBuckets, name, fieldName, cycleSize); + } + + @Override + protected Long computeResult() + { + long result = Long.MAX_VALUE; + int cycleSize = getCycleSize(); + int numBuckets = getNumBuckets(); + Number[] obj = getBuckets(); + + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + result = Long.min(result, (obj[(i + startFrom) % numBuckets]).longValue()); + } + } + + startFrom++; + return result; + } + +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java new file mode 100644 index 000000000000..2d9f342414ea --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LongMinAveragerFactory extends ComparableAveragerFactory +{ + + @JsonCreator + public LongMinAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") int cycleSize, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, numBuckets, fieldName, cycleSize); + } + + @Override + public Averager createAverager() + { + return new LongMinAverager(numBuckets, name, fieldName, cycleSize); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java new file mode 100644 index 000000000000..0756204fa5e6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.yahoo.sketches.Family; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.SetOperationBuilder; +import com.yahoo.sketches.theta.Union; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; + +public class SketchUnionAverager extends BaseAverager +{ + + private int startFrom = 0; + private final SetOperationBuilder builder; + + public SketchUnionAverager(int numBuckets, String name, String fieldName, int cycleSize, int size) + { + super(SketchHolder.class, numBuckets, name, fieldName, cycleSize, false); + this.builder = SetOperation.builder().setNominalEntries(size); + } + + @Override + protected SketchHolder computeResult() + { + int cycleSize = getCycleSize(); + Object[] obj = getBuckets(); + int numBuckets = getNumBuckets(); + + Union union = (Union) builder.build(Family.UNION); + for (int i = 0; i < numBuckets; i += cycleSize) { + if (obj[(i + startFrom) % numBuckets] != null) { + ((SketchHolder) obj[(i + startFrom) % numBuckets]).updateUnion(union); + } + } + + startFrom++; + return SketchHolder.of(union.getResult()); + } +} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java new file mode 100644 index 000000000000..5a67de968365 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import java.util.Comparator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.yahoo.sketches.Util; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; + +public class SketchUnionAveragerFactory extends BaseAveragerFactory +{ + + private final int size; + + @JsonCreator + public SketchUnionAveragerFactory( + @JsonProperty("name") String name, + @JsonProperty("buckets") int numBuckets, + @JsonProperty("cycleSize") Integer cycleSize, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size + ) + { + super(name, numBuckets, fieldName, cycleSize); + this.size = size == null ? Util.DEFAULT_NOMINAL_ENTRIES : size; + } + + public SketchUnionAveragerFactory( + String name, + int numBuckets, + Integer cycleSize, + String fieldName + ) + { + this(name, numBuckets, cycleSize, fieldName, null); + } + + @Override + public Averager createAverager() + { + return new SketchUnionAverager(numBuckets, name, fieldName, cycleSize, size); + } + + /* (non-Javadoc) + * @see AveragerFactory#finalize(java.lang.Object) + */ + @Override + public Double finalizeComputation(SketchHolder val) + { + return val.getSketch().getEstimate(); + } + + /* (non-Javadoc) + * @see BaseAveragerFactory#getComparator() + */ + @SuppressWarnings({"rawtypes", "unchecked"}) + @Override + public Comparator getComparator() + { + return SketchHolder.COMPARATOR; + } +} diff --git a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..72876559838d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1 @@ +org.apache.druid.query.movingaverage.MovingAverageQueryModule diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java new file mode 100644 index 000000000000..bc052b3b9854 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.junit.Test; + +import java.util.Arrays; + +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test IdentityYieldingAccumulator + */ +public class IdentityYieldingAccumulatorTest +{ + @Test + public void testAccumulator() + { + Sequence seq = Sequences.simple(Arrays.asList(1, 2, 3, 4, 5)); + + Yielder y = seq.toYielder(null, new IdentityYieldingAccumulator<>()); + + assertEquals(Integer.valueOf(1), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(2), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(3), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(4), y.get()); + y = y.next(null); + assertEquals(Integer.valueOf(5), y.get()); + y = y.next(null); + assertTrue(y.isDone()); + + assertNull(y.get()); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java new file mode 100644 index 000000000000..9e89b69bfb18 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -0,0 +1,802 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; +import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; + +import org.joda.time.DateTime; +import org.joda.time.chrono.ISOChronology; +import org.junit.Test; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * + */ +public class MovingAverageIterableTest +{ + private static final DateTime JAN_1 = new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_2 = new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_3 = new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + + private static final String GENDER = "gender"; + private static final String AGE = "age"; + private static final String COUNTRY = "country"; + + private static final Map dims1 = new HashMap<>(); + private static final Map dims2 = new HashMap<>(); + private static final Map dims3 = new HashMap<>(); + + static { + dims1.put(GENDER, "m"); + dims1.put(AGE, "10"); + dims1.put(COUNTRY, "US"); + + dims2.put(GENDER, "f"); + dims2.put(AGE, "8"); + dims2.put(COUNTRY, "US"); + + dims3.put(GENDER, "u"); + dims3.put(AGE, "5"); + dims3.put(COUNTRY, "UK"); + } + + @Test + public void testNext() + { + + Collection dims = Arrays.asList( + new DefaultDimensionSpec(GENDER, GENDER), + new DefaultDimensionSpec(AGE, AGE), + new DefaultDimensionSpec(COUNTRY, COUNTRY) + ); + + Sequence dayBuckets = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList( + new MapBasedRow(JAN_1, dims1), + new MapBasedRow(JAN_1, dims2) + )), + new RowBucket(JAN_2, Arrays.asList( + new MapBasedRow(JAN_2, dims1) + )), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Arrays.asList( + new MapBasedRow(JAN_4, dims2), + new MapBasedRow(JAN_4, dims3) + )) + )); + + Iterable iterable = new MovingAverageIterable( + dayBuckets, + dims, + Collections.singletonList(new ConstantAveragerFactory("noop", 1, 1.1f)), + Collections.emptyList(), + Collections.emptyList() + ); + + Iterator iter = iterable.iterator(); + + assertTrue(iter.hasNext()); + Row r = iter.next(); + assertEquals(JAN_1, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_1, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_2, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_2, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + Row r2 = r; + assertEquals(JAN_3, r.getTimestamp()); + assertEquals("US", r.getRaw(COUNTRY)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_3, r.getTimestamp()); + assertEquals("US", r.getRaw(COUNTRY)); + assertThat(r.getRaw(AGE), not(equalTo(r2.getRaw(AGE)))); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("f", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("u", r.getRaw(GENDER)); + + assertTrue(iter.hasNext()); + r = iter.next(); + assertEquals(JAN_4, r.getTimestamp()); + assertEquals("m", r.getRaw(GENDER)); + + assertFalse(iter.hasNext()); + } + + @Test + public void testAveraging() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + Map event4 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_2, event2); + + event3.put("gender", "m"); + event3.put("pageViews", 30L); + Row row3 = new MapBasedRow(JAN_3, event3); + + event4.put("gender", "f"); + event4.put("pageViews", 40L); + Row row4 = new MapBasedRow(JAN_2, event4); + + float retval = 14.5f; + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.singletonList(row2)), + new RowBucket(JAN_3, Arrays.asList(row3, row4)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new ConstantAveragerFactory("costPageViews", 7, retval), + new LongMeanAveragerFactory("movingAvgPageViews", 7, 1, "pageViews") + ), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row caResult = iter.next(); + + assertEquals(JAN_1, caResult.getTimestamp()); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(retval, caResult.getMetric("costPageViews").floatValue(), 0.0f); + assertEquals(1.4285715f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(4.285714f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("m", (caResult.getDimension("gender")).get(0)); + assertEquals(8.571428f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + caResult = iter.next(); + assertEquals("f", (caResult.getDimension("gender")).get(0)); + assertEquals(5.714285850f, caResult.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + + } + + + @Test + public void testCompleteData() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_2_row2 = new MapBasedRow(JAN_2, event2); + Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + + } + + // no injection if the data missing at the begining + @Test + public void testMissingDataAtBeginning() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_2_row2 = new MapBasedRow(JAN_2, event2); + Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + // test injection when the data is missing at the end + @Test + public void testMissingDataAtTheEnd() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + // test injection when the data is missing in the middle + @Test + public void testMissingDataAtMiddle() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + Map event3 = new HashMap<>(); + Map event4 = new HashMap<>(); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + event2.put("gender", "f"); + event2.put("pageViews", 20L); + event3.put("gender", "u"); + event3.put("pageViews", 30L); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan_1_row2 = new MapBasedRow(JAN_1, event2); + Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan_3_row1 = new MapBasedRow(JAN_3, event1); + Row jan_3_row2 = new MapBasedRow(JAN_3, event2); + Row jan_3_row3 = new MapBasedRow(JAN_3, event3); + Row jan_4_row1 = new MapBasedRow(JAN_4, event1); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), + new RowBucket(JAN_2, Arrays.asList(jan_2_row1)), + new RowBucket(JAN_3, Arrays.asList(jan_3_row1, jan_3_row2, jan_3_row3)), + new RowBucket(JAN_4, Arrays.asList(jan_4_row1)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 3, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_1, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals(JAN_2, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("f", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("u", (result.getDimension("gender")).get(0)); + assertEquals(JAN_3, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals(JAN_4, (result.getTimestamp())); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysAtBegining() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_3, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.emptyList()), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.singletonList(row1)), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysInMiddle() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testWithFilteredAggregation() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_4, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.emptyList()), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.singletonList(row2)) + )); + + AveragerFactory averagerfactory = new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews"); + AggregatorFactory aggregatorFactory = new LongSumAggregatorFactory("pageViews", "pageViews"); + DimFilter filter = new SelectorDimFilter("gender", "m", null); + FilteredAggregatorFactory filteredAggregatorFactory = new FilteredAggregatorFactory(aggregatorFactory, filter); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + averagerfactory), + Collections.emptyList(), + Collections.singletonList( + filteredAggregatorFactory) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + } + + @Test + public void testMissingDaysAtEnd() + { + + Map event1 = new HashMap<>(); + Map event2 = new HashMap<>(); + + List ds = new ArrayList<>(); + ds.add(new DefaultDimensionSpec("gender", "gender")); + + event1.put("gender", "m"); + event1.put("pageViews", 10L); + Row row1 = new MapBasedRow(JAN_1, event1); + + event2.put("gender", "m"); + event2.put("pageViews", 20L); + Row row2 = new MapBasedRow(JAN_2, event2); + + Sequence seq = Sequences.simple(Arrays.asList( + new RowBucket(JAN_1, Collections.singletonList(row1)), + new RowBucket(JAN_2, Collections.singletonList(row2)), + new RowBucket(JAN_3, Collections.emptyList()), + new RowBucket(JAN_4, Collections.emptyList()), + new RowBucket(JAN_5, Collections.emptyList()), + new RowBucket(JAN_6, Collections.emptyList()) + )); + + Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), + Collections.emptyList(), + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" + )) + ).iterator(); + + assertTrue(iter.hasNext()); + Row result = iter.next(); + + assertEquals(JAN_1, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(2.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_2, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_3, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_4, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(7.5f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_5, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(5.0f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertTrue(iter.hasNext()); + result = iter.next(); + assertEquals(JAN_6, result.getTimestamp()); + assertEquals("m", (result.getDimension("gender")).get(0)); + assertEquals(0.0f, result.getMetric("movingAvgPageViews").floatValue(), 0.0f); + + assertFalse(iter.hasNext()); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java new file mode 100644 index 000000000000..aaadd2344ec8 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -0,0 +1,442 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import com.google.inject.util.Providers; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.ForegroundCachePopulator; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.guice.DruidProcessingModule; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.RetryQueryRunnerConfig; +import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.timeline.TimelineLookup; +import org.hamcrest.core.IsInstanceOf; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; +import org.apache.druid.query.movingaverage.test.TestConfig; +import org.apache.druid.client.CachingClusteredClient; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.initialization.Initialization; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; + +import mockit.Mock; +import mockit.MockUp; + +import org.apache.druid.java.util.emitter.service.ServiceEmitter; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; + +/** + * Base class for implementing MovingAverageQuery tests + */ +@RunWith(Parameterized.class) +public class MovingAverageQueryTest +{ + private final ObjectMapper jsonMapper; + private final Injector injector; + private final QueryToolChestWarehouse warehouse; + private final RetryQueryRunnerConfig retryConfig; + private final ServerConfig serverConfig; + + private final List groupByResults = new ArrayList<>(); + private final List> timeseriesResults = new ArrayList<>(); + + private final TestConfig config; + + @Parameters(name = "{0}") + public static Iterable data() throws IOException + { + BufferedReader testReader = new BufferedReader( + new InputStreamReader(MovingAverageQueryTest.class.getResourceAsStream("/queryTests"), StandardCharsets.UTF_8)); + List tests = new ArrayList<>(); + + for (String line = testReader.readLine(); line != null; line = testReader.readLine()) { + tests.add(new String[] {line}); + } + + return tests; + } + + public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMappingException, IOException + { + List modules = getRequiredModules(); + modules.add( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("queryTest"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(1); + binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); + } + ); + + + Injector baseInjector = GuiceInjectors.makeStartupInjector(); + injector = Initialization.makeInjectorWithModules(baseInjector, modules); + + jsonMapper = injector.getInstance(ObjectMapper.class); + warehouse = injector.getInstance(QueryToolChestWarehouse.class); + retryConfig = injector.getInstance(RetryQueryRunnerConfig.class); + serverConfig = injector.getInstance(ServerConfig.class); + + InputStream is = getClass().getResourceAsStream("/queryTests/" + yamlFile); + ObjectMapper reader = new ObjectMapper(new YAMLFactory()); + config = reader.readValue(is, TestConfig.class); + } + + /** + * Returns the JSON query that should be used in the test. + * + * @return The JSON query + */ + protected String getQueryString() + { + return config.query.toString(); + } + + /** + * Returns the JSON result that should be expected from the query. + * + * @return The JSON result + */ + protected String getExpectedResultString() + { + return config.expectedOutput.toString(); + } + + /** + * Returns the JSON result that the nested groupby query should produce. + * Either this method or {@link #getTimeseriesResultJson()} must be defined + * by the subclass. + * + * @return The JSON result from the groupby query + */ + protected String getGroupByResultJson() + { + ArrayNode node = config.intermediateResults.get("groupBy"); + return node == null ? null : node.toString(); + } + + /** + * Returns the JSON result that the nested timeseries query should produce. + * Either this method or {@link #getGroupByResultJson()} must be defined + * by the subclass. + * + * @return The JSON result from the timeseries query + */ + protected String getTimeseriesResultJson() + { + ArrayNode node = config.intermediateResults.get("timeseries"); + return node == null ? null : node.toString(); + } + + /** + * Returns the expected query type. + * + * @return The Query type + */ + protected Class getExpectedQueryType() + { + return MovingAverageQuery.class; + } + + protected TypeReference getExpectedResultType() + { + return new TypeReference>() + { + }; + } + + /** + * Returns a list of any additional Druid Modules necessary to run the test. + * + * @return List of Druid Modules + */ + protected List getRequiredModules() + { + List list = new ArrayList<>(); + + list.add(new SketchModule()); + list.add(new QueryRunnerFactoryModule()); + list.add(new QueryableModule()); + list.add(new DruidProcessingModule()); + + return list; + } + + /** + * Set up any needed mocks to stub out backend query behavior. + * + * @param query + * + * @throws IOException + * @throws JsonMappingException + * @throws JsonParseException + */ + protected void defineMocks(Query query) throws JsonParseException, JsonMappingException, IOException + { + groupByResults.clear(); + timeseriesResults.clear(); + List aggs; + + if (query instanceof GroupByQuery) { + aggs = ((GroupByQuery) query).getAggregatorSpecs(); + } else if (query instanceof TimeseriesQuery) { + aggs = ((TimeseriesQuery) query).getAggregatorSpecs(); + } else if (query instanceof MovingAverageQuery) { + aggs = ((MovingAverageQuery) query).getAggregatorSpecs(); + } else { + // unrecognized query type + aggs = Collections.emptyList(); + + } + + if (getGroupByResultJson() != null) { + groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference>() + { + })); + for (Row r : groupByResults) { + Map map = ((MapBasedRow) r).getEvent(); + for (AggregatorFactory agg : aggs) { + Object serializedVal = map.get(agg.getName()); + if (serializedVal != null) { + map.put(agg.getName(), agg.deserialize(serializedVal)); + } + } + } + } + + if (getTimeseriesResultJson() != null) { + timeseriesResults.addAll(jsonMapper.readValue( + getTimeseriesResultJson(), + new TypeReference>>() + { + } + )); + for (Result r : timeseriesResults) { + Map map = r.getValue().getBaseObject(); + for (AggregatorFactory agg : aggs) { + Object serializedVal = map.get(agg.getName()); + if (serializedVal != null) { + map.put(agg.getName(), agg.deserialize(serializedVal)); + } + } + } + } + } + + /** + * converts Int to Long, Float to Double in the actual and expected result + * + * @param result + */ + protected void consistentTypeCasting(List result) + { + for (MapBasedRow row : result) { + Map event = row.getEvent(); + event.forEach((key, value) -> { + if (Integer.class.isInstance(value)) { + event.put(key, ((Integer) value).longValue()); + } + if (Float.class.isInstance(value)) { + event.put(key, ((Float) value).doubleValue()); + } + }); + + } + } + + /** + * Validate that the specified query behaves correctly. + * + * @throws IOException + * @throws JsonMappingException + * @throws JsonParseException + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + @Test + public void testQuery() throws JsonParseException, JsonMappingException, IOException + { + + + // create mocks for nested queries + @SuppressWarnings("unused") + + MockUp groupByQuery = new MockUp() + { + @Mock + public QueryRunner getRunner(QuerySegmentWalker walker) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + return Sequences.simple(groupByResults); + } + }; + } + }; + + + @SuppressWarnings("unused") + MockUp timeseriesQuery = new MockUp() + { + @Mock + public QueryRunner getRunner(QuerySegmentWalker walker) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + return Sequences.simple(timeseriesResults); + } + }; + } + }; + + + Query query = jsonMapper.readValue(getQueryString(), Query.class); + assertThat(query, IsInstanceOf.instanceOf(getExpectedQueryType())); + + List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); + assertNotNull(expectedResults); + assertThat(expectedResults, IsInstanceOf.instanceOf(List.class)); + + CachingClusteredClient baseClient = new CachingClusteredClient( + warehouse, + new TimelineServerView() + { + @Override + public TimelineLookup getTimeline(DataSource dataSource) + { + return null; + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + return null; + } + + @Override + public void registerTimelineCallback(Executor exec, TimelineCallback callback) + { + + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + + } + + @Override + public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback) + { + + } + }, + MapCache.create(100000), + jsonMapper, + new ForegroundCachePopulator(jsonMapper, new CachePopulatorStats(), -1), + new CacheConfig(), + new DruidHttpClientConfig() + { + @Override + public long getMaxQueuedBytes() + { + return 0L; + } + } + ); + + ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( + new ServiceEmitter("", "", null) + { + public void emit(org.apache.druid.java.util.emitter.core.Event event) {} + }, + baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() + ); + final Map responseContext = new ConcurrentHashMap<>(); + + defineMocks(query); + + QueryPlus queryPlus = QueryPlus.wrap(query); + final Sequence res = query.getRunner(walker).run(queryPlus, responseContext); + + List actualResults = new ArrayList(); + actualResults = (List) res.accumulate(actualResults, Accumulators.list()); + + consistentTypeCasting(expectedResults); + consistentTypeCasting(actualResults); + + assertEquals(expectedResults, actualResults); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java new file mode 100644 index 000000000000..dbd127fdf7e6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.query.movingaverage.averagers.DoubleMeanAveragerFactory; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.chrono.ISOChronology; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + + +/** + * Unit tests for PostAveragerCalcualtor + */ +public class PostAveragerCalculatorTest +{ + + private MovingAverageQuery query; + private PostAveragerCalculator pac; + private Map event; + private MapBasedRow row; + + @Before + public void setup() + { + query = new MovingAverageQuery( + new TableDataSource("d"), + new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval("2017-01-01/2017-01-01", ISOChronology.getInstanceUTC()))), + null, + Granularities.DAY, + null, + Collections.singletonList(new CountAggregatorFactory("count")), + Collections.emptyList(), + null, + Collections.singletonList(new DoubleMeanAveragerFactory("avgCount", 7, 1, "count")), + Collections.singletonList(new ArithmeticPostAggregator( + "avgCountRatio", + "/", + Arrays.asList( + new FieldAccessPostAggregator("count", "count"), + new FieldAccessPostAggregator("avgCount", "avgCount") + ) + )), + null, + null + ); + + pac = new PostAveragerCalculator(query); + event = new HashMap<>(); + row = new MapBasedRow(new DateTime(ISOChronology.getInstanceUTC()), event); + } + + @Test + public void testApply() + { + event.put("count", new Double(10.0)); + event.put("avgCount", new Double(12.0)); + + Row result = pac.apply(row); + + assertEquals(result.getMetric("avgCountRatio").floatValue(), 10.0f / 12.0f, 0.0); + } + + @Test + public void testApplyMissingColumn() + { + event.put("count", new Double(10.0)); + + Row result = pac.apply(row); + + assertEquals(result.getMetric("avgCountRatio").floatValue(), 0.0, 0.0); + assertNull(result.getRaw("avgCountRatio")); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java new file mode 100644 index 000000000000..912b91ad0294 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -0,0 +1,672 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; + +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.joda.time.chrono.ISOChronology; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class RowBucketIterableTest +{ + + private static final DateTime JAN_1 = new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_2 = new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_3 = new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_8 = new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + private static final DateTime JAN_9 = new DateTime(2017, 1, 9, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); + + private static final Map EVENT_M_10 = new HashMap<>(); + private static final Map EVENT_F_20 = new HashMap<>(); + private static final Map EVENT_U_30 = new HashMap<>(); + + private static final Row JAN_1_M_10 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_1_F_20 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_1_U_30 = new MapBasedRow(new DateTime(2017, 1, 1, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + private static final Row JAN_2_M_10 = new MapBasedRow(new DateTime(2017, 1, 2, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_3_M_10 = new MapBasedRow(new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_3_F_20 = new MapBasedRow(new DateTime(2017, 1, 3, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_4_M_10 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_4_F_20 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_4_U_30 = new MapBasedRow(new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + private static final Row JAN_5_M_10 = new MapBasedRow(new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_6_M_10 = new MapBasedRow(new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_M_10); + private static final Row JAN_7_F_20 = new MapBasedRow(new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_F_20); + private static final Row JAN_8_U_30 = new MapBasedRow(new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()), EVENT_U_30); + + private static final Interval INTERVAL_JAN_1_1 = new Interval(JAN_1, JAN_2); + private static final Interval INTERVAL_JAN_1_2 = new Interval(JAN_1, JAN_3); + private static final Interval INTERVAL_JAN_1_4 = new Interval(JAN_1, JAN_5); + private static final Interval INTERVAL_JAN_1_5 = new Interval(JAN_1, JAN_6); + private static final Interval INTERVAL_JAN_6_8 = new Interval(JAN_6, JAN_9); + private static final Period ONE_DAY = Period.days(1); + + private List rows = null; + private List intervals = new ArrayList<>(); + + @BeforeClass + public static void setupClass() + { + EVENT_M_10.put("gender", "m"); + EVENT_M_10.put("pageViews", 10L); + EVENT_F_20.put("gender", "f"); + EVENT_F_20.put("pageViews", 20L); + EVENT_U_30.put("gender", "u"); + EVENT_U_30.put("pageViews", 30L); + } + + // normal case. data for all the days present + @Test + public void testCompleteData() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // all days present and last day only has one row + @Test + public void testApplyLastDaySingleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + } + + // all days present and last day has multiple rows + @Test + public void testApplyLastDayMultipleRows() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + rows.add(JAN_4_F_20); + rows.add(JAN_4_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + } + + // test single day with single row + @Test + public void testSingleDaySingleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_1); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + + List expected_day1 = Arrays.asList(JAN_1_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + assertEquals(JAN_1, actual.getDateTime()); + + } + + // test single day with multiple rows + @Test + public void testSingleDayMultipleRow() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_1); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_1_U_30); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20, JAN_1_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + } + + // missing day at the beginning followed by single row + @Test + public void testMissingDaysAtBegining() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_2); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + } + + // missing day at the beginning followed by multiple row + @Test + public void testMissingDaysAtBeginingFollowedByMultipleRow() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // missing day at the beginning and at the end + @Test + public void testMissingDaysAtBeginingAndAtTheEnd() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Collections.emptyList(); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_3_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // multiple missing days in an interval + @Test + public void testMultipleMissingDays() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // multiple missing days in an interval followed by multiple row at the end + @Test + public void testMultipleMissingDaysMultipleRowAtTheEnd() + { + + List expected_day1 = Collections.emptyList(); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day5 = Arrays.asList(JAN_5_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_5); + + rows = new ArrayList(); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + rows.add(JAN_5_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_5, actual.getDateTime()); + assertEquals(expected_day5, actual.getRows()); + } + + + // missing day in the middle followed by single row + @Test + public void testMissingDaysInMiddleOneRow() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_2_M_10); + rows.add(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + + } + + // missing day in the middle followed by multiple rows + @Test + public void testMissingDaysInMiddleMultipleRow() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day2 = Collections.emptyList(); + List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day4 = Arrays.asList(JAN_4_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_3_M_10); + rows.add(JAN_4_M_10); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(JAN_1, actual.getDateTime()); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_2, actual.getDateTime()); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + + } + + // data missing for last day . + @Test + public void testApplyLastDayNoRows() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Collections.emptyList(); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + // data missing for last two days + @Test + public void testApplyLastTwoDayNoRows() + { + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Collections.emptyList(); + List expected_day4 = Collections.emptyList(); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_3, actual.getDateTime()); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(JAN_4, actual.getDateTime()); + assertEquals(expected_day4, actual.getRows()); + } + + + @Test + public void testApplyMultipleInterval() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + intervals.add(INTERVAL_JAN_6_8); + + List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + List expected_day6 = Arrays.asList(JAN_6_M_10); + List expected_day7 = Arrays.asList(JAN_7_F_20); + List expected_day8 = Arrays.asList(JAN_8_U_30); + + rows = new ArrayList(); + rows.add(JAN_1_M_10); + rows.add(JAN_1_F_20); + rows.add(JAN_2_M_10); + rows.add(JAN_3_F_20); + rows.add(JAN_4_M_10); + rows.add(JAN_4_F_20); + rows.add(JAN_4_U_30); + rows.add(JAN_6_M_10); + rows.add(JAN_7_F_20); + rows.add(JAN_8_U_30); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + RowBucket actual = iter.next(); + assertEquals(expected_day1, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day2, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day3, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day4, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day6, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day7, actual.getRows()); + + actual = iter.next(); + assertEquals(expected_day8, actual.getRows()); + } + + @Test + public void testNodata() + { + + intervals = new ArrayList<>(); + intervals.add(INTERVAL_JAN_1_4); + intervals.add(INTERVAL_JAN_6_8); + + rows = new ArrayList(); + + Sequence seq = Sequences.simple(rows); + RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); + Iterator iter = rbi.iterator(); + + assertTrue(iter.hasNext()); + RowBucket actual = iter.next(); + assertEquals(Collections.emptyList(), actual.getRows()); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java new file mode 100644 index 000000000000..8b16d29dcc44 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Comparator; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class BaseAveragerFactoryTest +{ + + private AveragerFactory fac; + + @Before + public void setup() + { + fac = new BaseAveragerFactory("test", 5, "field", 1) + { + @Override + public Averager createAverager() + { + return null; + } + + @Override + public Comparator getComparator() + { + return null; + } + }; + } + + @Test + public void testGetDependentFields() throws Exception + { + List dependentFields = fac.getDependentFields(); + assertEquals(1, dependentFields.size()); + assertEquals("field", dependentFields.get(0)); + } + + @Test + public void testFinalization() + { + Long input = Long.valueOf(5L); + assertEquals(input, fac.finalizeComputation(input)); + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java new file mode 100644 index 000000000000..302cb4c0b25e --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test class for BaseAverager + */ +public class BaseAveragerTest +{ + + public static class TestAverager extends BaseAverager + { + public TestAverager(Class clazz, int b, String name, String field, int cycleSize) + { + super(clazz, b, name, field, cycleSize); + } + + @Override + protected Integer computeResult() + { + return 1; + } + } + + @Test + public void testBaseAverager() + { + BaseAverager avg = new TestAverager(Integer.class, 5, "test", "field", 1); + + assertEquals("test", avg.getName()); + assertEquals(5, avg.getNumBuckets()); + assertEquals(5, avg.getBuckets().length); + assertTrue(avg.getBuckets().getClass().isArray()); + } + + @Test + public void testAddElement() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + Object[] buckets = avg.getBuckets(); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + + avg.addElement(Collections.singletonMap("field", 2), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertNull(buckets[2]); + + avg.addElement(Collections.singletonMap("field", 3), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertEquals(Integer.valueOf(3), buckets[2]); + + avg.addElement(Collections.singletonMap("field", 4), Collections.emptyMap()); + assertEquals(Integer.valueOf(4), buckets[0]); + assertEquals(Integer.valueOf(2), buckets[1]); + assertEquals(Integer.valueOf(3), buckets[2]); + } + + @Test + public void testSkip() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + Object[] buckets = avg.getBuckets(); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + + assertEquals(Integer.valueOf(1), buckets[0]); + assertEquals(Integer.valueOf(1), buckets[1]); + assertEquals(Integer.valueOf(1), buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNotNull(buckets[1]); + assertNotNull(buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNotNull(buckets[2]); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + + // poke some test data into the array + buckets[0] = Integer.valueOf(1); + + avg.skip(); + assertNull(buckets[0]); + assertNull(buckets[1]); + assertNull(buckets[2]); + } + + @Test + public void testHasData() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + + assertFalse(avg.hasData()); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertTrue(avg.hasData()); + + avg.skip(); + avg.skip(); + avg.skip(); + + assertFalse(avg.hasData()); + } + + @Test + public void testGetResult() + { + BaseAverager avg = new TestAverager(Integer.class, 3, "test", "field", 1); + + assertNull(avg.getResult()); + + avg.addElement(Collections.singletonMap("field", 1), Collections.emptyMap()); + assertEquals(Integer.valueOf(1), avg.getResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java new file mode 100644 index 000000000000..acea22bc09f5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertThat; + + +public class DoubleMaxAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMaxAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMaxAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java new file mode 100644 index 000000000000..989c38ab7ab5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMaxAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = new DoubleMaxAverager(3, "test", "field", 1); + + assertEquals(Double.NEGATIVE_INFINITY, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", -1.1e100), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(5.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(3.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java new file mode 100644 index 000000000000..df023e58338c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMeanAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMeanAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMeanAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java new file mode 100644 index 000000000000..c0a8b39e254b --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = new DoubleMeanAverager(3, "test", "field", 1); + + assertEquals(0.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(4.0 / 3, avg.computeResult(), 0.0); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java new file mode 100644 index 000000000000..929485574885 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanAveragerWithPeriodTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager averager = new DoubleMeanAverager(14, "test", "field", 7); + + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + + assertEquals(7, averager.computeResult(), 0.0); // (7+7)/2 + + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(1, averager.computeResult(), 0.0); // (1+1)/2 + + BaseAverager averager1 = new DoubleMeanAverager(14, "test", "field", 3); + + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + averager1.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + + assertEquals(1, averager1.computeResult(), 0.0); // (1+1+1+1+1)/5 + + assertEquals(2, averager1.computeResult(), 0.0); // (2+2+2+2+2)/5 + + assertEquals(13.0 / 5, averager1.computeResult(), 0.0); // (3+3+3+3+1)/5 + + } +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java new file mode 100644 index 000000000000..070f44914b91 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMeanNoNullAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMeanNoNullAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMeanNoNullAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java new file mode 100644 index 000000000000..46a6efa89deb --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMeanNoNullAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new DoubleMeanNoNullAverager(3, "test", "field", 1); + + assertEquals(Double.NaN, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(2.0, avg.computeResult(), 0.0); + + // testing cycleSize functionality + BaseAverager averager = new DoubleMeanNoNullAverager(14, "test", "field", 7); + + averager.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + assertEquals(2.0, averager.computeResult(), 0.0); + + averager.addElement(Collections.singletonMap("field", 4.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 6.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 7.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 8.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 9.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", null), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 11.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 12.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 13.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 14.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 15.0), new HashMap<>()); + averager.addElement(Collections.singletonMap("field", 16.0), new HashMap<>()); + + assertEquals(7.5, averager.computeResult(), 0.0); + + averager.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(8.5, averager.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java new file mode 100644 index 000000000000..88aa4d35d894 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class DoubleMinAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new DoubleMinAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(DoubleMinAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java new file mode 100644 index 000000000000..f5ccab25b981 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class DoubleMinAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new DoubleMinAverager(3, "test", "field", 1); + + assertEquals(Double.POSITIVE_INFINITY, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", -1.1e100), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 1.0), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(-1.1e100, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 5.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2.0), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3.0), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + avg.skip(); + assertEquals(3.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java new file mode 100644 index 000000000000..ce90efd439f6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMaxAveragerFactoryTest +{ + + @Test + public void testCreateAverager() throws Exception + { + AveragerFactory fac = new LongMaxAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMaxAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java new file mode 100644 index 000000000000..24cc7bbb7349 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMaxAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMaxAverager(3, "test", "field", 1); + + assertEquals(Long.MIN_VALUE, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", -1000000L), new HashMap<>()); + assertEquals(-1000000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 1L), new HashMap<>()); + assertEquals(1, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", new Integer(1)), new HashMap<>()); + assertEquals(1, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 5L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(5, (long) avg.computeResult()); + + avg.skip(); + assertEquals(3, (long) avg.computeResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java new file mode 100644 index 000000000000..e164383f6778 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMeanAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMeanAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMeanAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java new file mode 100644 index 000000000000..7187c876fb20 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMeanAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMeanAverager(3, "test", "field", 1); + + assertEquals(0.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(1.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(3)), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(4.0 / 3, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java new file mode 100644 index 000000000000..0dcf9e5c274c --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMeanNoNullAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMeanNoNullAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMeanNoNullAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java new file mode 100644 index 000000000000..34b9696eda3d --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMeanNoNullAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMeanNoNullAverager(3, "test", "field", 1); + + assertEquals(Double.NaN, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(3.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", new Integer(0)), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + assertEquals(2.0, avg.computeResult(), 0.0); + + avg.skip(); + assertEquals(2.0, avg.computeResult(), 0.0); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java new file mode 100644 index 000000000000..48e75371c704 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class LongMinAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new LongMinAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(LongMinAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java new file mode 100644 index 000000000000..1b0f874db824 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class LongMinAveragerTest +{ + + @Test + public void testComputeResult() + { + BaseAverager avg = new LongMinAverager(3, "test", "field", 1); + + assertEquals(Long.MAX_VALUE, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", -10000L), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 1L), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", new Integer(1000)), new HashMap<>()); + assertEquals(-10000, (long) avg.computeResult()); + + avg.addElement(Collections.singletonMap("field", 5L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); + assertEquals(2, (long) avg.computeResult()); + + avg.skip(); + avg.skip(); + assertEquals(3, (long) avg.computeResult()); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java new file mode 100644 index 000000000000..68cae5d19d54 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import org.junit.Test; + +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertThat; + +public class SketchUnionAveragerFactoryTest +{ + + @Test + public void testCreateAverager() + { + AveragerFactory fac = new SketchUnionAveragerFactory("test", 5, 1, "field"); + assertThat(fac.createAverager(), instanceOf(SketchUnionAverager.class)); + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java new file mode 100644 index 000000000000..0ff62f2ae409 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; + +import com.yahoo.sketches.Family; +import com.yahoo.sketches.Util; +import com.yahoo.sketches.theta.SetOperation; +import com.yahoo.sketches.theta.Union; +import com.yahoo.sketches.theta.UpdateSketch; +import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; + +public class SketchUnionAveragerTest +{ + + @Test + public void testComputeResult() throws Exception + { + BaseAverager avg = + new SketchUnionAverager(2, "test", "field", 1, Util.DEFAULT_NOMINAL_ENTRIES); + + UpdateSketch sketch1 = UpdateSketch.builder().build(); + for (int key = 0; key < 16; key++) { + sketch1.update(key); + } + + UpdateSketch sketch2 = UpdateSketch.builder().build(); + for (int key = 16; key < 32; key++) { + sketch2.update(key); + } + + UpdateSketch sketch3 = UpdateSketch.builder().build(); + for (int key = 32; key < 48; key++) { + sketch2.update(key); + } + + SketchHolder holder1 = SketchHolder.of(sketch1); + SketchHolder holder2 = SketchHolder.of(sketch2); + SketchHolder holder3 = SketchHolder.of(sketch3); + + Union union1 = (Union) SetOperation.builder().build(Family.UNION); + union1.update(sketch1); + union1.update(sketch2); + + avg.addElement(Collections.singletonMap("field", holder1), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", holder2), new HashMap<>()); + + assertEquals(avg.computeResult().getEstimate(), union1.getResult().getEstimate(), 0); + + avg.addElement(Collections.singletonMap("field", holder3), new HashMap<>()); + + Union union2 = (Union) SetOperation.builder().build(Family.UNION); + union2.update(sketch2); + union2.update(sketch3); + + assertEquals(avg.computeResult().getEstimate(), union2.getResult().getEstimate(), 0); + + } + +} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java new file mode 100644 index 000000000000..92c5715b954f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.test; + +import java.util.Map; + +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; + +/** + * Configuration for a unit test. + */ +public class TestConfig +{ + public ObjectNode query; + public ArrayNode expectedOutput; + public Map intermediateResults; +} diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml new file mode 100644 index 000000000000..a3d2d164d2dc --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage.yaml @@ -0,0 +1,57 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml new file mode 100644 index 000000000000..a3d2d164d2dc --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicGroupByMovingAverage2.yaml @@ -0,0 +1,57 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml new file mode 100644 index 000000000000..1458ed8c2af6 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/basicTimeseriesMovingAverage.yaml @@ -0,0 +1,51 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: [] + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + timeseries: + - timestamp: 2017-01-01T00:00Z + result: + timespent_secs: 120.0 + timeSpent: 2.0 + - timestamp: 2017-01-02T00:00Z + result: + timespent_secs: 240.0 + timeSpent: 4.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml new file mode 100644 index 000000000000..626b94c23c7f --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml @@ -0,0 +1,79 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + - name: regBcookies + fieldName: bcookies_with_yuid_sketch + size: 16384 + type: thetaSketch + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 + regBcookies: 0.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 0 + timeSpent: 0.0 + trailing7DayAvgTimeSpent: 1.0 + regBcookies: 0.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240 + timeSpent: 4.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml new file mode 100644 index 000000000000..460de3f4cfd5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml @@ -0,0 +1,70 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pr_pc_pl_us_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-01T00:00Z/2017-01-02T00:00Z + dimensions: + - gender + averagers: + - name: regBcookiesAvg + fieldName: regBcookies + size: 16384 + buckets: 2 + type: sketchUnion + aggregations: + - name: regBcookies + fieldName: bcookie_with_yuid_sketch + size: 16384 + type: thetaSketch + postAveragers: [ + ] + postAggregations: [ + ] +expectedOutput: +- version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: 0.0 + regBcookiesAvg: 0.0 +- version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: 0.0 + regBcookiesAvg: 0.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2016-12-31T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 120 + timeSpent: 2.0 + regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml new file mode 100644 index 000000000000..ba685ff8411a --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersAsc.yaml @@ -0,0 +1,81 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: trailing7DayAvgTimeSpent +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + trailing7DayAvgTimeSpent: 6.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml new file mode 100644 index 000000000000..59f75bc04807 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingAveragersDesc.yaml @@ -0,0 +1,82 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_pl_dt_os + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 2 + name: trailing7DayAvgTimeSpent + fieldName: timeSpent + type: doubleMean + aggregations: + - name: timespent_secs + fieldName: timespent + type: longSum + postAggregations: + - type: arithmetic + name: timeSpent + fn: / + fields: + - type: fieldAccess + fieldName: timespent_secs + - type: constant + name: seconds_per_minute + value: 60.0 + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: trailing7DayAvgTimeSpent + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + trailing7DayAvgTimeSpent: 6.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + trailing7DayAvgTimeSpent: 3.0 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + timespent_secs: 120.0 + timeSpent: 2.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + timespent_secs: 240.0 + timeSpent: 4.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + timespent_secs: 480.0 + timeSpent: 8.0 + \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml new file mode 100644 index 000000000000..c7d7ddc05110 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAndMovingAvgMetric.yaml @@ -0,0 +1,84 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_uc_ud + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 7 + name: trailing7DayAvgTotalPageViews + fieldName: totalPageViews + type: doubleMean + aggregations: + - name: addPageViews + fieldName: additive_page_views + type: longSum + - name: pageViews + fieldName: other_page_views + type: longSum + postAggregations: + - type: arithmetic + name: totalPageViews + fn: + + fields: + - type: fieldAccess + fieldName: addPageViews + - type: fieldAccess + fieldName: pageViews + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: addPageViews + direction: DESC + dimension: trailing7DayAvgTotalPageViews + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 + trailing7DayAvgTotalPageViews: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + addPageViews: 0 + pageViews: 0 + totalPageViews: 0.0 + trailing7DayAvgTotalPageViews: 2.142857142857143 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + addPageViews: 5.0 + pageViews: 10.0 + totalPageViews: 15.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + addPageViews: 6.0 + pageViews: 12.0 + totalPageViews: 18.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml new file mode 100644 index 000000000000..89ae94155e97 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/sortingWithNonMovingAvgMetric.yaml @@ -0,0 +1,82 @@ +query: + queryType: movingAverage + dataSource: + type: table + name: slice_pf_us_uc_ud + context: { + } + granularity: + type: period + period: P1D + intervals: + - 2017-01-02T00:00Z/2017-01-03T00:00Z + dimensions: + - gender + averagers: + - buckets: 7 + name: trailing7DayAvgTotalPageViews + fieldName: totalPageViews + type: doubleMean + aggregations: + - name: addPageViews + fieldName: additive_page_views + type: longSum + - name: pageViews + fieldName: other_page_views + type: longSum + postAggregations: + - type: arithmetic + name: totalPageViews + fn: + + fields: + - type: fieldAccess + fieldName: addPageViews + - type: fieldAccess + fieldName: pageViews + postAveragers: [ + ] + limitSpec: + type: default + columns: + - dimension: addPageViews + direction: DESC +expectedOutput: +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 + trailing7DayAvgTotalPageViews: 3.0 +- version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: m + addPageViews: 0 + pageViews: 0 + totalPageViews: 0.0 + trailing7DayAvgTotalPageViews: 2.142857142857143 +intermediateResults: + groupBy: + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: m + addPageViews: 5.0 + pageViews: 10.0 + totalPageViews: 15.0 + - version: v1 + timestamp: 2017-01-01T00:00Z + event: + gender: f + addPageViews: 6.0 + pageViews: 12.0 + totalPageViews: 18.0 + - version: v1 + timestamp: 2017-01-02T00:00Z + event: + gender: f + addPageViews: 1.0 + pageViews: 2.0 + totalPageViews: 3.0 \ No newline at end of file diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties new file mode 100644 index 000000000000..cf60cde85fe5 --- /dev/null +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -0,0 +1,57 @@ +druid.broker.cache.unCacheable=["groupBy","segmentMetadata"] +druid.broker.http.numConnections=50 +druid.server.http.numThreads=1 +druid.announcer.maxBytesPerNode=524288 +druid.announcer.segmentsPerNode=50 +druid.announcer.type=batch +druid.broker.balancer.type=random +druid.broker.cache.initialSize=500000 +druid.broker.cache.logEvictionCount=0 +druid.broker.cache.sizeInBytes=0 +druid.broker.cache.type=local +druid.broker.http.readTimeOut=PT15M +druid.broker.select.tier=highestPriority +druid.broker.select.tier.custom.priorities= +druid.coordinator.conversion.on=false +druid.coordinator.load.timeout=PT15M +druid.coordinator.merge.on=false +druid.coordinator.period=PT30S +druid.coordinator.period.indexingPeriod=PT1800S +druid.coordinator.startDelay=PT60S +druid.curator.compress=false +druid.emitter.logging.loggerClass=LoggingEmitter +druid.emitter.logging.logLevel=info +druid.extensions.coordinates=[] +druid.manager.config.pollDuration=PT1M +druid.manager.rules.defaultTier=tier1 +druid.manager.rules.pollDuration=PT1M +druid.manager.segment.pollDuration=PT1M +druid.monitoring.emissionPeriod=PT1M +druid.monitoring.monitors=[\"com.metamx.metrics.JvmMonitor\"] +druid.processing.buffer.sizeBytes=655360 +druid.processing.numThreads=1 +druid.processing.columnCache.sizeBytes=0 +druid.processing.formatString='processing-%s' +druid.publish.type=metadata +druid.query.chunkPeriod=P1M +druid.query.groupBy.defaultStrategy=v1 +druid.query.groupBy.maxIntermediateRows=50000 +druid.query.groupBy.maxResults=5000000 +druid.query.groupBy.singleThreaded=false +druid.query.search.maxSearchLimit=1000 +druid.request.logging.feed= +druid.request.logging.dir= +druid.request.logging.type=noop +druid.segmentCache.announceIntervalMillis=5000 +druid.segmentCache.deleteOnRemove=true +druid.segmentCache.dropSegmentDelayMillis=30000 +druid.segmentCache.numLoadingThreads=1 +druid.selectors.indexing.serviceName=overlord +druid.server.http.maxIdleTime=PT5m +druid.storage.type=hdfs +druid.worker.capacity=30 +druid.server.maxSize=1073741824 +druid.server.priority=0 +druid.server.tier=_default_tier +druid.worker.ip=localhost +druid.worker.version=0 diff --git a/pom.xml b/pom.xml index 1ab8ad1b6c42..824b8619a799 100644 --- a/pom.xml +++ b/pom.xml @@ -169,6 +169,7 @@ extensions-contrib/materialized-view-maintenance extensions-contrib/materialized-view-selection extensions-contrib/momentsketch + extensions-contrib/moving-average-query distribution @@ -1507,4 +1508,4 @@ - + \ No newline at end of file From 0e348298b71345076c30064f98004b19b9ddfcb7 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 8 Oct 2018 23:16:21 -0700 Subject: [PATCH 13/45] Fix failing code inspections. --- .../movingaverage/MovingAverageQuery.java | 12 ++-- .../MovingAverageQueryModule.java | 9 ++- .../MovingAverageQueryRunner.java | 5 +- .../movingaverage/RowBucketIterable.java | 2 +- .../MovingAverageIterableTest.java | 60 ++++++++-------- .../movingaverage/MovingAverageQueryTest.java | 6 +- .../movingaverage/RowBucketIterableTest.java | 70 +++++++++---------- 7 files changed, 86 insertions(+), 78 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 52ddd83a230c..f10efd0626c3 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -94,14 +94,14 @@ public MovingAverageQuery( this.dimFilter = dimFilter; this.granularity = granularity; - this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; + this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; for (DimensionSpec spec : this.dimensions) { Preconditions.checkArgument(spec != null, "dimensions has null DimensionSpec"); } - this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; - this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; - this.averagerSpecs = averagerSpecs == null ? ImmutableList.>of() : averagerSpecs; - this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; + this.aggregatorSpecs = aggregatorSpecs == null ? ImmutableList.of() : aggregatorSpecs; + this.postAggregatorSpecs = postAggregatorSpecs == null ? ImmutableList.of() : postAggregatorSpecs; + this.averagerSpecs = averagerSpecs == null ? ImmutableList.of() : averagerSpecs; + this.postAveragerSpecs = postAveragerSpecs == null ? ImmutableList.of() : postAveragerSpecs; this.havingSpec = havingSpec; this.limitSpec = (limitSpec == null) ? NoopLimitSpec.INSTANCE : limitSpec; @@ -232,12 +232,14 @@ public boolean getContextSortByDimsFirst() return getContextBoolean(CTX_KEY_SORT_BY_DIMS_FIRST, false); } + @Override @JsonProperty public DimFilter getFilter() { return dimFilter; } + @Override @JsonProperty public Granularity getGranularity() { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java index a7efa358e666..05ae2560d6f5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -32,7 +32,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryToolChest; -import java.util.Arrays; +import java.util.Collections; import java.util.List; public class MovingAverageQueryModule implements DruidModule @@ -53,8 +53,11 @@ public void configure(Binder binder) @Override public List getJacksonModules() { - return Arrays.asList(new SimpleModule("MovingAverageQueryModule") - .registerSubtypes(new NamedType(MovingAverageQuery.class, "movingAverage"))); + return Collections.singletonList(new SimpleModule("MovingAverageQueryModule") + .registerSubtypes(new NamedType( + MovingAverageQuery.class, + "movingAverage" + ))); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 1bf421ee1899..cb0c1b4c55ac 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -141,7 +141,7 @@ public Sequence run(QueryPlus query, Map responseConte "127.0.0.1", gbq, new QueryStats( - ImmutableMap.of( + ImmutableMap.of( "query/time", 0, "query/bytes", 0, "success", true @@ -179,7 +179,7 @@ public Sequence run(QueryPlus query, Map responseConte "127.0.0.1", tsq, new QueryStats( - ImmutableMap.of( + ImmutableMap.of( "query/time", 0, "query/bytes", 0, "success", true @@ -228,6 +228,7 @@ public Sequence run(QueryPlus query, Map responseConte static class TimeseriesResultToRow implements Function, Row> { + @Override public Row apply(Result lookbackResult) { Map event = lookbackResult.getValue().getBaseObject(); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 25dfe246ffca..f0201fbac784 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -76,7 +76,7 @@ public RowIterator(Sequence rows, List intervals, Period period) this.intervals = intervals; expectedBucket = intervals.get(intervalIndex).getStart(); endTime = intervals.get(intervals.size() - 1).getEnd(); - yielder = rows.toYielder(null, new BucketingAccumulator()); + yielder = rows.toYielder(null, new BucketingAccumulator()); } /* (non-Javadoc) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 9e89b69bfb18..415d13862911 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -106,7 +106,7 @@ public void testNext() new MapBasedRow(JAN_1, dims1), new MapBasedRow(JAN_1, dims2) )), - new RowBucket(JAN_2, Arrays.asList( + new RowBucket(JAN_2, Collections.singletonList( new MapBasedRow(JAN_2, dims1) )), new RowBucket(JAN_3, Collections.emptyList()), @@ -217,8 +217,8 @@ public void testAveraging() new LongMeanAveragerFactory("movingAvgPageViews", 7, 1, "pageViews") ), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -281,11 +281,11 @@ public void testCompleteData() new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -349,15 +349,15 @@ public void testMissingDataAtBeginning() Row jan_2_row3 = new MapBasedRow(JAN_2, event3); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan_1_row1)), + new RowBucket(JAN_1, Collections.singletonList(jan_1_row1)), new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -410,14 +410,14 @@ public void testMissingDataAtTheEnd() Sequence seq = Sequences.simple(Arrays.asList( new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1)) + new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 2, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -485,16 +485,16 @@ public void testMissingDataAtMiddle() Sequence seq = Sequences.simple(Arrays.asList( new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1)), + new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)), new RowBucket(JAN_3, Arrays.asList(jan_3_row1, jan_3_row2, jan_3_row3)), - new RowBucket(JAN_4, Arrays.asList(jan_4_row1)) + new RowBucket(JAN_4, Collections.singletonList(jan_4_row1)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 3, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -586,11 +586,11 @@ public void testMissingDaysAtBegining() new RowBucket(JAN_4, Collections.singletonList(row2)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -632,11 +632,11 @@ public void testMissingDaysInMiddle() new RowBucket(JAN_4, Collections.singletonList(row2)) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); @@ -693,10 +693,10 @@ public void testWithFilteredAggregation() DimFilter filter = new SelectorDimFilter("gender", "m", null); FilteredAggregatorFactory filteredAggregatorFactory = new FilteredAggregatorFactory(aggregatorFactory, filter); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( averagerfactory), Collections.emptyList(), - Collections.singletonList( + Collections.singletonList( filteredAggregatorFactory) ).iterator(); @@ -750,11 +750,11 @@ public void testMissingDaysAtEnd() new RowBucket(JAN_6, Collections.emptyList()) )); - Iterator iter = new MovingAverageIterable(seq, ds, Arrays.asList( + Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( new LongMeanAveragerFactory("movingAvgPageViews", 4, 1, "pageViews")), Collections.emptyList(), - Collections.singletonList(new LongSumAggregatorFactory("pageViews", - "pageViews" + Collections.singletonList(new LongSumAggregatorFactory("pageViews", + "pageViews" )) ).iterator(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index aaadd2344ec8..3e1313d7983e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.http.DruidHttpClientConfig; import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -131,7 +132,7 @@ public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMa binder.bindConstant().annotatedWith(Names.named("serviceName")).to("queryTest"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(1); - binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); + binder.bind(QuerySegmentWalker.class).toProvider(Providers.of(null)); } ); @@ -420,7 +421,8 @@ public long getMaxQueuedBytes() ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( new ServiceEmitter("", "", null) { - public void emit(org.apache.druid.java.util.emitter.core.Event event) {} + @Override + public void emit(Event event) {} }, baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() ); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 912b91ad0294..6746a1e31e74 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -107,10 +107,10 @@ public void testCompleteData() rows.add(JAN_3_M_10); rows.add(JAN_4_M_10); - List expected_day1 = Arrays.asList(JAN_1_M_10); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -142,9 +142,9 @@ public void testApplyLastDaySingleRow() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); + List expected_day4 = Collections.singletonList(JAN_4_M_10); rows = new ArrayList(); rows.add(JAN_1_M_10); @@ -179,8 +179,8 @@ public void testApplyLastDayMultipleRows() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); rows = new ArrayList(); @@ -220,7 +220,7 @@ public void testSingleDaySingleRow() rows = new ArrayList(); rows.add(JAN_1_M_10); - List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -263,7 +263,7 @@ public void testMissingDaysAtBegining() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_2); @@ -291,9 +291,9 @@ public void testMissingDaysAtBeginingFollowedByMultipleRow() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -330,8 +330,8 @@ public void testMissingDaysAtBeginingAndAtTheEnd() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); List expected_day4 = Collections.emptyList(); intervals = new ArrayList<>(); @@ -368,9 +368,9 @@ public void testMultipleMissingDays() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -406,10 +406,10 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() { List expected_day1 = Collections.emptyList(); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); - List expected_day5 = Arrays.asList(JAN_5_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expected_day5 = Collections.singletonList(JAN_5_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_5); @@ -450,10 +450,10 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() public void testMissingDaysInMiddleOneRow() { - List expected_day1 = Arrays.asList(JAN_1_M_10); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); rows = new ArrayList(); rows.add(JAN_1_M_10); @@ -487,10 +487,10 @@ public void testMissingDaysInMiddleOneRow() public void testMissingDaysInMiddleMultipleRow() { - List expected_day1 = Arrays.asList(JAN_1_M_10); + List expected_day1 = Collections.singletonList(JAN_1_M_10); List expected_day2 = Collections.emptyList(); - List expected_day3 = Arrays.asList(JAN_3_M_10); - List expected_day4 = Arrays.asList(JAN_4_M_10); + List expected_day3 = Collections.singletonList(JAN_3_M_10); + List expected_day4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); @@ -531,8 +531,8 @@ public void testApplyLastDayNoRows() intervals.add(INTERVAL_JAN_1_4); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Collections.emptyList(); rows = new ArrayList(); @@ -565,7 +565,7 @@ public void testApplyLastTwoDayNoRows() { List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); + List expected_day2 = Collections.singletonList(JAN_2_M_10); List expected_day3 = Collections.emptyList(); List expected_day4 = Collections.emptyList(); @@ -606,12 +606,12 @@ public void testApplyMultipleInterval() intervals.add(INTERVAL_JAN_6_8); List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Arrays.asList(JAN_2_M_10); - List expected_day3 = Arrays.asList(JAN_3_F_20); + List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expected_day3 = Collections.singletonList(JAN_3_F_20); List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); - List expected_day6 = Arrays.asList(JAN_6_M_10); - List expected_day7 = Arrays.asList(JAN_7_F_20); - List expected_day8 = Arrays.asList(JAN_8_U_30); + List expected_day6 = Collections.singletonList(JAN_6_M_10); + List expected_day7 = Collections.singletonList(JAN_7_F_20); + List expected_day8 = Collections.singletonList(JAN_8_U_30); rows = new ArrayList(); rows.add(JAN_1_M_10); From ae75015513b10f0070ca4b728558e5aa9cd71e6b Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Tue, 9 Oct 2018 12:29:15 -0700 Subject: [PATCH 14/45] See if explicit types will invoke the correct comparison function. --- .../druid/query/movingaverage/MovingAverageQueryTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 3e1313d7983e..43dae733a793 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -366,7 +366,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) Query query = jsonMapper.readValue(getQueryString(), Query.class); assertThat(query, IsInstanceOf.instanceOf(getExpectedQueryType())); - List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); + List expectedResults = jsonMapper.readValue(getExpectedResultString(), getExpectedResultType()); assertNotNull(expectedResults); assertThat(expectedResults, IsInstanceOf.instanceOf(List.class)); @@ -434,7 +434,7 @@ baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheCon final Sequence res = query.getRunner(walker).run(queryPlus, responseContext); List actualResults = new ArrayList(); - actualResults = (List) res.accumulate(actualResults, Accumulators.list()); + actualResults = (List) res.accumulate(actualResults, Accumulators.list()); consistentTypeCasting(expectedResults); consistentTypeCasting(actualResults); From 542ecd211a9ee9887610689e1616676809f20c01 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 10 Oct 2018 13:29:54 -0700 Subject: [PATCH 15/45] Explicitly remove support for druid.generic.useDefaultValueForNull configuration parameter. --- .../development/extensions-contrib/moving-average-query.md | 7 +++++-- .../druid/query/movingaverage/MovingAverageQuery.java | 4 ++++ .../query/movingaverage/MovingAverageIterableTest.java | 2 ++ .../druid/query/movingaverage/MovingAverageQueryTest.java | 2 +- .../query/movingaverage/PostAveragerCalculatorTest.java | 1 + 5 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 387b397ae991..37aaa99842c9 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -35,10 +35,13 @@ To use this extension, make sure to [load](../../operations/including-extensions ##Configuration There are currently no configuration properties specific to Moving Average. +##Limitations +* movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. +* movingAverage is missing support for the following timeseries properties: `descending`. +* movingAverage is missing support for [SQL-compatible null handling](https://github.com/apache/incubator-druid/issues/4349) (So setting druid.generic.useDefaultValueForNull in configuration will give an error). + ##Query spec: * Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. -* Note: movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. -* Note: movingAverage is missing support for the following timeseries properties: `descending`. |property|description|required?| |--------|-----------|---------| diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index f10efd0626c3..6d5fdde5c0fd 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -30,6 +30,7 @@ import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; @@ -92,6 +93,9 @@ public MovingAverageQuery( { super(dataSource, querySegmentSpec, false, context); + //TBD: Implement null awareness to respect the contract of this flag. + Preconditions.checkArgument(NullHandling.replaceWithDefault(), "movingAverage does not support druid.generic.useDefaultValueForNull=false"); + this.dimFilter = dimFilter; this.granularity = granularity; this.dimensions = dimensions == null ? ImmutableList.of() : dimensions; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 415d13862911..27c6a3148abb 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -610,6 +610,7 @@ public void testMissingDaysAtBegining() @Test public void testMissingDaysInMiddle() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Map event1 = new HashMap<>(); Map event2 = new HashMap<>(); @@ -726,6 +727,7 @@ public void testWithFilteredAggregation() @Test public void testMissingDaysAtEnd() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Map event1 = new HashMap<>(); Map event2 = new HashMap<>(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 43dae733a793..491af896545c 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -136,7 +136,7 @@ public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMa } ); - + System.setProperty("druid.generic.useDefaultValueForNull", "true"); Injector baseInjector = GuiceInjectors.makeStartupInjector(); injector = Initialization.makeInjectorWithModules(baseInjector, modules); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java index dbd127fdf7e6..4073302c7b0a 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -57,6 +57,7 @@ public class PostAveragerCalculatorTest @Before public void setup() { + System.setProperty("druid.generic.useDefaultValueForNull", "true"); query = new MovingAverageQuery( new TableDataSource("d"), new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval("2017-01-01/2017-01-01", ISOChronology.getInstanceUTC()))), From 4a04cc7349d4ca5d4b2059ed74f2262e203ee914 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 10:46:32 -0800 Subject: [PATCH 16/45] Update styling and headers for complience. --- .../moving-average-query.md | 19 +++++++ .../moving-average-query/pom.xml | 20 +++---- .../movingaverage/AveragerFactoryWrapper.java | 9 +-- .../movingaverage/BucketingAccumulator.java | 7 ++- .../DefaultMovingAverageQueryMetrics.java | 1 + ...faultMovingAverageQueryMetricsFactory.java | 1 + .../IdentityYieldingAccumulator.java | 1 + .../movingaverage/MovingAverageHelper.java | 8 +-- .../movingaverage/MovingAverageIterable.java | 6 +- .../movingaverage/MovingAverageQuery.java | 2 +- .../MovingAverageQueryMetrics.java | 1 + .../MovingAverageQueryMetricsFactory.java | 1 + .../MovingAverageQueryModule.java | 2 - .../MovingAverageQueryRunner.java | 26 ++++----- .../MovingAverageQueryToolChest.java | 9 ++- .../movingaverage/PostAveragerCalculator.java | 7 ++- .../druid/query/movingaverage/RowBucket.java | 4 +- .../movingaverage/RowBucketIterable.java | 8 +-- .../movingaverage/averagers/Averager.java | 4 +- .../averagers/AveragerFactory.java | 7 ++- .../movingaverage/averagers/BaseAverager.java | 11 ++-- .../averagers/BaseAveragerFactory.java | 7 ++- .../averagers/ComparableAveragerFactory.java | 4 +- .../averagers/ConstantAverager.java | 5 +- .../averagers/ConstantAveragerFactory.java | 6 +- .../averagers/DoubleMaxAverager.java | 1 + .../averagers/DoubleMaxAveragerFactory.java | 1 + .../averagers/DoubleMeanAverager.java | 1 + .../averagers/DoubleMeanAveragerFactory.java | 1 + .../averagers/DoubleMeanNoNullAverager.java | 1 + .../DoubleMeanNoNullAveragerFactory.java | 1 + .../averagers/DoubleMinAverager.java | 1 + .../averagers/DoubleMinAveragerFactory.java | 1 + .../averagers/LongMaxAverager.java | 1 + .../averagers/LongMaxAveragerFactory.java | 1 + .../averagers/LongMeanAverager.java | 1 + .../averagers/LongMeanAveragerFactory.java | 1 + .../averagers/LongMeanNoNullAverager.java | 1 + .../LongMeanNoNullAveragerFactory.java | 1 + .../averagers/LongMinAverager.java | 1 + .../averagers/LongMinAveragerFactory.java | 1 + .../averagers/SketchUnionAverager.java | 1 + .../averagers/SketchUnionAveragerFactory.java | 5 +- .../IdentityYieldingAccumulatorTest.java | 8 +-- .../MovingAverageIterableTest.java | 33 ++++++----- .../movingaverage/MovingAverageQueryTest.java | 55 +++++++++---------- .../PostAveragerCalculatorTest.java | 18 +++--- .../movingaverage/RowBucketIterableTest.java | 8 +-- .../averagers/BaseAveragerFactoryTest.java | 1 + .../averagers/BaseAveragerTest.java | 1 + .../DoubleMaxAveragerFactoryTest.java | 1 + .../averagers/DoubleMaxAveragerTest.java | 1 + .../DoubleMeanAveragerFactoryTest.java | 1 + .../averagers/DoubleMeanAveragerTest.java | 1 + .../DoubleMeanAveragerWithPeriodTest.java | 1 + .../DoubleMeanNoNullAveragerFactoryTest.java | 1 + .../DoubleMeanNoNullAveragerTest.java | 1 + .../DoubleMinAveragerFactoryTest.java | 1 + .../averagers/DoubleMinAveragerTest.java | 1 + .../averagers/LongMaxAveragerFactoryTest.java | 1 + .../averagers/LongMaxAveragerTest.java | 1 + .../LongMeanAveragerFactoryTest.java | 1 + .../averagers/LongMeanAveragerTest.java | 1 + .../LongMeanNoNullAveragerFactoryTest.java | 1 + .../averagers/LongMeanNoNullAveragerTest.java | 1 + .../averagers/LongMinAveragerFactoryTest.java | 1 + .../averagers/LongMinAveragerTest.java | 1 + .../SketchUnionAveragerFactoryTest.java | 1 + .../averagers/SketchUnionAveragerTest.java | 1 + .../query/movingaverage/test/TestConfig.java | 5 +- 70 files changed, 203 insertions(+), 143 deletions(-) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 37aaa99842c9..804b4fdb6866 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -2,6 +2,25 @@ layout: doc_page --- + + # Moving Average Queries ## Overview diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index e9c4e77671c0..a7e76d1bca97 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -21,11 +21,11 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 - + - druid org.apache.druid - 0.13.0-SNAPSHOT + druid + 0.13.0-incubating-SNAPSHOT ../../pom.xml @@ -63,7 +63,7 @@ org.apache.druid - druid-api + druid-core ${project.parent.version} provided @@ -73,17 +73,17 @@ ${project.parent.version} provided - - org.apache.druid - druid-server - ${project.parent.version} - provided - org.apache.druid.extensions druid-datasketches ${project.parent.version} provided + + org.apache.druid + druid-server + 0.13.0-incubating-SNAPSHOT + compile + diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 00b5982db946..198f898d9f18 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -16,17 +16,18 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.Comparator; -import java.util.List; +package org.apache.druid.query.movingaverage; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.segment.ColumnSelectorFactory; +import java.util.Comparator; +import java.util.List; + /** * A wrapper around averagers that makes them appear to be aggregators. * This is necessary purely to allow existing common druid code that only knows diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java index 34ec4353e65a..1aa8782b242f 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.ArrayList; -import java.util.Collection; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import java.util.ArrayList; +import java.util.Collection; + /** * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java index 857e48201f2e..8dced39ccc97 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetrics.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java index d494a2e48b9b..64b7c5b6910b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java index b0250b0a5c7e..4171a72781cb 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.java.util.common.guava.YieldingAccumulator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java index 4188af4cba7c..f0cfb0bd15bf 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageHelper.java @@ -19,14 +19,14 @@ package org.apache.druid.query.movingaverage; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.dimension.DimensionSpec; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + public class MovingAverageHelper { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 1cf6aa25a51a..c01aaa4f5a65 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -16,10 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import org.apache.druid.query.movingaverage.averagers.Averager; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -29,6 +27,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.movingaverage.averagers.Averager; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 6d5fdde5c0fd..7f12e5e1ab57 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -31,7 +31,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularity; @@ -47,6 +46,7 @@ import org.apache.druid.query.groupby.having.HavingSpec; import org.apache.druid.query.groupby.orderby.LimitSpec; import org.apache.druid.query.groupby.orderby.NoopLimitSpec; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.spec.QuerySegmentSpec; import java.util.ArrayList; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java index 48a8e564a2c3..6b9f39ad0f2e 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetrics.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.query.QueryMetrics; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java index 3857d323b7fd..db344a0f0ec5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryMetricsFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; /** diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java index 05ae2560d6f5..9655678680da 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryModule.java @@ -22,10 +22,8 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; - import com.google.inject.Binder; import com.google.inject.multibindings.MapBinder; - import org.apache.druid.guice.DruidBinders; import org.apache.druid.guice.LazySingleton; import org.apache.druid.initialization.DruidModule; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index cb0c1b4c55ac..d64ba18222d2 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -19,46 +19,44 @@ package org.apache.druid.query.movingaverage; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.query.QueryContexts; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DataSource; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; -import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.QueryStats; import org.apache.druid.server.RequestLogLine; import org.apache.druid.server.log.RequestLogger; +import org.joda.time.Interval; +import org.joda.time.Period; import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; /** * The QueryRunner for MovingAverage query. diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index e35b22df32d0..1bc9a8a37273 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -19,16 +19,11 @@ package org.apache.druid.query.movingaverage; -import java.util.Map; - -import javax.annotation.Nullable; - import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.collect.Maps; import com.google.inject.Inject; -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.QueryMetrics; @@ -38,8 +33,12 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.server.log.RequestLogger; +import javax.annotation.Nullable; +import java.util.Map; + /** * The QueryToolChest for MovingAverage Query */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java index 39e988b4bb76..3490f3e4192c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java @@ -16,16 +16,17 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import java.util.List; -import java.util.Map; +package org.apache.druid.query.movingaverage; import com.google.common.base.Function; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.aggregation.PostAggregator; +import java.util.List; +import java.util.Map; + /** * Function that can be applied to a Sequence to calculate PostAverager results */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java index 8bf058fdf68f..2c96530a209a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -16,14 +16,14 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.Row; import org.joda.time.DateTime; import java.util.Collection; -import org.apache.druid.data.input.Row; - /** * Represents a set of rows for a specific date */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index f0201fbac784..1e1fda717215 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -16,8 +16,12 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -27,10 +31,6 @@ import java.util.List; import java.util.NoSuchElementException; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; - /** * It is the iterable used to bucket data into days, * doing appropriate lookahead to see if the next row is in the same day or a new day. diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java index 20da12ca3fca..506380cac1bb 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/Averager.java @@ -19,10 +19,10 @@ package org.apache.druid.query.movingaverage.averagers; -import java.util.Map; - import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Map; + /** * Interface for an averager * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java index e8f2ad4fe7d5..e2041b0db40a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Comparator; -import java.util.List; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import java.util.Comparator; +import java.util.List; + /** * Interface representing Averager in the movingAverage query. * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index e525deb223e4..37497c695ead 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -16,13 +16,14 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; +import org.apache.druid.query.aggregation.AggregatorFactory; + import java.lang.reflect.Array; import java.util.Map; -import org.apache.druid.query.aggregation.AggregatorFactory; - /** * Common base class available for use by averagers. The base class implements methods that * capture incoming and skipped rows and store them in an array, to be used later for @@ -53,10 +54,8 @@ public abstract class BaseAverager implements Averager * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). * @param shouldFinalize Whether the intermediate values need to be finalized prior to averaging */ - public BaseAverager( - Class storageType, int numBuckets, String name, String fieldName, int cycleSize, - boolean shouldFinalize - ) + public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize, + boolean shouldFinalize) { this.numBuckets = numBuckets; this.name = name; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java index d91e74f687b7..dda08c04be08 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java @@ -16,14 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Collections; -import java.util.List; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; + /** * Common base class for AveragerFactories * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java index ca76b505ae45..0463d55b97da 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ComparableAveragerFactory.java @@ -16,9 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -/** - * Copyright 2018 Oath Inc. All rights reserved. - */ + package org.apache.druid.query.movingaverage.averagers; import java.util.Comparator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java index 806fa6b21242..bc76c99610f9 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAverager.java @@ -16,12 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Map; +package org.apache.druid.query.movingaverage.averagers; import org.apache.druid.query.aggregation.AggregatorFactory; +import java.util.Map; + /** * The constant averager.Created soley for incremental development and wiring things up. */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java index cffc876fa399..45339c37058b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/ConstantAveragerFactory.java @@ -19,13 +19,13 @@ package org.apache.druid.query.movingaverage.averagers; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Collections; import java.util.Comparator; import java.util.List; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - /** * Implementation of AveragerFacvtory created solely for incremental development */ diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java index c9db3f2cd069..2c6ac9cdf5ab 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMaxAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java index 9c708428fff9..1e82f09e9e95 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java index ece43f105992..9b0459df0db8 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMeanAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java index 233b7c521bad..58f544671a96 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java index 5f98c25ea9bd..c3b8adcd519d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMeanNoNullAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java index 9abf4fcce7d1..d6e11893a5e7 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java index 0ded39e82f23..1c6a4cb58d74 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class DoubleMinAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java index d3d81f1be13b..35a783b2235d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java index 343a1c8f9e3a..b2c04d12aecd 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMaxAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java index 473026fd7180..847bbcb9e341 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java index 8c6d97d53863..8d7892d7c22c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMeanAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java index 2c413b976a75..d02e06d96173 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java index 5cf6548e3605..0e59af32d628 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMeanNoNullAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java index 5761e61ae4d1..03ad7d1e654c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java index 0a275988395d..fedf055ce91e 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; public class LongMinAverager extends BaseAverager diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java index 2d9f342414ea..ff2562541172 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactory.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java index 0756204fa5e6..12ce0bca3d9a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.yahoo.sketches.Family; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java index 5a67de968365..5e687ba3a2e7 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java @@ -16,15 +16,16 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.averagers; -import java.util.Comparator; +package org.apache.druid.query.movingaverage.averagers; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.yahoo.sketches.Util; import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; +import java.util.Comparator; + public class SketchUnionAveragerFactory extends BaseAveragerFactory { diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java index bc052b3b9854..dac2d693a13a 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java @@ -16,15 +16,15 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; - -import org.junit.Test; -import java.util.Arrays; +package org.apache.druid.query.movingaverage; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; +import org.junit.Test; + +import java.util.Arrays; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 27c6a3148abb..a1c7e9acc0f5 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -16,24 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage; -import static org.hamcrest.CoreMatchers.anyOf; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.not; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -import org.apache.druid.query.movingaverage.averagers.AveragerFactory; -import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; -import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; - -import org.joda.time.DateTime; -import org.joda.time.chrono.ISOChronology; -import org.junit.Test; +package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -46,6 +30,12 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.movingaverage.averagers.AveragerFactory; +import org.apache.druid.query.movingaverage.averagers.ConstantAveragerFactory; +import org.apache.druid.query.movingaverage.averagers.LongMeanAveragerFactory; +import org.joda.time.DateTime; +import org.joda.time.chrono.ISOChronology; +import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; @@ -56,6 +46,15 @@ import java.util.List; import java.util.Map; +import static org.hamcrest.CoreMatchers.anyOf; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + /** * */ diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 491af896545c..ff5746449f99 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -16,23 +16,43 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.inject.Injector; +import com.google.inject.Module; +import com.google.inject.name.Names; import com.google.inject.util.Providers; +import mockit.Mock; +import mockit.MockUp; +import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.client.cache.MapCache; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryRunnerFactoryModule; +import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.guava.Accumulators; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; @@ -41,6 +61,12 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.RetryQueryRunnerConfig; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.movingaverage.test.TestConfig; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.timeline.TimelineLookup; @@ -62,33 +88,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; -import com.fasterxml.jackson.core.JsonParseException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.node.ArrayNode; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import org.apache.druid.query.movingaverage.test.TestConfig; -import org.apache.druid.client.CachingClusteredClient; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.initialization.Initialization; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; -import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.timeseries.TimeseriesQuery; -import org.apache.druid.query.timeseries.TimeseriesResultValue; - -import mockit.Mock; -import mockit.MockUp; - -import org.apache.druid.java.util.emitter.service.ServiceEmitter; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java index 4073302c7b0a..d961bc47059e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java @@ -16,9 +16,18 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.movingaverage.averagers.DoubleMeanAveragerFactory; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.chrono.ISOChronology; @@ -30,15 +39,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.druid.data.input.MapBasedRow; -import org.apache.druid.data.input.Row; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; -import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; -import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 6746a1e31e74..6633d0068a85 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -16,16 +16,13 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.Period; @@ -41,6 +38,9 @@ import java.util.List; import java.util.Map; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class RowBucketIterableTest { diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java index 8b16d29dcc44..9956a7065f3d 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Before; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java index 302cb4c0b25e..c6e960d84b89 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java index acea22bc09f5..773cae4843ba 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java index 989c38ab7ab5..496063355743 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java index df023e58338c..68d9b67024f8 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java index c0a8b39e254b..a0840a8d33f7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java index 929485574885..49c73b34c5ca 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java index 070f44914b91..9359fc2cefee 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java index 46a6efa89deb..6d946e4835c7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java index 88aa4d35d894..ef2bb6f0882c 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java index f5ccab25b981..02fd2c2cec8b 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java index ce90efd439f6..7246e8bd006e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java index 24cc7bbb7349..c799a1ad60b7 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java index e164383f6778..763d9c75321f 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java index 7187c876fb20..c34d1ec2e612 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java index 0dcf9e5c274c..f3c4dac4902e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java index 34b9696eda3d..0681db77ffcd 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java index 48e75371c704..067f6b223a10 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java index 1b0f874db824..4cbcdaed4c0d 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMinAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java index 68cae5d19d54..540dc5ffccd9 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import org.junit.Test; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java index 0ff62f2ae409..b70b7a9539ad 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.druid.query.movingaverage.averagers; import com.yahoo.sketches.Family; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java index 92c5715b954f..792394e2723e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/test/TestConfig.java @@ -16,13 +16,14 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.druid.query.movingaverage.test; -import java.util.Map; +package org.apache.druid.query.movingaverage.test; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.Map; + /** * Configuration for a unit test. */ From 277220681732b53d7e79301a1db13d459906fcec Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 14:06:09 -0800 Subject: [PATCH 17/45] Refresh code with latest master changes: * Remove NullDimensionSelector. * Apply changes of RequestLogger. * Apply changes of TimelineServerView. --- .../DefaultMovingAverageQueryMetricsFactory.java | 4 ++++ .../druid/query/movingaverage/MovingAverageIterable.java | 3 +-- .../query/movingaverage/MovingAverageQueryRunner.java | 8 ++++---- .../druid/query/movingaverage/MovingAverageQueryTest.java | 6 ++++++ .../src/test/resources/runtime.properties | 1 + 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java index 64b7c5b6910b..c9efb8bc25d3 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/DefaultMovingAverageQueryMetricsFactory.java @@ -33,6 +33,10 @@ public class DefaultMovingAverageQueryMetricsFactory implements MovingAverageQue private static final MovingAverageQueryMetricsFactory INSTANCE = new DefaultMovingAverageQueryMetricsFactory(new DefaultObjectMapper()); + /** + * Should be used only in tests, directly or indirectly (via {@link + * MovingAverageQueryToolChest#MovingAverageQueryToolChest}). + */ @VisibleForTesting public static MovingAverageQueryMetricsFactory instance() { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index c01aaa4f5a65..463634a93e98 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -32,7 +32,6 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.NullDimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; import java.util.Collection; @@ -84,7 +83,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) { // Generating empty records while aggregating on Filtered aggregators requires a dimension selector // for initialization. This dimension selector is not actually used for generating values - return NullDimensionSelector.instance(); + return DimensionSelector.constant(null); } @Override diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index d64ba18222d2..0328f4b029f5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -134,10 +134,10 @@ public Sequence run(QueryPlus query, Map responseConte Sequence results = gbq.getRunner(walker).run(QueryPlus.wrap(gbq), gbqResponse); try { // use localhost for remote address - requestLogger.log(new RequestLogLine( + requestLogger.logNativeQuery(RequestLogLine.forNative( + gbq, DateTimes.nowUtc(), "127.0.0.1", - gbq, new QueryStats( ImmutableMap.of( "query/time", 0, @@ -172,10 +172,10 @@ public Sequence run(QueryPlus query, Map responseConte Sequence> results = tsq.getRunner(walker).run(QueryPlus.wrap(tsq), tsqResponse); try { // use localhost for remote address - requestLogger.log(new RequestLogLine( + requestLogger.logNativeQuery(RequestLogLine.forNative( + tsq, DateTimes.nowUtc(), "127.0.0.1", - tsq, new QueryStats( ImmutableMap.of( "query/time", 0, diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index ff5746449f99..baf887063c61 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -379,6 +379,12 @@ public TimelineLookup getTimeline(DataSource dataSource) return null; } + @Override + public List getDruidServers() + { + return null; + } + @Override public QueryRunner getQueryRunner(DruidServer server) { diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index cf60cde85fe5..2877ef660145 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -55,3 +55,4 @@ druid.server.priority=0 druid.server.tier=_default_tier druid.worker.ip=localhost druid.worker.version=0 +druid.request.logging.type=slf4j \ No newline at end of file From f245403de614e525dbb3e3354d555d0f6b39f570 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 15:46:20 -0800 Subject: [PATCH 18/45] Small checkstyle fix. --- .../moving-average-query/src/test/resources/runtime.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index 2877ef660145..1c6b09b91b36 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -55,4 +55,4 @@ druid.server.priority=0 druid.server.tier=_default_tier druid.worker.ip=localhost druid.worker.version=0 -druid.request.logging.type=slf4j \ No newline at end of file +druid.request.logging.type=slf4j From 70c14a346c5a4dc5c318a56c4122b68a7e3d7e5f Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 30 Jan 2019 16:58:52 -0800 Subject: [PATCH 19/45] Checkstyle fixes. --- .../apache/druid/query/movingaverage/MovingAverageQuery.java | 4 ++-- .../query/movingaverage/MovingAverageQueryToolChest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java index 7f12e5e1ab57..38fc1ebcc124 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQuery.java @@ -29,7 +29,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.IAE; @@ -50,6 +49,7 @@ import org.apache.druid.query.spec.QuerySegmentSpec; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -164,7 +164,7 @@ private static void verifyOutputNames( ) { - final Set outputNames = Sets.newHashSet(); + final Set outputNames = new HashSet<>(); for (DimensionSpec dimension : dimensions) { if (!outputNames.add(dimension.getOutputName())) { throw new IAE("Duplicate output name[%s]", dimension.getOutputName()); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index 1bc9a8a37273..ae293f440a8b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Maps; import com.google.inject.Inject; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -37,6 +36,7 @@ import org.apache.druid.server.log.RequestLogger; import javax.annotation.Nullable; +import java.util.HashMap; import java.util.Map; /** @@ -100,7 +100,7 @@ public Function makePostComputeManipulatorFn(MovingAverageQuery query, public Row apply(Row result) { MapBasedRow mRow = (MapBasedRow) result; - final Map values = Maps.newHashMap(mRow.getEvent()); + final Map values = new HashMap(mRow.getEvent()); for (AggregatorFactory agg : query.getAggregatorSpecs()) { Object aggVal = values.get(agg.getName()); From 3e349f7ee4c73e595531477acaf1e5c89e978669 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Thu, 31 Jan 2019 00:15:16 -0800 Subject: [PATCH 20/45] Fixing rat errors; Teamcity errors. --- .../moving-average-query/README.md | 19 +++++++++++++++++++ .../movingaverage/MovingAverageIterable.java | 8 ++++---- ...rg.apache.druid.initialization.DruidModule | 15 +++++++++++++++ .../movingaverage/MovingAverageQueryTest.java | 6 +++--- 4 files changed, 41 insertions(+), 7 deletions(-) diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md index b92a6dbce303..502f1e68304b 100644 --- a/extensions-contrib/moving-average-query/README.md +++ b/extensions-contrib/moving-average-query/README.md @@ -1,3 +1,22 @@ + + druid-moving-average-query ============= diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 463634a93e98..936a9525f84b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -104,7 +104,7 @@ public ColumnCapabilities getColumnCapabilities(String s) Aggregator aggFactorized = agg.factorize(colFact); fakeEvents.put(agg.getName(), aggFactorized.get()); }); - postAggMap.values().stream().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); + postAggMap.values().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); } /* (non-Javadoc) @@ -280,12 +280,12 @@ private Row computeMovingAverage(Map key, Row r, boolean skip) } if (!skip) { - avg.stream().forEach(af -> af.addElement(event, aggMap)); + avg.forEach(af -> af.addElement(event, aggMap)); } else { - avg.stream().forEach(af -> af.skip()); + avg.forEach(af -> af.skip()); } - avg.stream().forEach(af -> result.put(af.getName(), af.getResult())); + avg.forEach(af -> result.put(af.getName(), af.getResult())); // At least one non-dimension value must be in the record for it to be valid. if (result.entrySet().stream().anyMatch(e -> !key.containsKey(e.getKey()) && e.getValue() != null)) { diff --git a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule index 72876559838d..ec70e7d9c464 100644 --- a/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule +++ b/extensions-contrib/moving-average-query/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -1 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF 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. + org.apache.druid.query.movingaverage.MovingAverageQueryModule diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index baf887063c61..d2ee24547160 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -123,7 +123,7 @@ public static Iterable data() throws IOException return tests; } - public MovingAverageQueryTest(String yamlFile) throws JsonParseException, JsonMappingException, IOException + public MovingAverageQueryTest(String yamlFile) throws IOException { List modules = getRequiredModules(); modules.add( @@ -238,7 +238,7 @@ protected List getRequiredModules() * @throws JsonMappingException * @throws JsonParseException */ - protected void defineMocks(Query query) throws JsonParseException, JsonMappingException, IOException + protected void defineMocks(Query query) throws IOException { groupByResults.clear(); timeseriesResults.clear(); @@ -320,7 +320,7 @@ protected void consistentTypeCasting(List result) */ @SuppressWarnings({"unchecked", "rawtypes"}) @Test - public void testQuery() throws JsonParseException, JsonMappingException, IOException + public void testQuery() throws IOException { From 516d40bdb6071f0cb12fbf4c0837b2bbe9ef9dff Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 25 Feb 2019 12:00:20 -0800 Subject: [PATCH 21/45] Removing support theta sketches. Will be added back in this pr or a following once DI conflicts with datasketches are resolved. --- .../moving-average-query.md | 20 ----- .../moving-average-query/pom.xml | 6 -- .../averagers/AveragerFactory.java | 3 +- .../averagers/SketchUnionAverager.java | 57 ------------- .../averagers/SketchUnionAveragerFactory.java | 81 ------------------ .../movingaverage/MovingAverageQueryTest.java | 2 - .../SketchUnionAveragerFactoryTest.java | 37 --------- .../averagers/SketchUnionAveragerTest.java | 82 ------------------- .../queryTests/missingGroupByValues.yaml | 17 ++-- .../queryTests/movingAverageWithSketches.yaml | 70 ---------------- 10 files changed, 9 insertions(+), 366 deletions(-) delete mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java delete mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java delete mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java delete mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java delete mode 100644 extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index 804b4fdb6866..e8b4294154fb 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -106,8 +106,6 @@ These are properties which are common to all Averagers: * longMeanNoNulls * longMax * longMin -* [Sketch averagers](#sketch-averagers): - * sketchUnion #### Standard averagers @@ -127,24 +125,6 @@ Example of usage: { "type" : "doubleMean", "name" : , "fieldName": } ``` -#### Sketch averagers - -Sketch averager are meant to perform merge operations on [DataSketches](../extensions-core/datasketches-extension.html) (When using a Sketch averager, please include the [DataSketches](../extensions-core/datasketches-extension.html) extension as well). - -Extra properties for Sketch averagers: - -|property|description|required?| -|--------|-----------|---------| -|size|Sketch size; See [DataSketches aggregator](../extensions-core/datasketches-aggregators.html)|no, defaults to 4096| - -Available functions: -* sketchUnion - -Example of usage: -```json -{ "type" : "sketchUnion", "name" : , "fieldName": } -``` - ### Cycle size (Day of Week) This optional parameter is used to calculate over a single bucket within each cycle instead of all buckets. A prime example would be weekly buckets, resulting in a Day of Week calculation. (Other examples: Month of year, Hour of day). diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index a7e76d1bca97..416e34c6174b 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -73,12 +73,6 @@ ${project.parent.version} provided - - org.apache.druid.extensions - druid-datasketches - ${project.parent.version} - provided - org.apache.druid druid-server diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java index e2041b0db40a..a3c82781b233 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/AveragerFactory.java @@ -41,8 +41,7 @@ @JsonSubTypes.Type(name = "longMean", value = LongMeanAveragerFactory.class), @JsonSubTypes.Type(name = "longMeanNoNulls", value = LongMeanNoNullAveragerFactory.class), @JsonSubTypes.Type(name = "longMax", value = LongMaxAveragerFactory.class), - @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class), - @JsonSubTypes.Type(name = "sketchUnion", value = SketchUnionAveragerFactory.class) + @JsonSubTypes.Type(name = "longMin", value = LongMinAveragerFactory.class) }) public interface AveragerFactory { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java deleted file mode 100644 index 12ce0bca3d9a..000000000000 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAverager.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.yahoo.sketches.Family; -import com.yahoo.sketches.theta.SetOperation; -import com.yahoo.sketches.theta.SetOperationBuilder; -import com.yahoo.sketches.theta.Union; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; - -public class SketchUnionAverager extends BaseAverager -{ - - private int startFrom = 0; - private final SetOperationBuilder builder; - - public SketchUnionAverager(int numBuckets, String name, String fieldName, int cycleSize, int size) - { - super(SketchHolder.class, numBuckets, name, fieldName, cycleSize, false); - this.builder = SetOperation.builder().setNominalEntries(size); - } - - @Override - protected SketchHolder computeResult() - { - int cycleSize = getCycleSize(); - Object[] obj = getBuckets(); - int numBuckets = getNumBuckets(); - - Union union = (Union) builder.build(Family.UNION); - for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - ((SketchHolder) obj[(i + startFrom) % numBuckets]).updateUnion(union); - } - } - - startFrom++; - return SketchHolder.of(union.getResult()); - } -} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java deleted file mode 100644 index 5e687ba3a2e7..000000000000 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.yahoo.sketches.Util; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; - -import java.util.Comparator; - -public class SketchUnionAveragerFactory extends BaseAveragerFactory -{ - - private final int size; - - @JsonCreator - public SketchUnionAveragerFactory( - @JsonProperty("name") String name, - @JsonProperty("buckets") int numBuckets, - @JsonProperty("cycleSize") Integer cycleSize, - @JsonProperty("fieldName") String fieldName, - @JsonProperty(value = "size", required = false) Integer size - ) - { - super(name, numBuckets, fieldName, cycleSize); - this.size = size == null ? Util.DEFAULT_NOMINAL_ENTRIES : size; - } - - public SketchUnionAveragerFactory( - String name, - int numBuckets, - Integer cycleSize, - String fieldName - ) - { - this(name, numBuckets, cycleSize, fieldName, null); - } - - @Override - public Averager createAverager() - { - return new SketchUnionAverager(numBuckets, name, fieldName, cycleSize, size); - } - - /* (non-Javadoc) - * @see AveragerFactory#finalize(java.lang.Object) - */ - @Override - public Double finalizeComputation(SketchHolder val) - { - return val.getSketch().getEstimate(); - } - - /* (non-Javadoc) - * @see BaseAveragerFactory#getComparator() - */ - @SuppressWarnings({"rawtypes", "unchecked"}) - @Override - public Comparator getComparator() - { - return SketchHolder.COMPARATOR; - } -} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index d2ee24547160..9ee1d28ea693 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -62,7 +62,6 @@ import org.apache.druid.query.Result; import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.datasketches.theta.SketchModule; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.movingaverage.test.TestConfig; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -221,7 +220,6 @@ protected List getRequiredModules() { List list = new ArrayList<>(); - list.add(new SketchModule()); list.add(new QueryRunnerFactoryModule()); list.add(new QueryableModule()); list.add(new DruidProcessingModule()); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java deleted file mode 100644 index 540dc5ffccd9..000000000000 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerFactoryTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import org.junit.Test; - -import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.junit.Assert.assertThat; - -public class SketchUnionAveragerFactoryTest -{ - - @Test - public void testCreateAverager() - { - AveragerFactory fac = new SketchUnionAveragerFactory("test", 5, 1, "field"); - assertThat(fac.createAverager(), instanceOf(SketchUnionAverager.class)); - } - -} diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java deleted file mode 100644 index b70b7a9539ad..000000000000 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/SketchUnionAveragerTest.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage.averagers; - -import com.yahoo.sketches.Family; -import com.yahoo.sketches.Util; -import com.yahoo.sketches.theta.SetOperation; -import com.yahoo.sketches.theta.Union; -import com.yahoo.sketches.theta.UpdateSketch; -import org.apache.druid.query.aggregation.datasketches.theta.SketchHolder; -import org.junit.Test; - -import java.util.Collections; -import java.util.HashMap; - -import static org.junit.Assert.assertEquals; - -public class SketchUnionAveragerTest -{ - - @Test - public void testComputeResult() throws Exception - { - BaseAverager avg = - new SketchUnionAverager(2, "test", "field", 1, Util.DEFAULT_NOMINAL_ENTRIES); - - UpdateSketch sketch1 = UpdateSketch.builder().build(); - for (int key = 0; key < 16; key++) { - sketch1.update(key); - } - - UpdateSketch sketch2 = UpdateSketch.builder().build(); - for (int key = 16; key < 32; key++) { - sketch2.update(key); - } - - UpdateSketch sketch3 = UpdateSketch.builder().build(); - for (int key = 32; key < 48; key++) { - sketch2.update(key); - } - - SketchHolder holder1 = SketchHolder.of(sketch1); - SketchHolder holder2 = SketchHolder.of(sketch2); - SketchHolder holder3 = SketchHolder.of(sketch3); - - Union union1 = (Union) SetOperation.builder().build(Family.UNION); - union1.update(sketch1); - union1.update(sketch2); - - avg.addElement(Collections.singletonMap("field", holder1), new HashMap<>()); - avg.addElement(Collections.singletonMap("field", holder2), new HashMap<>()); - - assertEquals(avg.computeResult().getEstimate(), union1.getResult().getEstimate(), 0); - - avg.addElement(Collections.singletonMap("field", holder3), new HashMap<>()); - - Union union2 = (Union) SetOperation.builder().build(Family.UNION); - union2.update(sketch2); - union2.update(sketch3); - - assertEquals(avg.computeResult().getEstimate(), union2.getResult().getEstimate(), 0); - - } - -} diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml index 626b94c23c7f..c4ab5a4fa8a3 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml +++ b/extensions-contrib/moving-average-query/src/test/resources/queryTests/missingGroupByValues.yaml @@ -21,10 +21,9 @@ query: - name: timespent_secs fieldName: timespent type: longSum - - name: regBcookies - fieldName: bcookies_with_yuid_sketch - size: 16384 - type: thetaSketch + - name: someSum + fieldName: someSum_field + type: doubleSum postAggregations: - type: arithmetic name: timeSpent @@ -45,7 +44,7 @@ expectedOutput: timespent_secs: 240 timeSpent: 4.0 trailing7DayAvgTimeSpent: 3.0 - regBcookies: 0.0 + someSum: 3.0 - version: v1 timestamp: 2017-01-02T00:00Z event: @@ -53,7 +52,7 @@ expectedOutput: timespent_secs: 0 timeSpent: 0.0 trailing7DayAvgTimeSpent: 1.0 - regBcookies: 0.0 + someSum: 0.0 intermediateResults: groupBy: - version: v1 @@ -62,18 +61,18 @@ intermediateResults: gender: m timespent_secs: 120 timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 5.0 - version: v1 timestamp: 2017-01-01T00:00Z event: gender: f timespent_secs: 120 timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 2.0 - version: v1 timestamp: 2017-01-02T00:00Z event: gender: m timespent_secs: 240 timeSpent: 4.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ + someSum: 3.0 diff --git a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml b/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml deleted file mode 100644 index 460de3f4cfd5..000000000000 --- a/extensions-contrib/moving-average-query/src/test/resources/queryTests/movingAverageWithSketches.yaml +++ /dev/null @@ -1,70 +0,0 @@ -query: - queryType: movingAverage - dataSource: - type: table - name: slice_pr_pc_pl_us_os - context: { - } - granularity: - type: period - period: P1D - intervals: - - 2017-01-01T00:00Z/2017-01-02T00:00Z - dimensions: - - gender - averagers: - - name: regBcookiesAvg - fieldName: regBcookies - size: 16384 - buckets: 2 - type: sketchUnion - aggregations: - - name: regBcookies - fieldName: bcookie_with_yuid_sketch - size: 16384 - type: thetaSketch - postAveragers: [ - ] - postAggregations: [ - ] -expectedOutput: -- version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: 0.0 - regBcookiesAvg: 0.0 -- version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: f - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: 0.0 - regBcookiesAvg: 0.0 -intermediateResults: - groupBy: - - version: v1 - timestamp: 2016-12-31T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - - version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: m - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - - version: v1 - timestamp: 2017-01-01T00:00Z - event: - gender: f - timespent_secs: 120 - timeSpent: 2.0 - regBcookies: AwEDAAAAAgAAAAAAAAAAAP////////9/ - From 208af0dcb3e026e473d21eb4c81346563966fda1 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 25 Feb 2019 23:11:32 -0800 Subject: [PATCH 22/45] Implements some of the review fixes. --- .../moving-average-query.md | 24 ++++---- .../moving-average-query/README.md | 6 -- .../moving-average-query/pom.xml | 6 +- .../movingaverage/AveragerFactoryWrapper.java | 48 ++++++++------- .../movingaverage/MovingAverageIterable.java | 2 +- .../MovingAverageQueryRunner.java | 2 +- .../MovingAverageQueryToolChest.java | 6 +- ... => PostAveragerAggregatorCalculator.java} | 4 +- .../movingaverage/RowBucketIterable.java | 15 ++--- .../movingaverage/averagers/BaseAverager.java | 2 +- .../MovingAverageIterableTest.java | 1 - .../movingaverage/MovingAverageQueryTest.java | 3 +- ...PostAveragerAggregatorCalculatorTest.java} | 6 +- .../movingaverage/RowBucketIterableTest.java | 2 - .../averagers/BaseAveragerFactoryTest.java | 2 +- .../averagers/DoubleMaxAveragerTest.java | 2 +- .../averagers/DoubleMeanAveragerTest.java | 2 +- .../DoubleMeanAveragerWithPeriodTest.java | 2 +- .../averagers/LongMaxAveragerFactoryTest.java | 2 +- .../averagers/LongMeanAveragerTest.java | 2 +- .../src/test/resources/runtime.properties | 58 +------------------ 21 files changed, 65 insertions(+), 132 deletions(-) rename extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/{PostAveragerCalculator.java => PostAveragerAggregatorCalculator.java} (92%) rename extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/{PostAveragerCalculatorTest.java => PostAveragerAggregatorCalculatorTest.java} (95%) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index e8b4294154fb..bf5a0da48412 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -51,34 +51,34 @@ It runs the query in two main phases: ## Operations To use this extension, make sure to [load](../../operations/including-extensions.html) `druid-moving-average-query` only to the Broker. -##Configuration +## Configuration There are currently no configuration properties specific to Moving Average. -##Limitations +## Limitations * movingAverage is missing support for the following groupBy properties: `subtotalsSpec`, `virtualColumns`. * movingAverage is missing support for the following timeseries properties: `descending`. * movingAverage is missing support for [SQL-compatible null handling](https://github.com/apache/incubator-druid/issues/4349) (So setting druid.generic.useDefaultValueForNull in configuration will give an error). ##Query spec: -* Most properties in the For the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. +* Most properties in the query spec derived from [groupBy query](../../querying/groupbyquery.html) / [timeseries](../../querying/timeseriesquery.html), see documentation for these query types. |property|description|required?| |--------|-----------|---------| -|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query|yes| -|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information |yes| -|dimensions|A JSON list of dimensions to do the groupBy over; can be ommitted for using timeseries back-engine; or see [DimensionSpec](../../querying/dimensionspecs.html) for ways to extract dimensions |no| -|limitSpec|See [LimitSpec](../../querying/limitspec.html).|no| -|having|See [Having](../../querying/having.html).|no| -|granularity|Granularity type must be:`period`; See [Granularities](../../querying/granularities.html)|yes| +|queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query.|yes| +|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information.|yes| +|dimensions|A JSON list of [DimensionSpec](../../querying/dimensionspecs.html)|no| +|limitSpec|See [LimitSpec](../../querying/limitspec.html)|no| +|having|See [Having](../../querying/having.html)|no| +|granularity|A period granilarity; See [Period Granularities](../../querying/granularities.html#period-granularities)|yes| |filter|See [Filters](../../querying/filters.html)|no| |aggregations|Aggregations forms the input to Averagers; See [Aggregations](../../querying/aggregations.html)|yes| |postAggregations|Supports only aggregations as input; See [Post Aggregations](../../querying/post-aggregations.html)|no| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| |context|An additional JSON Object which can be used to specify certain flags.|no| -|averagers|Defines the moving average function; See [Aggregations](../../querying/aggregations.html)|yes| +|averagers|Defines the moving average function; See [Averagers](#averagers)|yes| |postAveragers|Support input of both averagers and aggregations; Syntax is identical to postAggregations (See [Post Aggregations](../../querying/post-aggregations.html))|no| -##Averagers +## Averagers Averagers are used to define the Moving-Average function. Averagers are not limited to an average - they can also provide other types of window functions such as MAX()/MIN(). @@ -95,7 +95,7 @@ These are properties which are common to all Averagers: |cycleSize|Cycle size; Used to calculate day-of-week option; See [Cycle size (Day of Week)](#cycle-size-day-of-week)|no, defaults to 1| -###Averager types: +### Averager types: * [Standard averagers](#standard-averagers): * doubleMean diff --git a/extensions-contrib/moving-average-query/README.md b/extensions-contrib/moving-average-query/README.md index 502f1e68304b..33156e74a055 100644 --- a/extensions-contrib/moving-average-query/README.md +++ b/extensions-contrib/moving-average-query/README.md @@ -27,9 +27,3 @@ Overview Documentation ============= See the druid.io website or under [Druid Github Repo](https://github.com/apache/incubator-druid/tree/master/docs/content/development/extensions-contrib/moving-average-query.md). - -Thanks -=========== - -This module was created by [Will Lauer](https://github.com/will-lauer) and [Mangesh Pardeshi](https://github.com/mangesh-pardeshi), with additional contributions by: [Deepak Babu](https://github.com/deepakb91), [Asif Mansoor](https://github.com/asifmansoora), [Ke Wang](https://github.com/kenuiuc), [Eyal Yurman](https://github.com/yurmix). -Thanks to all contributors! diff --git a/extensions-contrib/moving-average-query/pom.xml b/extensions-contrib/moving-average-query/pom.xml index 416e34c6174b..ae6f68a3b881 100644 --- a/extensions-contrib/moving-average-query/pom.xml +++ b/extensions-contrib/moving-average-query/pom.xml @@ -25,7 +25,7 @@ org.apache.druid druid - 0.13.0-incubating-SNAPSHOT + 0.15.0-incubating-SNAPSHOT ../../pom.xml @@ -76,8 +76,8 @@ org.apache.druid druid-server - 0.13.0-incubating-SNAPSHOT - compile + ${project.parent.version} + provided diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 198f898d9f18..f6f1d90b4f65 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -33,6 +33,10 @@ * This is necessary purely to allow existing common druid code that only knows * about aggregators to work with the MovingAverageQuery query as well. * + * NOTE: The {@link AggregatorFactory} abstract class is only partially extended. + * Most methods are not implemented and throw {@link UnsupportedOperationException} if called. + * This is becsuse these methods are invalid for the AveragerFactoryWrapper. + * * @param Result type * @param Finalized Result type */ @@ -55,21 +59,21 @@ public AveragerFactoryWrapper(AveragerFactory af, String prefix) } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) + public Aggregator factorize(ColumnSelectorFactory metricFactory) throws UnsupportedOperationException { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /* (non-Javadoc) @@ -82,43 +86,43 @@ public Comparator getComparator() } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public Object combine(Object lhs, Object rhs) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public AggregatorFactory getCombiningFactory() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public List getRequiredColumns() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public Object deserialize(Object object) { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @SuppressWarnings("unchecked") @Override @@ -137,39 +141,39 @@ public String getName() } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public List requiredFields() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public byte[] getCacheKey() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public String getTypeName() { - return null; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } /** - * Returns null because Averagers aren't actually Aggregators + * Not implemented. Throws UnsupportedOperationException. */ @Override public int getMaxIntermediateSize() { - return 0; + throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 936a9525f84b..be919dd4dd44 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -195,7 +195,7 @@ private Row internalNext() cacheIter = cache.getRows().iterator(); } - Row r = null; + Row r; // return rows from the cached RowBucket if (cacheIter != null) { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 0328f4b029f5..64a503cfd97d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -207,7 +207,7 @@ public Sequence run(QueryPlus query, Map responseConte // Apply any postAveragers Sequence movingAvgResultsWithPostAveragers = - Sequences.map(movingAvgResults, new PostAveragerCalculator(maq)); + Sequences.map(movingAvgResults, new PostAveragerAggregatorCalculator(maq)); // remove rows outside the reporting window List reportingIntervals = maq.getIntervals(); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index ae293f440a8b..9164c30a039b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -49,8 +49,6 @@ public class MovingAverageQueryToolChest extends QueryToolChest makePostComputeManipulatorFn(MovingAverageQuery query, public Row apply(Row result) { MapBasedRow mRow = (MapBasedRow) result; - final Map values = new HashMap(mRow.getEvent()); + final Map values = new HashMap<>(mRow.getEvent()); for (AggregatorFactory agg : query.getAggregatorSpecs()) { Object aggVal = values.get(agg.getName()); @@ -114,7 +112,7 @@ public Row apply(Row result) for (AveragerFactory avg : query.getAveragerSpecs()) { Object aggVal = values.get(avg.getName()); if (aggVal != null) { - values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper(avg, avg.getName() + "_"), aggVal)); + values.put(avg.getName(), fn.manipulate(new AveragerFactoryWrapper<>(avg, avg.getName() + "_"), aggVal)); } else { values.put(avg.getName(), null); } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java similarity index 92% rename from extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java rename to extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java index 3490f3e4192c..5031330badaa 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java @@ -30,12 +30,12 @@ /** * Function that can be applied to a Sequence to calculate PostAverager results */ -public class PostAveragerCalculator implements Function +public class PostAveragerAggregatorCalculator implements Function { private final List postAveragers; - public PostAveragerCalculator(MovingAverageQuery maq) + public PostAveragerAggregatorCalculator(MovingAverageQuery maq) { this.postAveragers = maq.getPostAveragerSpecs(); } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 1e1fda717215..688eb1943302 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -55,13 +55,12 @@ public RowBucketIterable(Sequence seq, List intervals, Period per @Override public Iterator iterator() { - return new RowIterator(seq, intervals, period); + return new RowBucketIterator(seq, intervals, period); } - static class RowIterator implements Iterator + static class RowBucketIterator implements Iterator { private Yielder yielder; - private boolean done = false; private DateTime endTime; private DateTime expectedBucket; private Period period; @@ -70,7 +69,7 @@ static class RowIterator implements Iterator private boolean processedLastRow = false; private boolean processedExtraRow = false; - public RowIterator(Sequence rows, List intervals, Period period) + public RowBucketIterator(Sequence rows, List intervals, Period period) { this.period = period; this.intervals = intervals; @@ -85,11 +84,7 @@ public RowIterator(Sequence rows, List intervals, Period period) @Override public boolean hasNext() { - // expectedBucket < endTime - if (expectedBucket.compareTo(endTime) < 0) { - return true; - } - return false; + return expectedBucket.compareTo(endTime) < 0; } /* (non-Javadoc) @@ -102,7 +97,7 @@ public RowBucket next() if (expectedBucket.compareTo(intervals.get(intervalIndex).getEnd()) >= 0) { intervalIndex++; - if (intervalIndex <= intervals.size()) { + if (intervalIndex < intervals.size()) { expectedBucket = intervals.get(intervalIndex).getStart(); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index 37497c695ead..00fee22216ea 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -176,7 +176,7 @@ public int getCycleSize() */ protected I[] getBuckets() { - return (I[]) buckets; + return buckets; } /** diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index a1c7e9acc0f5..39c13604e373 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -66,7 +66,6 @@ public class MovingAverageIterableTest private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final String GENDER = "gender"; private static final String AGE = "age"; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 9ee1d28ea693..1169653dd1b9 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -82,6 +82,7 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -429,7 +430,7 @@ public void emit(Event event) {} }, baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheConfig() ); - final Map responseContext = new ConcurrentHashMap<>(); + final Map responseContext = new HashMap<>(); defineMocks(query); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java similarity index 95% rename from extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java rename to extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java index d961bc47059e..51c707749f18 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerCalculatorTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculatorTest.java @@ -46,11 +46,11 @@ /** * Unit tests for PostAveragerCalcualtor */ -public class PostAveragerCalculatorTest +public class PostAveragerAggregatorCalculatorTest { private MovingAverageQuery query; - private PostAveragerCalculator pac; + private PostAveragerAggregatorCalculator pac; private Map event; private MapBasedRow row; @@ -80,7 +80,7 @@ public void setup() null ); - pac = new PostAveragerCalculator(query); + pac = new PostAveragerAggregatorCalculator(query); event = new HashMap<>(); row = new MapBasedRow(new DateTime(ISOChronology.getInstanceUTC()), event); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 6633d0068a85..0dcfe1c6e4d1 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -50,8 +50,6 @@ public class RowBucketIterableTest private static final DateTime JAN_4 = new DateTime(2017, 1, 4, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_5 = new DateTime(2017, 1, 5, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_6 = new DateTime(2017, 1, 6, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_7 = new DateTime(2017, 1, 7, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); - private static final DateTime JAN_8 = new DateTime(2017, 1, 8, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final DateTime JAN_9 = new DateTime(2017, 1, 9, 0, 0, 0, 0, ISOChronology.getInstanceUTC()); private static final Map EVENT_M_10 = new HashMap<>(); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java index 9956a7065f3d..98104e675b04 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactoryTest.java @@ -52,7 +52,7 @@ public Comparator getComparator() } @Test - public void testGetDependentFields() throws Exception + public void testGetDependentFields() { List dependentFields = fac.getDependentFields(); assertEquals(1, dependentFields.size()); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java index 496063355743..e1ba10fad193 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAveragerTest.java @@ -30,7 +30,7 @@ public class DoubleMaxAveragerTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager avg = new DoubleMaxAverager(3, "test", "field", 1); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java index a0840a8d33f7..0d5f2c7cc8f5 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerTest.java @@ -30,7 +30,7 @@ public class DoubleMeanAveragerTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager avg = new DoubleMeanAverager(3, "test", "field", 1); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java index 49c73b34c5ca..8cde307c88bf 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAveragerWithPeriodTest.java @@ -30,7 +30,7 @@ public class DoubleMeanAveragerWithPeriodTest { @Test - public void testComputeResult() throws Exception + public void testComputeResult() { BaseAverager averager = new DoubleMeanAverager(14, "test", "field", 7); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java index 7246e8bd006e..7601a5d49156 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMaxAveragerFactoryTest.java @@ -28,7 +28,7 @@ public class LongMaxAveragerFactoryTest { @Test - public void testCreateAverager() throws Exception + public void testCreateAverager() { AveragerFactory fac = new LongMaxAveragerFactory("test", 5, 1, "field"); assertThat(fac.createAverager(), instanceOf(LongMaxAverager.class)); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java index c34d1ec2e612..cb037a233148 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/averagers/LongMeanAveragerTest.java @@ -42,7 +42,7 @@ public void testComputeResult() avg.addElement(Collections.singletonMap("field", 3L), new HashMap<>()); assertEquals(2.0, avg.computeResult(), 0.0); - avg.addElement(Collections.singletonMap("field", new Integer(3)), new HashMap<>()); + avg.addElement(Collections.singletonMap("field", 3), new HashMap<>()); assertEquals(3.0, avg.computeResult(), 0.0); avg.addElement(Collections.singletonMap("field", 2L), new HashMap<>()); diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index 1c6b09b91b36..1e395f3164e0 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -1,58 +1,2 @@ -druid.broker.cache.unCacheable=["groupBy","segmentMetadata"] -druid.broker.http.numConnections=50 -druid.server.http.numThreads=1 -druid.announcer.maxBytesPerNode=524288 -druid.announcer.segmentsPerNode=50 -druid.announcer.type=batch -druid.broker.balancer.type=random -druid.broker.cache.initialSize=500000 -druid.broker.cache.logEvictionCount=0 -druid.broker.cache.sizeInBytes=0 -druid.broker.cache.type=local -druid.broker.http.readTimeOut=PT15M -druid.broker.select.tier=highestPriority -druid.broker.select.tier.custom.priorities= -druid.coordinator.conversion.on=false -druid.coordinator.load.timeout=PT15M -druid.coordinator.merge.on=false -druid.coordinator.period=PT30S -druid.coordinator.period.indexingPeriod=PT1800S -druid.coordinator.startDelay=PT60S -druid.curator.compress=false -druid.emitter.logging.loggerClass=LoggingEmitter -druid.emitter.logging.logLevel=info -druid.extensions.coordinates=[] -druid.manager.config.pollDuration=PT1M -druid.manager.rules.defaultTier=tier1 -druid.manager.rules.pollDuration=PT1M -druid.manager.segment.pollDuration=PT1M -druid.monitoring.emissionPeriod=PT1M -druid.monitoring.monitors=[\"com.metamx.metrics.JvmMonitor\"] druid.processing.buffer.sizeBytes=655360 -druid.processing.numThreads=1 -druid.processing.columnCache.sizeBytes=0 -druid.processing.formatString='processing-%s' -druid.publish.type=metadata -druid.query.chunkPeriod=P1M -druid.query.groupBy.defaultStrategy=v1 -druid.query.groupBy.maxIntermediateRows=50000 -druid.query.groupBy.maxResults=5000000 -druid.query.groupBy.singleThreaded=false -druid.query.search.maxSearchLimit=1000 -druid.request.logging.feed= -druid.request.logging.dir= -druid.request.logging.type=noop -druid.segmentCache.announceIntervalMillis=5000 -druid.segmentCache.deleteOnRemove=true -druid.segmentCache.dropSegmentDelayMillis=30000 -druid.segmentCache.numLoadingThreads=1 -druid.selectors.indexing.serviceName=overlord -druid.server.http.maxIdleTime=PT5m -druid.storage.type=hdfs -druid.worker.capacity=30 -druid.server.maxSize=1073741824 -druid.server.priority=0 -druid.server.tier=_default_tier -druid.worker.ip=localhost -druid.worker.version=0 -druid.request.logging.type=slf4j +druid.request.logging.type=slf4j \ No newline at end of file From 239bde97e34a9a2cbb82d4a36aec544146691782 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Tue, 26 Feb 2019 17:11:49 -0800 Subject: [PATCH 23/45] More fixes for review. --- .../moving-average-query.md | 2 +- .../movingaverage/MovingAverageIterable.java | 25 ++++++++++--------- .../MovingAverageIterableTest.java | 2 +- .../movingaverage/MovingAverageQueryTest.java | 1 - .../movingaverage/RowBucketIterableTest.java | 16 ++++++------ .../src/test/resources/runtime.properties | 2 +- 6 files changed, 24 insertions(+), 24 deletions(-) diff --git a/docs/content/development/extensions-contrib/moving-average-query.md b/docs/content/development/extensions-contrib/moving-average-query.md index bf5a0da48412..5fc72688236b 100644 --- a/docs/content/development/extensions-contrib/moving-average-query.md +++ b/docs/content/development/extensions-contrib/moving-average-query.md @@ -66,7 +66,7 @@ There are currently no configuration properties specific to Moving Average. |--------|-----------|---------| |queryType|This String should always be "movingAverage"; this is the first thing Druid looks at to figure out how to interpret the query.|yes| |dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../../querying/datasource.html) for more information.|yes| -|dimensions|A JSON list of [DimensionSpec](../../querying/dimensionspecs.html)|no| +|dimensions|A JSON list of [DimensionSpec](../../querying/dimensionspecs.html) (Notice that property is optional)|no| |limitSpec|See [LimitSpec](../../querying/limitspec.html)|no| |having|See [Having](../../querying/having.html)|no| |granularity|A period granilarity; See [Period Granularities](../../querying/granularities.html#period-granularities)|yes| diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index be919dd4dd44..9d755b35fd3d 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import java.util.Collection; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -55,16 +55,16 @@ public class MovingAverageIterable implements Iterable { private final Sequence seq; - private final Collection dims; - private final Collection> factories; + private final List dims; + private final List> factories; private final Map postAggMap; private final Map aggMap; private final Map fakeEvents; public MovingAverageIterable( Sequence buckets, - Collection dims, - Collection> factories, + List dims, + List> factories, List postAggList, List aggList ) @@ -119,9 +119,9 @@ public Iterator iterator() static class MovingAverageIterator implements Iterator { - private final Collection dims; - private final Map, Collection>> averagers = new HashMap<>(); - private final Collection> factories; + private final List dims; + private final Map, List>> averagers = new HashMap<>(); + private final List> factories; private Yielder yielder; private RowBucket cache = null; @@ -134,8 +134,8 @@ static class MovingAverageIterator implements Iterator public MovingAverageIterator( Sequence rows, - Collection dims, - Collection> factories, + List dims, + List> factories, Map fakeEvents, Map aggMap ) @@ -238,7 +238,7 @@ private Row internalNext() cache = null; } - if (cacheIter == null && averagersKeysIter == null && yielder.isDone()) { + if (cacheIter == null && yielder.isDone()) { // we should never get here. For some reason, there is // no more work to do, so continuing to iterate will infinite loop throw new NoSuchElementException(); @@ -267,12 +267,13 @@ private Row internalNext() * * @return The updated row containing averager results, or null if no averagers computed a result */ + @Nullable private Row computeMovingAverage(Map key, Row r, boolean skip) { Map event = ((MapBasedRow) r).getEvent(); Map result = new HashMap<>(event); - Collection> avg = averagers.get(key); + List> avg = averagers.get(key); if (avg == null) { avg = factories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index 39c13604e373..fbf3560ccd17 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -93,7 +93,7 @@ public class MovingAverageIterableTest public void testNext() { - Collection dims = Arrays.asList( + List dims = Arrays.asList( new DefaultDimensionSpec(GENDER, GENDER), new DefaultDimensionSpec(AGE, AGE), new DefaultDimensionSpec(COUNTRY, COUNTRY) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 1169653dd1b9..00f787ea4c85 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -85,7 +85,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import static org.junit.Assert.assertEquals; diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index 0dcfe1c6e4d1..c48bd3de2b86 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -105,10 +105,10 @@ public void testCompleteData() rows.add(JAN_3_M_10); rows.add(JAN_4_M_10); - List expected_day1 = Collections.singletonList(JAN_1_M_10); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_M_10); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Collections.singletonList(JAN_1_M_10); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_M_10); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -116,19 +116,19 @@ public void testCompleteData() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // all days present and last day only has one row diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties index 1e395f3164e0..e158009ccaa4 100644 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties @@ -1,2 +1,2 @@ druid.processing.buffer.sizeBytes=655360 -druid.request.logging.type=slf4j \ No newline at end of file +druid.request.logging.type=slf4j From 8157fce23d8c60af72f56379479458b4428e4ab1 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Wed, 27 Feb 2019 12:17:26 -0800 Subject: [PATCH 24/45] More fixes from review. --- .../movingaverage/BucketingAccumulator.java | 3 +- .../MovingAverageQueryRunner.java | 1 - .../MovingAverageQueryToolChest.java | 7 +- .../PostAveragerAggregatorCalculator.java | 5 +- .../druid/query/movingaverage/RowBucket.java | 7 +- .../MovingAverageIterableTest.java | 64 +++++++++---------- .../movingaverage/MovingAverageQueryTest.java | 1 + 7 files changed, 47 insertions(+), 41 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java index 1aa8782b242f..33e1857e50d5 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.List; /** * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. @@ -37,7 +38,7 @@ public class BucketingAccumulator extends YieldingAccumulator @Override public RowBucket accumulate(RowBucket accumulated, Row in) { - Collection rows; + List rows; if (accumulated == null) { // first row, initializing diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 64a503cfd97d..3ea81996cfa4 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -76,7 +76,6 @@ public class MovingAverageQueryRunner implements QueryRunner private final RequestLogger requestLogger; public MovingAverageQueryRunner( - QueryToolChestWarehouse warehouse, @Nullable QuerySegmentWalker walker, RequestLogger requestLogger ) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index 9164c30a039b..3d02378fdf56 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.inject.Inject; +import com.google.inject.Provider; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.QueryMetrics; @@ -59,10 +60,10 @@ public class MovingAverageQueryToolChest extends QueryToolChest walker, RequestLogger requestLogger) { - this.walker = walker; + this.walker = walker.get(); this.requestLogger = requestLogger; this.movingAverageQueryMetricsFactory = DefaultMovingAverageQueryMetricsFactory.instance(); } @@ -76,7 +77,7 @@ public void setWarehouse(QueryToolChestWarehouse warehouse) @Override public QueryRunner mergeResults(QueryRunner runner) { - return new MovingAverageQueryRunner(warehouse, walker, requestLogger); + return new MovingAverageQueryRunner(walker, requestLogger); } @Override diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java index 5031330badaa..021754183aae 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java @@ -24,8 +24,11 @@ import org.apache.druid.data.input.Row; import org.apache.druid.query.aggregation.PostAggregator; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; /** * Function that can be applied to a Sequence to calculate PostAverager results @@ -47,7 +50,7 @@ public PostAveragerAggregatorCalculator(MovingAverageQuery maq) public Row apply(Row input) { MapBasedRow row = (MapBasedRow) input; - Map event = row.getEvent(); + Map event = new HashMap<>(row.getEvent()); for (PostAggregator postAverager : postAveragers) { boolean allColsPresent = postAverager.getDependentFields().stream().allMatch(c -> event.get(c) != null); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java index 2c96530a209a..3cf14e16ed07 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -23,6 +23,7 @@ import org.joda.time.DateTime; import java.util.Collection; +import java.util.List; /** * Represents a set of rows for a specific date @@ -30,10 +31,10 @@ public class RowBucket { private final DateTime dateTime; - private final Collection rows; + private final List rows; private RowBucket nextBucket = null; - public RowBucket(DateTime dateTime, Collection rows) + public RowBucket(DateTime dateTime, List rows) { this.dateTime = dateTime; this.rows = rows; @@ -44,7 +45,7 @@ public DateTime getDateTime() return dateTime; } - public Collection getRows() + public List getRows() { return rows; } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index fbf3560ccd17..c9ef22e7e4cf 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -266,17 +266,17 @@ public void testCompleteData() List ds = new ArrayList<>(); ds.add(new DefaultDimensionSpec("gender", "gender")); - Row jan_1_row1 = new MapBasedRow(JAN_1, event1); - Row jan_1_row2 = new MapBasedRow(JAN_1, event2); - Row jan_1_row3 = new MapBasedRow(JAN_1, event3); + Row jan1Row1 = new MapBasedRow(JAN_1, event1); + Row jan1Row2 = new MapBasedRow(JAN_1, event2); + Row jan1Row3 = new MapBasedRow(JAN_1, event3); - Row jan_2_row1 = new MapBasedRow(JAN_2, event1); - Row jan_2_row2 = new MapBasedRow(JAN_2, event2); - Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + Row jan2Row1 = new MapBasedRow(JAN_2, event1); + Row jan2Row2 = new MapBasedRow(JAN_2, event2); + Row jan2Row3 = new MapBasedRow(JAN_2, event3); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + new RowBucket(JAN_1, Arrays.asList(jan1Row1, jan1Row2, jan1Row3)), + new RowBucket(JAN_2, Arrays.asList(jan2Row1, jan2Row2, jan2Row3)) )); Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( @@ -340,15 +340,15 @@ public void testMissingDataAtBeginning() List ds = new ArrayList<>(); ds.add(new DefaultDimensionSpec("gender", "gender")); - Row jan_1_row1 = new MapBasedRow(JAN_1, event1); + Row jan1Row1 = new MapBasedRow(JAN_1, event1); - Row jan_2_row1 = new MapBasedRow(JAN_2, event1); - Row jan_2_row2 = new MapBasedRow(JAN_2, event2); - Row jan_2_row3 = new MapBasedRow(JAN_2, event3); + Row jan2Row1 = new MapBasedRow(JAN_2, event1); + Row jan2Row2 = new MapBasedRow(JAN_2, event2); + Row jan2Row3 = new MapBasedRow(JAN_2, event3); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Collections.singletonList(jan_1_row1)), - new RowBucket(JAN_2, Arrays.asList(jan_2_row1, jan_2_row2, jan_2_row3)) + new RowBucket(JAN_1, Collections.singletonList(jan1Row1)), + new RowBucket(JAN_2, Arrays.asList(jan2Row1, jan2Row2, jan2Row3)) )); Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( @@ -401,14 +401,14 @@ public void testMissingDataAtTheEnd() List ds = new ArrayList<>(); ds.add(new DefaultDimensionSpec("gender", "gender")); - Row jan_1_row1 = new MapBasedRow(JAN_1, event1); - Row jan_1_row2 = new MapBasedRow(JAN_1, event2); - Row jan_1_row3 = new MapBasedRow(JAN_1, event3); - Row jan_2_row1 = new MapBasedRow(JAN_2, event1); + Row jan1Row1 = new MapBasedRow(JAN_1, event1); + Row jan1Row2 = new MapBasedRow(JAN_1, event2); + Row jan1Row3 = new MapBasedRow(JAN_1, event3); + Row jan2Row1 = new MapBasedRow(JAN_2, event1); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)) + new RowBucket(JAN_1, Arrays.asList(jan1Row1, jan1Row2, jan1Row3)), + new RowBucket(JAN_2, Collections.singletonList(jan2Row1)) )); Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( @@ -472,20 +472,20 @@ public void testMissingDataAtMiddle() List ds = new ArrayList<>(); ds.add(new DefaultDimensionSpec("gender", "gender")); - Row jan_1_row1 = new MapBasedRow(JAN_1, event1); - Row jan_1_row2 = new MapBasedRow(JAN_1, event2); - Row jan_1_row3 = new MapBasedRow(JAN_1, event3); - Row jan_2_row1 = new MapBasedRow(JAN_2, event1); - Row jan_3_row1 = new MapBasedRow(JAN_3, event1); - Row jan_3_row2 = new MapBasedRow(JAN_3, event2); - Row jan_3_row3 = new MapBasedRow(JAN_3, event3); - Row jan_4_row1 = new MapBasedRow(JAN_4, event1); + Row jan1Row1 = new MapBasedRow(JAN_1, event1); + Row jan1Row2 = new MapBasedRow(JAN_1, event2); + Row jan1Row3 = new MapBasedRow(JAN_1, event3); + Row jan2Row1 = new MapBasedRow(JAN_2, event1); + Row jan3Row1 = new MapBasedRow(JAN_3, event1); + Row jan3Row2 = new MapBasedRow(JAN_3, event2); + Row jan3Row3 = new MapBasedRow(JAN_3, event3); + Row jan4Row1 = new MapBasedRow(JAN_4, event1); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan_1_row1, jan_1_row2, jan_1_row3)), - new RowBucket(JAN_2, Collections.singletonList(jan_2_row1)), - new RowBucket(JAN_3, Arrays.asList(jan_3_row1, jan_3_row2, jan_3_row3)), - new RowBucket(JAN_4, Collections.singletonList(jan_4_row1)) + new RowBucket(JAN_1, Arrays.asList(jan1Row1, jan1Row2, jan1Row3)), + new RowBucket(JAN_2, Collections.singletonList(jan2Row1)), + new RowBucket(JAN_3, Arrays.asList(jan3Row1, jan3Row2, jan3Row3)), + new RowBucket(JAN_4, Collections.singletonList(jan4Row1)) )); Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 00f787ea4c85..1893615492b9 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -27,6 +27,7 @@ import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.google.inject.Injector; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.name.Names; import com.google.inject.util.Providers; import mockit.Mock; From 4f0fbf07f0bfac11b29305808a49f252cb518a3f Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:06:58 -0700 Subject: [PATCH 25/45] MapBasedRow is Unmodifiable. Create new rows instead of modifying existing ones. --- .../PostAveragerAggregatorCalculator.java | 22 ++++++++----- .../movingaverage/MovingAverageQueryTest.java | 33 +++++-------------- 2 files changed, 22 insertions(+), 33 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java index 021754183aae..350cee3e09ca 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java @@ -20,9 +20,11 @@ package org.apache.druid.query.movingaverage; import com.google.common.base.Function; +import com.google.common.collect.Maps; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.groupby.GroupByQueryConfig; import java.util.Collection; import java.util.HashMap; @@ -43,21 +45,23 @@ public PostAveragerAggregatorCalculator(MovingAverageQuery maq) this.postAveragers = maq.getPostAveragerSpecs(); } - /* (non-Javadoc) - * @see com.google.common.base.Function#apply(java.lang.Object) - */ @Override - public Row apply(Row input) + public Row apply(final Row row) { - MapBasedRow row = (MapBasedRow) input; - Map event = new HashMap<>(row.getEvent()); + if (postAveragers.isEmpty()) { + return row; + } + + final Map newMap; + + newMap = Maps.newLinkedHashMap(((MapBasedRow) row).getEvent()); for (PostAggregator postAverager : postAveragers) { - boolean allColsPresent = postAverager.getDependentFields().stream().allMatch(c -> event.get(c) != null); - event.put(postAverager.getName(), allColsPresent ? postAverager.compute(event) : null); + boolean allColsPresent = postAverager.getDependentFields().stream().allMatch(c -> newMap.get(c) != null); + newMap.put(postAverager.getName(), allColsPresent ? postAverager.compute(newMap) : null); } - return input; + return new MapBasedRow(row.getTimestamp(), newMap); } } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 1893615492b9..74ace2358d18 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -25,9 +25,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.common.collect.Maps; import com.google.inject.Injector; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.name.Names; import com.google.inject.util.Providers; import mockit.Mock; @@ -259,15 +259,6 @@ protected void defineMocks(Query query) throws IOException groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference>() { })); - for (Row r : groupByResults) { - Map map = ((MapBasedRow) r).getEvent(); - for (AggregatorFactory agg : aggs) { - Object serializedVal = map.get(agg.getName()); - if (serializedVal != null) { - map.put(agg.getName(), agg.deserialize(serializedVal)); - } - } - } } if (getTimeseriesResultJson() != null) { @@ -277,15 +268,6 @@ protected void defineMocks(Query query) throws IOException { } )); - for (Result r : timeseriesResults) { - Map map = r.getValue().getBaseObject(); - for (AggregatorFactory agg : aggs) { - Object serializedVal = map.get(agg.getName()); - if (serializedVal != null) { - map.put(agg.getName(), agg.deserialize(serializedVal)); - } - } - } } } @@ -294,10 +276,11 @@ protected void defineMocks(Query query) throws IOException * * @param result */ - protected void consistentTypeCasting(List result) + protected List consistentTypeCasting(List result) { + List newResult = new ArrayList<>(); for (MapBasedRow row : result) { - Map event = row.getEvent(); + final Map event = Maps.newLinkedHashMap((row).getEvent()); event.forEach((key, value) -> { if (Integer.class.isInstance(value)) { event.put(key, ((Integer) value).longValue()); @@ -306,8 +289,10 @@ protected void consistentTypeCasting(List result) event.put(key, ((Float) value).doubleValue()); } }); - + newResult.add(new MapBasedRow(row.getTimestamp(), event)); } + + return newResult; } /** @@ -440,8 +425,8 @@ baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheCon List actualResults = new ArrayList(); actualResults = (List) res.accumulate(actualResults, Accumulators.list()); - consistentTypeCasting(expectedResults); - consistentTypeCasting(actualResults); + expectedResults = consistentTypeCasting(expectedResults); + actualResults = consistentTypeCasting(actualResults); assertEquals(expectedResults, actualResults); } From 5221a8363e6d397d264dd59e81994278e2fc38a5 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:28:16 -0700 Subject: [PATCH 26/45] Remove more changes related to datasketches support. --- .../query/movingaverage/averagers/BaseAverager.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index 00fee22216ea..876f6299add1 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -52,10 +52,8 @@ public abstract class BaseAverager implements Averager * @param name The name of the resulting metric * @param fieldName The field to extra from incoming rows and stored in the window cache * @param cycleSize Cycle group size. Used to calculate day-of-week option. Default=1 (single element in group). - * @param shouldFinalize Whether the intermediate values need to be finalized prior to averaging */ - public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize, - boolean shouldFinalize) + public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize) { this.numBuckets = numBuckets; this.name = name; @@ -64,15 +62,9 @@ public BaseAverager(Class storageType, int numBuckets, String name, String fi @SuppressWarnings("unchecked") final I[] array = (I[]) Array.newInstance(storageType, numBuckets); this.buckets = array; - this.shouldFinalize = shouldFinalize; this.cycleSize = cycleSize; } - public BaseAverager(Class storageType, int numBuckets, String name, String fieldName, int cycleSize) - { - this(storageType, numBuckets, name, fieldName, cycleSize, true); - } - /* (non-Javadoc) * @see Averager#addElement(java.util.Map, java.util.Map) @@ -83,7 +75,7 @@ public void addElement(Map e, Map a) { Object metric = e.get(fieldName); I finalMetric; - if (a.containsKey(fieldName) && shouldFinalize) { + if (a.containsKey(fieldName)) { AggregatorFactory af = a.get(fieldName); finalMetric = metric != null ? (I) af.finalizeComputation(metric) : null; } else { From e2a5317037abb47b33910035ded8f87382952630 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:28:55 -0700 Subject: [PATCH 27/45] Refactor BaseAverager startFrom field and add a comment. --- .../query/movingaverage/averagers/BaseAverager.java | 12 ++++++------ .../movingaverage/averagers/DoubleMaxAverager.java | 9 ++------- .../movingaverage/averagers/DoubleMeanAverager.java | 9 ++------- .../averagers/DoubleMeanNoNullAverager.java | 9 ++------- .../movingaverage/averagers/DoubleMinAverager.java | 9 ++------- .../movingaverage/averagers/LongMaxAverager.java | 9 ++------- .../movingaverage/averagers/LongMeanAverager.java | 9 ++------- .../averagers/LongMeanNoNullAverager.java | 9 ++------- .../movingaverage/averagers/LongMinAverager.java | 9 ++------- 9 files changed, 22 insertions(+), 62 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index 876f6299add1..8ecef9c15c77 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -35,16 +35,16 @@ public abstract class BaseAverager implements Averager { - private final int numBuckets; - private final int cycleSize; + final int numBuckets; + final int cycleSize; private final String name; private final String fieldName; - private final I[] buckets; + final I[] buckets; private int index; - /* This flag checks if the intermediate sketches have to be finalized or not - for further computation involving sketches */ - private boolean shouldFinalize; + /* startFrom is needed because `buckets` field is a fixed array, not a list. + It makes computeResults() start from the correct bucket in the array. */ + int startFrom = 0; /** * @param storageType The class to use for storing intermediate values diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java index 2c6ac9cdf5ab..5e25617025b6 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMaxAverager.java @@ -22,8 +22,6 @@ public class DoubleMaxAverager extends BaseAverager { - private int startFrom = 0; - public DoubleMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -33,13 +31,10 @@ public DoubleMaxAverager(int numBuckets, String name, String fieldName, int cycl protected Double computeResult() { double result = Double.NEGATIVE_INFINITY; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result = Double.max(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + if (buckets[(i + startFrom) % numBuckets] != null) { + result = Double.max(result, (buckets[(i + startFrom) % numBuckets]).doubleValue()); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java index 9b0459df0db8..be9292c94c7b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanAverager.java @@ -22,8 +22,6 @@ public class DoubleMeanAverager extends BaseAverager { - private int startFrom = 0; - public DoubleMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -34,13 +32,10 @@ protected Double computeResult() { double result = 0.0; int validBuckets = 0; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + if (buckets[(i + startFrom) % numBuckets] != null) { + result += (buckets[(i + startFrom) % numBuckets]).doubleValue(); } else { result += 0.0; } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java index c3b8adcd519d..573f12a9e8b8 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMeanNoNullAverager.java @@ -22,8 +22,6 @@ public class DoubleMeanNoNullAverager extends BaseAverager { - private int startFrom = 0; - public DoubleMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -34,13 +32,10 @@ protected Double computeResult() { double result = 0.0; int validBuckets = 0; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result += (obj[(i + startFrom) % numBuckets]).doubleValue(); + if (buckets[(i + startFrom) % numBuckets] != null) { + result += (buckets[(i + startFrom) % numBuckets]).doubleValue(); validBuckets++; } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java index 1c6a4cb58d74..d108feed0224 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/DoubleMinAverager.java @@ -22,8 +22,6 @@ public class DoubleMinAverager extends BaseAverager { - private int startFrom = 0; - public DoubleMinAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -33,13 +31,10 @@ public DoubleMinAverager(int numBuckets, String name, String fieldName, int cycl protected Double computeResult() { double result = Double.POSITIVE_INFINITY; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result = Double.min(result, (obj[(i + startFrom) % numBuckets]).doubleValue()); + if (buckets[(i + startFrom) % numBuckets] != null) { + result = Double.min(result, (buckets[(i + startFrom) % numBuckets]).doubleValue()); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java index b2c04d12aecd..a45503ca58c1 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMaxAverager.java @@ -22,8 +22,6 @@ public class LongMaxAverager extends BaseAverager { - private int startFrom = 0; - public LongMaxAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -33,13 +31,10 @@ public LongMaxAverager(int numBuckets, String name, String fieldName, int cycleS protected Long computeResult() { long result = Long.MIN_VALUE; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result = Long.max(result, (obj[(i + startFrom) % numBuckets]).longValue()); + if (buckets[(i + startFrom) % numBuckets] != null) { + result = Long.max(result, (buckets[(i + startFrom) % numBuckets]).longValue()); } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java index 8d7892d7c22c..a5919d727f78 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanAverager.java @@ -22,8 +22,6 @@ public class LongMeanAverager extends BaseAverager { - private int startFrom = 0; - public LongMeanAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -34,13 +32,10 @@ protected Double computeResult() { long result = 0; int validBuckets = 0; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result += (obj[(i + startFrom) % numBuckets]).longValue(); + if (buckets[(i + startFrom) % numBuckets] != null) { + result += (buckets[(i + startFrom) % numBuckets]).longValue(); } else { result += 0; } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java index 0e59af32d628..ecdd17a6f265 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMeanNoNullAverager.java @@ -22,8 +22,6 @@ public class LongMeanNoNullAverager extends BaseAverager { - private int startFrom = 0; - public LongMeanNoNullAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -34,13 +32,10 @@ protected Double computeResult() { long result = 0; int validBuckets = 0; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result += (obj[(i + startFrom) % numBuckets]).longValue(); + if (buckets[(i + startFrom) % numBuckets] != null) { + result += (buckets[(i + startFrom) % numBuckets]).longValue(); validBuckets++; } } diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java index fedf055ce91e..cc999e6b9abf 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/LongMinAverager.java @@ -22,8 +22,6 @@ public class LongMinAverager extends BaseAverager { - private int startFrom = 0; - public LongMinAverager(int numBuckets, String name, String fieldName, int cycleSize) { super(Number.class, numBuckets, name, fieldName, cycleSize); @@ -33,13 +31,10 @@ public LongMinAverager(int numBuckets, String name, String fieldName, int cycleS protected Long computeResult() { long result = Long.MAX_VALUE; - int cycleSize = getCycleSize(); - int numBuckets = getNumBuckets(); - Number[] obj = getBuckets(); for (int i = 0; i < numBuckets; i += cycleSize) { - if (obj[(i + startFrom) % numBuckets] != null) { - result = Long.min(result, (obj[(i + startFrom) % numBuckets]).longValue()); + if (buckets[(i + startFrom) % numBuckets] != null) { + result = Long.min(result, (buckets[(i + startFrom) % numBuckets]).longValue()); } } From 4b425b2706ea474c03efdd2f7e66f36708b06a04 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:29:41 -0700 Subject: [PATCH 28/45] fakeEvents field: Refactor initialization and add comment. --- .../movingaverage/MovingAverageIterable.java | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 9d755b35fd3d..8657471b1235 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -34,6 +34,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.HashMap; import java.util.HashSet; @@ -75,8 +76,28 @@ public MovingAverageIterable( postAggMap = postAggList.stream().collect(Collectors.toMap(postAgg -> postAgg.getName(), postAgg -> postAgg)); aggMap = aggList.stream().collect(Collectors.toMap(agg -> agg.getName(), agg -> agg)); + fakeEvents = generateFakeEventsFromAggregators(aggMap, postAggMap); + } - ColumnSelectorFactory colFact = new ColumnSelectorFactory() + // Build a list of dummy events from Aggregators/PostAggregators to be used by Iterator to build fake rows. + // These fake rows will be used by computeMovingAverage() in skip=true mode. + // See fakeEventsCopy in internalNext() and computeMovingAverage() documentation. + private Map generateFakeEventsFromAggregators(Map aggMap, + Map postAggMap) + { + Map fakeEvents = new LinkedHashMap<>(); + aggMap.values().forEach(agg -> { + Aggregator aggFactorized = agg.factorize(getEmptyColumnSelectorFactory()); + fakeEvents.put(agg.getName(), aggFactorized.get()); + }); + postAggMap.values().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); + return fakeEvents; + } + + @Nonnull + private ColumnSelectorFactory getEmptyColumnSelectorFactory() + { + return new ColumnSelectorFactory() { @Override public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) @@ -98,13 +119,6 @@ public ColumnCapabilities getColumnCapabilities(String s) return null; } }; - // Fill in all the fake events - fakeEvents = new LinkedHashMap<>(); - aggMap.values().forEach(agg -> { - Aggregator aggFactorized = agg.factorize(colFact); - fakeEvents.put(agg.getName(), aggFactorized.get()); - }); - postAggMap.values().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); } /* (non-Javadoc) From 8244353539d9d402e992009de4506b9957b57c5a Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:29:56 -0700 Subject: [PATCH 29/45] Rename parameters (tiny change). --- .../druid/query/movingaverage/MovingAverageIterable.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 8657471b1235..1644232d4abb 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -135,7 +135,7 @@ static class MovingAverageIterator implements Iterator private final List dims; private final Map, List>> averagers = new HashMap<>(); - private final List> factories; + private final List> averagerFactories; private Yielder yielder; private RowBucket cache = null; @@ -149,13 +149,13 @@ static class MovingAverageIterator implements Iterator public MovingAverageIterator( Sequence rows, List dims, - List> factories, + List> averagerFactories, Map fakeEvents, Map aggMap ) { this.dims = dims; - this.factories = factories; + this.averagerFactories = averagerFactories; this.fakeEvents = fakeEvents; this.aggMap = aggMap; @@ -290,7 +290,7 @@ private Row computeMovingAverage(Map key, Row r, boolean skip) List> avg = averagers.get(key); if (avg == null) { - avg = factories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); + avg = averagerFactories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); averagers.put(key, avg); } From f10a343ae0b597972033ba2fda91dbd3e37faf48 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:30:11 -0700 Subject: [PATCH 30/45] Fix variable name typo in test (JAN_4). --- .../druid/query/movingaverage/MovingAverageIterableTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index c9ef22e7e4cf..d5e0c6ffe75f 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -200,7 +200,7 @@ public void testAveraging() event4.put("gender", "f"); event4.put("pageViews", 40L); - Row row4 = new MapBasedRow(JAN_2, event4); + Row row4 = new MapBasedRow(JAN_3, event4); float retval = 14.5f; From c4ad4f5d0483d5e56dd635ffd5a72b25bb3dbe6d Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:30:22 -0700 Subject: [PATCH 31/45] Fix styling of non camelCase fields. --- .../movingaverage/RowBucketIterableTest.java | 240 +++++++++--------- 1 file changed, 120 insertions(+), 120 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java index c48bd3de2b86..7504a979bac2 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/RowBucketIterableTest.java @@ -99,7 +99,7 @@ public void testCompleteData() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_2_M_10); rows.add(JAN_3_M_10); @@ -139,12 +139,12 @@ public void testApplyLastDaySingleRow() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_F_20); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_F_20); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_2_M_10); @@ -156,16 +156,16 @@ public void testApplyLastDaySingleRow() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // all days present and last day has multiple rows @@ -176,12 +176,12 @@ public void testApplyLastDayMultipleRows() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_F_20); - List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_F_20); + List expectedDay4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_2_M_10); @@ -195,16 +195,16 @@ public void testApplyLastDayMultipleRows() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // test single day with single row @@ -215,17 +215,17 @@ public void testSingleDaySingleRow() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_1); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); - List expected_day1 = Collections.singletonList(JAN_1_M_10); + List expectedDay1 = Collections.singletonList(JAN_1_M_10); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); assertEquals(JAN_1, actual.getDateTime()); } @@ -238,12 +238,12 @@ public void testSingleDayMultipleRow() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_1); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_1_U_30); - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20, JAN_1_U_30); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20, JAN_1_U_30); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); @@ -251,7 +251,7 @@ public void testSingleDayMultipleRow() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); } @@ -260,13 +260,13 @@ public void testSingleDayMultipleRow() public void testMissingDaysAtBegining() { - List expected_day1 = Collections.emptyList(); - List expected_day2 = Collections.singletonList(JAN_2_M_10); + List expectedDay1 = Collections.emptyList(); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_2); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_2_M_10); Sequence seq = Sequences.simple(rows); @@ -275,11 +275,11 @@ public void testMissingDaysAtBegining() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); } @@ -288,15 +288,15 @@ public void testMissingDaysAtBegining() public void testMissingDaysAtBeginingFollowedByMultipleRow() { - List expected_day1 = Collections.emptyList(); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_M_10); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Collections.emptyList(); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_M_10); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_2_M_10); rows.add(JAN_3_M_10); rows.add(JAN_4_M_10); @@ -307,19 +307,19 @@ public void testMissingDaysAtBeginingFollowedByMultipleRow() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // missing day at the beginning and at the end @@ -327,15 +327,15 @@ public void testMissingDaysAtBeginingFollowedByMultipleRow() public void testMissingDaysAtBeginingAndAtTheEnd() { - List expected_day1 = Collections.emptyList(); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_M_10); - List expected_day4 = Collections.emptyList(); + List expectedDay1 = Collections.emptyList(); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_M_10); + List expectedDay4 = Collections.emptyList(); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_2_M_10); rows.add(JAN_3_M_10); @@ -345,19 +345,19 @@ public void testMissingDaysAtBeginingAndAtTheEnd() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // multiple missing days in an interval @@ -365,15 +365,15 @@ public void testMissingDaysAtBeginingAndAtTheEnd() public void testMultipleMissingDays() { - List expected_day1 = Collections.emptyList(); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.emptyList(); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Collections.emptyList(); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.emptyList(); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_2_M_10); rows.add(JAN_4_M_10); @@ -383,19 +383,19 @@ public void testMultipleMissingDays() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // multiple missing days in an interval followed by multiple row at the end @@ -403,16 +403,16 @@ public void testMultipleMissingDays() public void testMultipleMissingDaysMultipleRowAtTheEnd() { - List expected_day1 = Collections.emptyList(); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.emptyList(); - List expected_day4 = Collections.singletonList(JAN_4_M_10); - List expected_day5 = Collections.singletonList(JAN_5_M_10); + List expectedDay1 = Collections.emptyList(); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.emptyList(); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); + List expectedDay5 = Collections.singletonList(JAN_5_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_5); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_2_M_10); rows.add(JAN_4_M_10); rows.add(JAN_5_M_10); @@ -423,23 +423,23 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); actual = iter.next(); assertEquals(JAN_5, actual.getDateTime()); - assertEquals(expected_day5, actual.getRows()); + assertEquals(expectedDay5, actual.getRows()); } @@ -448,12 +448,12 @@ public void testMultipleMissingDaysMultipleRowAtTheEnd() public void testMissingDaysInMiddleOneRow() { - List expected_day1 = Collections.singletonList(JAN_1_M_10); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.emptyList(); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Collections.singletonList(JAN_1_M_10); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.emptyList(); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_2_M_10); rows.add(JAN_4_M_10); @@ -466,17 +466,17 @@ public void testMissingDaysInMiddleOneRow() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } @@ -485,15 +485,15 @@ public void testMissingDaysInMiddleOneRow() public void testMissingDaysInMiddleMultipleRow() { - List expected_day1 = Collections.singletonList(JAN_1_M_10); - List expected_day2 = Collections.emptyList(); - List expected_day3 = Collections.singletonList(JAN_3_M_10); - List expected_day4 = Collections.singletonList(JAN_4_M_10); + List expectedDay1 = Collections.singletonList(JAN_1_M_10); + List expectedDay2 = Collections.emptyList(); + List expectedDay3 = Collections.singletonList(JAN_3_M_10); + List expectedDay4 = Collections.singletonList(JAN_4_M_10); intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_3_M_10); rows.add(JAN_4_M_10); @@ -504,19 +504,19 @@ public void testMissingDaysInMiddleMultipleRow() RowBucket actual = iter.next(); assertEquals(JAN_1, actual.getDateTime()); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); assertEquals(JAN_2, actual.getDateTime()); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } @@ -528,12 +528,12 @@ public void testApplyLastDayNoRows() intervals = new ArrayList<>(); intervals.add(INTERVAL_JAN_1_4); - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_F_20); - List expected_day4 = Collections.emptyList(); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_F_20); + List expectedDay4 = Collections.emptyList(); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_2_M_10); @@ -544,17 +544,17 @@ public void testApplyLastDayNoRows() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } // data missing for last two days @@ -562,12 +562,12 @@ public void testApplyLastDayNoRows() public void testApplyLastTwoDayNoRows() { - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.emptyList(); - List expected_day4 = Collections.emptyList(); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.emptyList(); + List expectedDay4 = Collections.emptyList(); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_2_M_10); @@ -580,18 +580,18 @@ public void testApplyLastTwoDayNoRows() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); assertEquals(JAN_3, actual.getDateTime()); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); assertEquals(JAN_4, actual.getDateTime()); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); } @@ -603,15 +603,15 @@ public void testApplyMultipleInterval() intervals.add(INTERVAL_JAN_1_4); intervals.add(INTERVAL_JAN_6_8); - List expected_day1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); - List expected_day2 = Collections.singletonList(JAN_2_M_10); - List expected_day3 = Collections.singletonList(JAN_3_F_20); - List expected_day4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); - List expected_day6 = Collections.singletonList(JAN_6_M_10); - List expected_day7 = Collections.singletonList(JAN_7_F_20); - List expected_day8 = Collections.singletonList(JAN_8_U_30); + List expectedDay1 = Arrays.asList(JAN_1_M_10, JAN_1_F_20); + List expectedDay2 = Collections.singletonList(JAN_2_M_10); + List expectedDay3 = Collections.singletonList(JAN_3_F_20); + List expectedDay4 = Arrays.asList(JAN_4_M_10, JAN_4_F_20, JAN_4_U_30); + List expectedDay6 = Collections.singletonList(JAN_6_M_10); + List expectedDay7 = Collections.singletonList(JAN_7_F_20); + List expectedDay8 = Collections.singletonList(JAN_8_U_30); - rows = new ArrayList(); + rows = new ArrayList<>(); rows.add(JAN_1_M_10); rows.add(JAN_1_F_20); rows.add(JAN_2_M_10); @@ -628,25 +628,25 @@ public void testApplyMultipleInterval() Iterator iter = rbi.iterator(); RowBucket actual = iter.next(); - assertEquals(expected_day1, actual.getRows()); + assertEquals(expectedDay1, actual.getRows()); actual = iter.next(); - assertEquals(expected_day2, actual.getRows()); + assertEquals(expectedDay2, actual.getRows()); actual = iter.next(); - assertEquals(expected_day3, actual.getRows()); + assertEquals(expectedDay3, actual.getRows()); actual = iter.next(); - assertEquals(expected_day4, actual.getRows()); + assertEquals(expectedDay4, actual.getRows()); actual = iter.next(); - assertEquals(expected_day6, actual.getRows()); + assertEquals(expectedDay6, actual.getRows()); actual = iter.next(); - assertEquals(expected_day7, actual.getRows()); + assertEquals(expectedDay7, actual.getRows()); actual = iter.next(); - assertEquals(expected_day8, actual.getRows()); + assertEquals(expectedDay8, actual.getRows()); } @Test @@ -657,7 +657,7 @@ public void testNodata() intervals.add(INTERVAL_JAN_1_4); intervals.add(INTERVAL_JAN_6_8); - rows = new ArrayList(); + rows = new ArrayList<>(); Sequence seq = Sequences.simple(rows); RowBucketIterable rbi = new RowBucketIterable(seq, intervals, ONE_DAY); From 621019673678caae183570fa63b41b49daa23a27 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:30:48 -0700 Subject: [PATCH 32/45] Fix Preconditions.checkArgument for cycleSize. --- .../query/movingaverage/averagers/BaseAveragerFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java index dda08c04be08..831000fed5c1 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAveragerFactory.java @@ -57,7 +57,7 @@ public BaseAveragerFactory(String name, int numBuckets, String fieldName, Intege Preconditions.checkNotNull(fieldName, "Must have a valid, non-null field name"); Preconditions.checkArgument(this.cycleSize > 0, "Cycle size must be greater than zero"); Preconditions.checkArgument(numBuckets > 0, "Bucket size must be greater than zero"); - Preconditions.checkArgument(this.cycleSize < numBuckets, "Cycle size must be less than the bucket size"); + Preconditions.checkArgument(!(this.cycleSize > numBuckets), "Cycle size must be less than the bucket size"); Preconditions.checkArgument(numBuckets % this.cycleSize == 0, "cycleSize must devide numBuckets without a remainder"); } From 3ffa46c8bd636f2f3da47b372e2493df840c2f70 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:31:17 -0700 Subject: [PATCH 33/45] Add more documentation to RowBucketIterable and other classes. --- .../query/movingaverage/BucketingAccumulator.java | 4 +++- .../query/movingaverage/MovingAverageIterable.java | 1 + .../movingaverage/MovingAverageQueryRunner.java | 2 +- .../druid/query/movingaverage/RowBucket.java | 1 + .../query/movingaverage/RowBucketIterable.java | 14 +++++++++++--- .../movingaverage/MovingAverageQueryTest.java | 3 +++ 6 files changed, 20 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java index 33e1857e50d5..40c77595ef67 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -27,7 +27,9 @@ import java.util.List; /** - * Works together with {@link RowBucketIterable} to group all the rows for a specific day together. + * Groups all the rows for a specific period together. + * Rows of each period are placed in a single {@link RowBucket} (timed through the dateTime field). + * (Assumpltion: Input arrives sorted by timestamp). */ public class BucketingAccumulator extends YieldingAccumulator { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 1644232d4abb..81d3977d6a40 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -289,6 +289,7 @@ private Row computeMovingAverage(Map key, Row r, boolean skip) List> avg = averagers.get(key); + // Initialize key's averagers. if (avg == null) { avg = averagerFactories.stream().map(af -> af.createAverager()).collect(Collectors.toList()); averagers.put(key, avg); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 3ea81996cfa4..5558dc97192a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -190,7 +190,7 @@ public Sequence run(QueryPlus query, Map responseConte resultsSeq = Sequences.map(results, new TimeseriesResultToRow()); } - // Process into day buckets + // Process into period buckets Sequence bucketedMovingAvgResults = Sequences.simple(new RowBucketIterable(resultsSeq, intervals, period)); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java index 3cf14e16ed07..da390dd99733 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -27,6 +27,7 @@ /** * Represents a set of rows for a specific date + * Each RowBucket is an element in a list (holds a pointer to the next RowBucket) */ public class RowBucket { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 688eb1943302..63831e0b5d38 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -32,8 +32,15 @@ import java.util.NoSuchElementException; /** - * It is the iterable used to bucket data into days, - * doing appropriate lookahead to see if the next row is in the same day or a new day. + * An iterator which takes list of rows ({@link Sequence}) and generates a new list of {@link RowBucket}s from it. + * + * It calls {@link BucketingAccumulator} for naive backeting to buckets of periods, + * But does more suttle logic to cover edge cases, such as: + * - Handling periods with no rows. + * - Handling last record. + * + * Please notice this is being called by {@link MovingAverageIterable.MovingAverageIterator#internalNext()} + * and the logic for skipping records is comprised by the interaction between the two classes. */ public class RowBucketIterable implements Iterable { @@ -95,13 +102,14 @@ public RowBucket next() { RowBucket currentBucket = yielder.get(); + // Iterate to next interval if (expectedBucket.compareTo(intervals.get(intervalIndex).getEnd()) >= 0) { intervalIndex++; if (intervalIndex < intervals.size()) { expectedBucket = intervals.get(intervalIndex).getStart(); } } - // currentBucket > expectedBucket + // currentBucket > expectedBucket (No rows found for period). Iterate to next period. if (currentBucket != null && currentBucket.getDateTime().compareTo(expectedBucket) > 0) { currentBucket = new RowBucket(expectedBucket, Collections.emptyList()); expectedBucket = expectedBucket.plus(period); diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 74ace2358d18..fd32d3f4f0e0 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -108,6 +108,7 @@ public class MovingAverageQueryTest private final List> timeseriesResults = new ArrayList<>(); private final TestConfig config; + private final String yamlFile; @Parameters(name = "{0}") public static Iterable data() throws IOException @@ -125,6 +126,8 @@ public static Iterable data() throws IOException public MovingAverageQueryTest(String yamlFile) throws IOException { + this.yamlFile = yamlFile; + List modules = getRequiredModules(); modules.add( binder -> { From 5f0bbb4a481cefea916052c2098eb1eed6a62a50 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 15:42:09 -0700 Subject: [PATCH 34/45] key/value comment on in MovingAverageIterable. --- .../apache/druid/query/movingaverage/MovingAverageIterable.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 81d3977d6a40..e91650a06c18 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -134,6 +134,7 @@ static class MovingAverageIterator implements Iterator { private final List dims; + // Key: Row's dimension set. Value: Averager. See MovingAverageIterator#computeMovingAverage for more details. private final Map, List>> averagers = new HashMap<>(); private final List> averagerFactories; From 0ff63a5fbc4f05d8aefca6cce6622ce1819c11ae Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 16:29:36 -0700 Subject: [PATCH 35/45] Fix anonymous makeColumnValueSelector returning null. --- .../druid/query/movingaverage/MovingAverageIterable.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index e91650a06c18..d2181f62287f 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -32,6 +32,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nonnull; @@ -110,7 +111,7 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) @Override public ColumnValueSelector makeColumnValueSelector(String s) { - return null; + return NilColumnValueSelector.instance(); } @Override From e313a2b58a8edd819194369772706179f7b98768 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 17:43:21 -0700 Subject: [PATCH 36/45] Replace IdentityYieldingAccumolator with Yielders.each(). --- .../IdentityYieldingAccumulator.java | 40 ------------- .../movingaverage/MovingAverageIterable.java | 3 +- .../IdentityYieldingAccumulatorTest.java | 59 ------------------- 3 files changed, 2 insertions(+), 100 deletions(-) delete mode 100644 extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java delete mode 100644 extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java deleted file mode 100644 index 4171a72781cb..000000000000 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulator.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; - -import org.apache.druid.java.util.common.guava.YieldingAccumulator; - -/** - * Simple yielding accumulator that yields each row. - */ -public class IdentityYieldingAccumulator extends YieldingAccumulator -{ - - /* (non-Javadoc) - * @see YieldingAccumulator#accumulate(java.lang.Object, java.lang.Object) - */ - @Override - public T accumulate(T accumulated, T in) - { - yield(); - return in; - } - -} diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index d2181f62287f..33d95705cd34 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -23,6 +23,7 @@ import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; @@ -161,7 +162,7 @@ public MovingAverageIterator( this.fakeEvents = fakeEvents; this.aggMap = aggMap; - yielder = rows.toYielder(null, new IdentityYieldingAccumulator()); + yielder = Yielders.each(rows); } /* (non-Javadoc) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java deleted file mode 100644 index dac2d693a13a..000000000000 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/IdentityYieldingAccumulatorTest.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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.apache.druid.query.movingaverage; - -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.Yielder; -import org.junit.Test; - -import java.util.Arrays; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -/** - * Test IdentityYieldingAccumulator - */ -public class IdentityYieldingAccumulatorTest -{ - @Test - public void testAccumulator() - { - Sequence seq = Sequences.simple(Arrays.asList(1, 2, 3, 4, 5)); - - Yielder y = seq.toYielder(null, new IdentityYieldingAccumulator<>()); - - assertEquals(Integer.valueOf(1), y.get()); - y = y.next(null); - assertEquals(Integer.valueOf(2), y.get()); - y = y.next(null); - assertEquals(Integer.valueOf(3), y.get()); - y = y.next(null); - assertEquals(Integer.valueOf(4), y.get()); - y = y.next(null); - assertEquals(Integer.valueOf(5), y.get()); - y = y.next(null); - assertTrue(y.isDone()); - - assertNull(y.get()); - } -} From 67087208f21083e3ef2246af61ee4e6a185875da Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 17:44:16 -0700 Subject: [PATCH 37/45] * internalNext() should return null instead of throwing exception. * Remove unused variables/prarameters. --- .../movingaverage/MovingAverageIterable.java | 14 +++++--------- .../movingaverage/MovingAverageQueryTest.java | 19 ++----------------- 2 files changed, 7 insertions(+), 26 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 33d95705cd34..244bde70eb6f 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -176,15 +176,11 @@ public boolean hasNext() return true; } - try { - saveNext = internalNext(); - return true; - } - catch (NoSuchElementException e) { - return false; + Row row = internalNext(); + if (row != null) { + saveNext = row; } - - + return (row != null); } /* (non-Javadoc) @@ -258,7 +254,7 @@ private Row internalNext() if (cacheIter == null && yielder.isDone()) { // we should never get here. For some reason, there is // no more work to do, so continuing to iterate will infinite loop - throw new NoSuchElementException(); + return null; } // nothing to do here, so move on to the next row diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index fd32d3f4f0e0..944112d8a1e6 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -234,29 +234,14 @@ protected List getRequiredModules() /** * Set up any needed mocks to stub out backend query behavior. * - * @param query - * * @throws IOException * @throws JsonMappingException * @throws JsonParseException */ - protected void defineMocks(Query query) throws IOException + protected void defineMocks() throws IOException { groupByResults.clear(); timeseriesResults.clear(); - List aggs; - - if (query instanceof GroupByQuery) { - aggs = ((GroupByQuery) query).getAggregatorSpecs(); - } else if (query instanceof TimeseriesQuery) { - aggs = ((TimeseriesQuery) query).getAggregatorSpecs(); - } else if (query instanceof MovingAverageQuery) { - aggs = ((MovingAverageQuery) query).getAggregatorSpecs(); - } else { - // unrecognized query type - aggs = Collections.emptyList(); - - } if (getGroupByResultJson() != null) { groupByResults.addAll(jsonMapper.readValue(getGroupByResultJson(), new TypeReference>() @@ -420,7 +405,7 @@ baseClient, warehouse, retryConfig, jsonMapper, serverConfig, null, new CacheCon ); final Map responseContext = new HashMap<>(); - defineMocks(query); + defineMocks(); QueryPlus queryPlus = QueryPlus.wrap(query); final Sequence res = query.getRunner(walker).run(queryPlus, responseContext); From fa3fbbcfae23e45b5213292ca93cdcce735f02b7 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Mon, 18 Mar 2019 18:29:12 -0700 Subject: [PATCH 38/45] Harden MovingAverageIterableTest (Switch anyOf to exact match). --- .../MovingAverageIterableTest.java | 63 ++++++++++--------- 1 file changed, 32 insertions(+), 31 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java index d5e0c6ffe75f..3acc1f71a46f 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageIterableTest.java @@ -39,16 +39,13 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import static org.hamcrest.CoreMatchers.anyOf; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -441,12 +438,12 @@ public void testMissingDataAtTheEnd() assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals("u", (result.getDimension("gender")).get(0)); assertEquals(JAN_2, (result.getTimestamp())); assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals("f", (result.getDimension("gender")).get(0)); assertEquals(JAN_2, (result.getTimestamp())); assertFalse(iter.hasNext()); @@ -457,35 +454,35 @@ public void testMissingDataAtTheEnd() public void testMissingDataAtMiddle() { - Map event1 = new HashMap<>(); - Map event2 = new HashMap<>(); - Map event3 = new HashMap<>(); + Map eventM = new HashMap<>(); + Map eventF = new HashMap<>(); + Map eventU = new HashMap<>(); Map event4 = new HashMap<>(); - event1.put("gender", "m"); - event1.put("pageViews", 10L); - event2.put("gender", "f"); - event2.put("pageViews", 20L); - event3.put("gender", "u"); - event3.put("pageViews", 30L); + eventM.put("gender", "m"); + eventM.put("pageViews", 10L); + eventF.put("gender", "f"); + eventF.put("pageViews", 20L); + eventU.put("gender", "u"); + eventU.put("pageViews", 30L); List ds = new ArrayList<>(); ds.add(new DefaultDimensionSpec("gender", "gender")); - Row jan1Row1 = new MapBasedRow(JAN_1, event1); - Row jan1Row2 = new MapBasedRow(JAN_1, event2); - Row jan1Row3 = new MapBasedRow(JAN_1, event3); - Row jan2Row1 = new MapBasedRow(JAN_2, event1); - Row jan3Row1 = new MapBasedRow(JAN_3, event1); - Row jan3Row2 = new MapBasedRow(JAN_3, event2); - Row jan3Row3 = new MapBasedRow(JAN_3, event3); - Row jan4Row1 = new MapBasedRow(JAN_4, event1); + Row jan1Row1M = new MapBasedRow(JAN_1, eventM); + Row jan1Row2F = new MapBasedRow(JAN_1, eventF); + Row jan1Row3U = new MapBasedRow(JAN_1, eventU); + Row jan2Row1M = new MapBasedRow(JAN_2, eventM); + Row jan3Row1M = new MapBasedRow(JAN_3, eventM); + Row jan3Row2F = new MapBasedRow(JAN_3, eventF); + Row jan3Row3U = new MapBasedRow(JAN_3, eventU); + Row jan4Row1M = new MapBasedRow(JAN_4, eventM); Sequence seq = Sequences.simple(Arrays.asList( - new RowBucket(JAN_1, Arrays.asList(jan1Row1, jan1Row2, jan1Row3)), - new RowBucket(JAN_2, Collections.singletonList(jan2Row1)), - new RowBucket(JAN_3, Arrays.asList(jan3Row1, jan3Row2, jan3Row3)), - new RowBucket(JAN_4, Collections.singletonList(jan4Row1)) + new RowBucket(JAN_1, Arrays.asList(jan1Row1M, jan1Row2F, jan1Row3U)), + new RowBucket(JAN_2, Collections.singletonList(jan2Row1M)), + new RowBucket(JAN_3, Arrays.asList(jan3Row1M, jan3Row2F, jan3Row3U)), + new RowBucket(JAN_4, Collections.singletonList(jan4Row1M)) )); Iterator iter = new MovingAverageIterable(seq, ds, Collections.singletonList( @@ -496,6 +493,7 @@ public void testMissingDataAtMiddle() )) ).iterator(); + // Jan 1 assertTrue(iter.hasNext()); Row result = iter.next(); assertEquals("m", (result.getDimension("gender")).get(0)); @@ -511,6 +509,7 @@ public void testMissingDataAtMiddle() assertEquals("u", (result.getDimension("gender")).get(0)); assertEquals(JAN_1, (result.getTimestamp())); + // Jan 2 assertTrue(iter.hasNext()); result = iter.next(); assertEquals("m", (result.getDimension("gender")).get(0)); @@ -518,14 +517,15 @@ public void testMissingDataAtMiddle() assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals("u", (result.getDimension("gender")).get(0)); assertEquals(JAN_2, (result.getTimestamp())); assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("f"), is("u"))); + assertEquals("f", (result.getDimension("gender")).get(0)); assertEquals(JAN_2, (result.getTimestamp())); + // Jan 3 assertTrue(iter.hasNext()); result = iter.next(); assertEquals("m", (result.getDimension("gender")).get(0)); @@ -541,19 +541,20 @@ public void testMissingDataAtMiddle() assertEquals("u", (result.getDimension("gender")).get(0)); assertEquals(JAN_3, (result.getTimestamp())); + // Jan 4 assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals("m", (result.getDimension("gender")).get(0)); assertEquals(JAN_4, (result.getTimestamp())); assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals("u", (result.getDimension("gender")).get(0)); assertEquals(JAN_4, (result.getTimestamp())); assertTrue(iter.hasNext()); result = iter.next(); - assertThat((result.getDimension("gender")).get(0), anyOf(is("m"), is("f"), is("u"))); + assertEquals("f", (result.getDimension("gender")).get(0)); assertEquals(JAN_4, (result.getTimestamp())); assertFalse(iter.hasNext()); From 304c43dac51e60acae045ac73f46c42305963f99 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Tue, 19 Mar 2019 01:27:07 -0700 Subject: [PATCH 39/45] Change internalNext() from recursion to iteration; Simplify next() and hasNext(). --- .../movingaverage/MovingAverageIterable.java | 114 +++++++++--------- 1 file changed, 55 insertions(+), 59 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 244bde70eb6f..faeda92b389a 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -171,16 +171,12 @@ public MovingAverageIterator( @Override public boolean hasNext() { - if (saveNext != null) { return true; } - Row row = internalNext(); - if (row != null) { - saveNext = row; - } - return (row != null); + saveNext = internalNext(); + return (saveNext != null); } /* (non-Javadoc) @@ -189,76 +185,76 @@ public boolean hasNext() @Override public Row next() { - - if (saveNext != null) { - Row retVal = saveNext; - saveNext = null; - return retVal; + if (!hasNext()) { + throw new NoSuchElementException(); } - return internalNext(); + Row retVal = saveNext; + saveNext = null; + return retVal; } private Row internalNext() { - if (cache == null && !yielder.isDone()) { - cache = yielder.get(); - yielder = yielder.next(cache); - - cacheIter = cache.getRows().iterator(); - } + // Iterate until there is a row to return or Yielder is exahusted, in such a case return null. + // This is used in order to skip empty buckets (iterate to the next one). + while (true) { + if (cache == null && !yielder.isDone()) { + cache = yielder.get(); + yielder = yielder.next(cache); + + cacheIter = cache.getRows().iterator(); + } - Row r; - - // return rows from the cached RowBucket - if (cacheIter != null) { - if (cacheIter.hasNext()) { - r = cacheIter.next(); - Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); - seenKeys.add(key); - r = computeMovingAverage(key, r, false); - if (r != null) { - return r; + Row r; + + // return rows from the cached RowBucket + if (cacheIter != null) { + if (cacheIter.hasNext()) { + r = cacheIter.next(); + Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); + seenKeys.add(key); + r = computeMovingAverage(key, r, false); + if (r != null) { + return r; + } else { + throw new NoSuchElementException(); + } } else { - throw new NoSuchElementException(); + Set> averagerKeys = new HashSet<>(averagers.keySet()); + averagerKeys.removeAll(seenKeys); + averagersKeysIter = averagerKeys.iterator(); + cacheIter = null; } - } else { - Set> averagerKeys = new HashSet<>(averagers.keySet()); - averagerKeys.removeAll(seenKeys); - averagersKeysIter = averagerKeys.iterator(); - cacheIter = null; } - } - // return fake rows for unseen dimension combinations - if (averagersKeysIter != null) { - while (averagersKeysIter.hasNext()) { - Map dims = averagersKeysIter.next(); - Map fakeEventsCopy = new HashMap<>(fakeEvents); + // return fake rows for unseen dimension combinations + if (averagersKeysIter != null) { + while (averagersKeysIter.hasNext()) { + Map dims = averagersKeysIter.next(); + Map fakeEventsCopy = new HashMap<>(fakeEvents); - dims.forEach((dim, value) -> { - fakeEventsCopy.put(dim, value); - }); + dims.forEach((dim, value) -> { + fakeEventsCopy.put(dim, value); + }); - r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), fakeEventsCopy), true); - if (r != null) { - return r; + r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), fakeEventsCopy), true); + if (r != null) { + return r; + } } - } - seenKeys.clear(); - averagersKeysIter = null; - cache = null; - } + seenKeys.clear(); + averagersKeysIter = null; + cache = null; + } - if (cacheIter == null && yielder.isDone()) { - // we should never get here. For some reason, there is - // no more work to do, so continuing to iterate will infinite loop - return null; + if (cacheIter == null && yielder.isDone()) { + // we should never get here. For some reason, there is + // no more work to do, so continuing to iterate will infinite loop + return null; + } } - - // nothing to do here, so move on to the next row - return internalNext(); } /** From 81d0909780febd75eb8c1b3d138439ea3bb3293f Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Tue, 19 Mar 2019 02:17:49 -0700 Subject: [PATCH 40/45] Remove unused imports. --- .../druid/query/movingaverage/BucketingAccumulator.java | 1 - .../druid/query/movingaverage/MovingAverageQueryRunner.java | 1 - .../query/movingaverage/MovingAverageQueryToolChest.java | 1 - .../query/movingaverage/PostAveragerAggregatorCalculator.java | 4 ---- .../java/org/apache/druid/query/movingaverage/RowBucket.java | 1 - .../druid/query/movingaverage/MovingAverageQueryTest.java | 2 -- 6 files changed, 10 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java index 40c77595ef67..a79e24bf3d97 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/BucketingAccumulator.java @@ -23,7 +23,6 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator; import java.util.ArrayList; -import java.util.Collection; import java.util.List; /** diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java index 5558dc97192a..8834d0dd42f4 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryRunner.java @@ -35,7 +35,6 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; -import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index 3d02378fdf56..95bf7d4d0e26 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -36,7 +36,6 @@ import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.server.log.RequestLogger; -import javax.annotation.Nullable; import java.util.HashMap; import java.util.Map; diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java index 350cee3e09ca..5af34871dc4c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/PostAveragerAggregatorCalculator.java @@ -24,13 +24,9 @@ import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.groupby.GroupByQueryConfig; -import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; /** * Function that can be applied to a Sequence to calculate PostAverager results diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java index da390dd99733..fa614fa4218e 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucket.java @@ -22,7 +22,6 @@ import org.apache.druid.data.input.Row; import org.joda.time.DateTime; -import java.util.Collection; import java.util.List; /** diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 944112d8a1e6..549620855b3d 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -62,7 +62,6 @@ import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.RetryQueryRunnerConfig; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.movingaverage.test.TestConfig; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -82,7 +81,6 @@ import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; From 1c577ae9da2082d86df7997c977a219bc3ad8e6f Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Fri, 19 Apr 2019 13:09:30 -0700 Subject: [PATCH 41/45] Address review comments. --- .../query/movingaverage/MovingAverageQueryToolChest.java | 8 -------- .../druid/query/movingaverage/RowBucketIterable.java | 4 ++-- .../druid/query/movingaverage/averagers/BaseAverager.java | 6 ++++-- .../druid/query/movingaverage/MovingAverageQueryTest.java | 1 + .../src/test/resources/runtime.properties | 2 -- 5 files changed, 7 insertions(+), 14 deletions(-) delete mode 100644 extensions-contrib/moving-average-query/src/test/resources/runtime.properties diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java index 95bf7d4d0e26..b0e14affaf5b 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageQueryToolChest.java @@ -30,7 +30,6 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.query.movingaverage.averagers.AveragerFactory; @@ -47,7 +46,6 @@ public class MovingAverageQueryToolChest extends QueryToolChest walker, RequestL this.movingAverageQueryMetricsFactory = DefaultMovingAverageQueryMetricsFactory.instance(); } - @Inject(optional = true) - public void setWarehouse(QueryToolChestWarehouse warehouse) - { - this.warehouse = warehouse; - } - @Override public QueryRunner mergeResults(QueryRunner runner) { diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index 63831e0b5d38..a9a76ab05aa7 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -34,8 +34,8 @@ /** * An iterator which takes list of rows ({@link Sequence}) and generates a new list of {@link RowBucket}s from it. * - * It calls {@link BucketingAccumulator} for naive backeting to buckets of periods, - * But does more suttle logic to cover edge cases, such as: + * It calls {@link BucketingAccumulator} for naive bucketing to buckets of periods, + * But does more subtle logic to cover edge cases, such as: * - Handling periods with no rows. * - Handling last record. * diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java index 8ecef9c15c77..0c236b899ea2 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/averagers/BaseAverager.java @@ -42,8 +42,10 @@ public abstract class BaseAverager implements Averager final I[] buckets; private int index; - /* startFrom is needed because `buckets` field is a fixed array, not a list. - It makes computeResults() start from the correct bucket in the array. */ + /** + * {@link BaseAverager#startFrom} is needed because `buckets` field is a fixed array, not a list. + * It makes computeResults() start from the correct bucket in the array. + */ int startFrom = 0; /** diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 549620855b3d..a7e1eb738301 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -137,6 +137,7 @@ public MovingAverageQueryTest(String yamlFile) throws IOException ); System.setProperty("druid.generic.useDefaultValueForNull", "true"); + System.setProperty("druid.processing.buffer.sizeBytes", "655360"); Injector baseInjector = GuiceInjectors.makeStartupInjector(); injector = Initialization.makeInjectorWithModules(baseInjector, modules); diff --git a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties b/extensions-contrib/moving-average-query/src/test/resources/runtime.properties deleted file mode 100644 index e158009ccaa4..000000000000 --- a/extensions-contrib/moving-average-query/src/test/resources/runtime.properties +++ /dev/null @@ -1,2 +0,0 @@ -druid.processing.buffer.sizeBytes=655360 -druid.request.logging.type=slf4j From ab1ae00953ca33e99132fd6559c9b31ddfde97e6 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Fri, 19 Apr 2019 13:10:02 -0700 Subject: [PATCH 42/45] Rename fakeEvents to emptyEvents. --- .../movingaverage/MovingAverageIterable.java | 40 +++++++++---------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index faeda92b389a..a63c52e91c16 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -62,7 +62,7 @@ public class MovingAverageIterable implements Iterable private final List> factories; private final Map postAggMap; private final Map aggMap; - private final Map fakeEvents; + private final Map emptyEvents; public MovingAverageIterable( Sequence buckets, @@ -78,22 +78,22 @@ public MovingAverageIterable( postAggMap = postAggList.stream().collect(Collectors.toMap(postAgg -> postAgg.getName(), postAgg -> postAgg)); aggMap = aggList.stream().collect(Collectors.toMap(agg -> agg.getName(), agg -> agg)); - fakeEvents = generateFakeEventsFromAggregators(aggMap, postAggMap); + emptyEvents = generateEmptyEventsFromAggregators(aggMap, postAggMap); } - // Build a list of dummy events from Aggregators/PostAggregators to be used by Iterator to build fake rows. + // Build a list of empty events from Aggregators/PostAggregators to be used by Iterator to build fake rows. // These fake rows will be used by computeMovingAverage() in skip=true mode. - // See fakeEventsCopy in internalNext() and computeMovingAverage() documentation. - private Map generateFakeEventsFromAggregators(Map aggMap, - Map postAggMap) + // See emptyEventsCopy in internalNext() and computeMovingAverage() documentation. + private Map generateEmptyEventsFromAggregators(Map aggMap, + Map postAggMap) { - Map fakeEvents = new LinkedHashMap<>(); + Map emptyEvents = new LinkedHashMap<>(); aggMap.values().forEach(agg -> { Aggregator aggFactorized = agg.factorize(getEmptyColumnSelectorFactory()); - fakeEvents.put(agg.getName(), aggFactorized.get()); + emptyEvents.put(agg.getName(), aggFactorized.get()); }); - postAggMap.values().forEach(postAgg -> fakeEvents.put(postAgg.getName(), postAgg.compute(fakeEvents))); - return fakeEvents; + postAggMap.values().forEach(postAgg -> emptyEvents.put(postAgg.getName(), postAgg.compute(emptyEvents))); + return emptyEvents; } @Nonnull @@ -129,7 +129,7 @@ public ColumnCapabilities getColumnCapabilities(String s) @Override public Iterator iterator() { - return new MovingAverageIterator(seq, dims, factories, fakeEvents, aggMap); + return new MovingAverageIterator(seq, dims, factories, emptyEvents, aggMap); } static class MovingAverageIterator implements Iterator @@ -147,19 +147,19 @@ static class MovingAverageIterator implements Iterator private Set> seenKeys = new HashSet<>(); private Row saveNext; private Map aggMap; - private Map fakeEvents; + private Map emptyEvents; public MovingAverageIterator( Sequence rows, List dims, List> averagerFactories, - Map fakeEvents, + Map emptyEvents, Map aggMap ) { this.dims = dims; this.averagerFactories = averagerFactories; - this.fakeEvents = fakeEvents; + this.emptyEvents = emptyEvents; this.aggMap = aggMap; yielder = Yielders.each(rows); @@ -228,17 +228,15 @@ private Row internalNext() } } - // return fake rows for unseen dimension combinations + // return empty rows for unseen dimension combinations if (averagersKeysIter != null) { while (averagersKeysIter.hasNext()) { Map dims = averagersKeysIter.next(); - Map fakeEventsCopy = new HashMap<>(fakeEvents); + Map emptyEventsCopy = new HashMap<>(emptyEvents); - dims.forEach((dim, value) -> { - fakeEventsCopy.put(dim, value); - }); + dims.forEach((dim, value) -> emptyEventsCopy.put(dim, value)); - r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), fakeEventsCopy), true); + r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), emptyEventsCopy), true); if (r != null) { return r; } @@ -267,7 +265,7 @@ private Row internalNext() * decaying of the average values. * *

Usually, the contents of key will be contained by the row R being passed in, but in the case of a - * dummy row, its possible that the dimensions will be known but the row empty. Hence, the values are + * dummy row, it's possible that the dimensions will be known but the row empty. Hence, the values are * passed as two separate arguments. * * @param key The dimension set that this row applies to. From 7b6f56e32d0bd2c8865af3745fad4cd419e9f6e2 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Thu, 25 Apr 2019 12:35:19 -0700 Subject: [PATCH 43/45] Remove redundant parameter key from computeMovingAverage. --- .../query/movingaverage/MovingAverageIterable.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index a63c52e91c16..59000c1a8383 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -212,9 +212,10 @@ private Row internalNext() if (cacheIter != null) { if (cacheIter.hasNext()) { r = cacheIter.next(); + // Convert full event (key + metrics) to key Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); seenKeys.add(key); - r = computeMovingAverage(key, r, false); + r = computeMovingAverage((MapBasedRow) r, false); if (r != null) { return r; } else { @@ -234,9 +235,10 @@ private Row internalNext() Map dims = averagersKeysIter.next(); Map emptyEventsCopy = new HashMap<>(emptyEvents); + // Convert key to a full dummy event (key + dummy metrics). dims.forEach((dim, value) -> emptyEventsCopy.put(dim, value)); - r = computeMovingAverage(dims, new MapBasedRow(cache.getDateTime(), emptyEventsCopy), true); + r = computeMovingAverage(new MapBasedRow(cache.getDateTime(), emptyEventsCopy), true); if (r != null) { return r; } @@ -268,17 +270,17 @@ private Row internalNext() * dummy row, it's possible that the dimensions will be known but the row empty. Hence, the values are * passed as two separate arguments. * - * @param key The dimension set that this row applies to. * @param r The Row to operate on * @param skip Indicates whether skip or add should be called * * @return The updated row containing averager results, or null if no averagers computed a result */ @Nullable - private Row computeMovingAverage(Map key, Row r, boolean skip) + private Row computeMovingAverage(MapBasedRow r, boolean skip) { - Map event = ((MapBasedRow) r).getEvent(); + Map event = r.getEvent(); Map result = new HashMap<>(event); + Map key = MovingAverageHelper.getDimKeyFromRow(dims, r); List> avg = averagers.get(key); From 8fccf198a56efcf2f42e9e503ae9f75bc4a427ee Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Fri, 26 Apr 2019 10:29:44 -0700 Subject: [PATCH 44/45] Check yielder as well in RowBucketIterable#hasNext() --- .../org/apache/druid/query/movingaverage/RowBucketIterable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java index a9a76ab05aa7..308d5551c866 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/RowBucketIterable.java @@ -91,7 +91,7 @@ public RowBucketIterator(Sequence rows, List intervals, Period pe @Override public boolean hasNext() { - return expectedBucket.compareTo(endTime) < 0; + return expectedBucket.compareTo(endTime) < 0 || !this.yielder.isDone(); } /* (non-Javadoc) From 001b06179b9e89f5c615f6bd109c0daec012cc45 Mon Sep 17 00:00:00 2001 From: eyurman14 Date: Fri, 26 Apr 2019 10:38:10 -0700 Subject: [PATCH 45/45] Fix javadoc. --- .../apache/druid/query/movingaverage/MovingAverageIterable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java index 59000c1a8383..b92604d9f23c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/MovingAverageIterable.java @@ -52,7 +52,7 @@ * {@link MovingAverageIterable} iterates over days {@link RowBucket}, producing rows for each dimension combination, * filling in missing entries with "empty" rows so that the averaging buckets have enough data to operate on. * It then computes the moving average on the buckets and returns the row. - * See {@link MovingAverageIterator#computeMovingAverage(Map, Row, boolean)} for more details. + * See computeMovingAverage for more details. */ public class MovingAverageIterable implements Iterable {