Document shouldFinalize for sketches that have the parameter (#13524)

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
This commit is contained in:
Victoria Lim 2022-12-17 10:48:06 -08:00 committed by GitHub
parent e34e56295f
commit 09d8b16447
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 94 additions and 77 deletions

View File

@ -23,8 +23,9 @@ title: "DataSketches HLL Sketch module"
-->
This module provides Apache Druid aggregators for distinct counting based on HLL sketch from [Apache DataSketches](https://datasketches.apache.org/) library. At ingestion time, this aggregator creates the HLL sketch objects to be stored in Druid segments. At query time, sketches are read and merged together. In the end, by default, you receive the estimate of the number of distinct values presented to the sketch. Also, you can use post aggregator to produce a union of sketch columns in the same row.
You can use the HLL sketch aggregator on columns of any identifiers. It will return estimated cardinality of the column.
This module provides Apache Druid aggregators for distinct counting based on HLL sketch from [Apache DataSketches](https://datasketches.apache.org/) library. At ingestion time, this aggregator creates the HLL sketch objects to store in Druid segments. By default, Druid reads and merges sketches at query time. The default result is
the estimate of the number of distinct values presented to the sketch. You can also use post aggregators to produce a union of sketch columns in the same row.
You can use the HLL sketch aggregator on any column to estimate its cardinality.
To use this aggregator, make sure you [include](../../development/extensions.md#loading-extensions) the extension in your config file:
@ -32,20 +33,23 @@ To use this aggregator, make sure you [include](../../development/extensions.md#
druid.extensions.loadList=["druid-datasketches"]
```
### Aggregators
For additional sketch types supported in Druid, see [DataSketches extension](datasketches-extension.md).
|property|description|required?|
## Aggregators
|Property|Description|Required?|
|--------|-----------|---------|
|`type`|This String should be [`HLLSketchBuild`](#hllsketchbuild-aggregator) or [`HLLSketchMerge`](#hllsketchmerge-aggregator)|yes|
|`name`|A String for the output (result) name of the calculation.|yes|
|`fieldName`|A String for the name of the input field.|yes|
|`type`|Either [`HLLSketchBuild`](#hllsketchbuild-aggregator) or [`HLLSketchMerge`](#hllsketchmerge-aggregator).|yes|
|`name`|String representing the output column to store sketch values.|yes|
|`fieldName`|The name of the input field.|yes|
|`lgK`|log2 of K that is the number of buckets in the sketch, parameter that controls the size and the accuracy. Must be between 4 and 21 inclusively.|no, defaults to `12`|
|`tgtHllType`|The type of the target HLL sketch. Must be `HLL_4`, `HLL_6` or `HLL_8` |no, defaults to `HLL_4`|
|`round`|Round off values to whole numbers. Only affects query-time behavior and is ignored at ingestion-time.|no, defaults to `false`|
|`shouldFinalize`|Return the final double type representing the estimate rather than the intermediate sketch type itself. In addition to controlling the finalization of this aggregator, you can control whether all aggregators are finalized with the query context parameters [`finalize`](../../querying/query-context.md) and [`sqlFinalizeOuterSketches`](../../querying/sql-query-context.md).|no, defaults to `true`|
> The default `lgK` value has proven to be sufficient for most use cases; expect only very negligible improvements in accuracy with `lgK` values over `16` in normal circumstances.
#### HLLSketchBuild Aggregator
### HLLSketchBuild aggregator
```
{
@ -76,7 +80,7 @@ When applied at query time on an existing dimension, you can use the resulting c
> ```
>
#### HLLSketchMerge Aggregator
### HLLSketchMerge aggregator
```
{
@ -91,9 +95,9 @@ When applied at query time on an existing dimension, you can use the resulting c
You can use the `HLLSketchMerge` aggregator to ingest pre-generated sketches from an input dataset. For example, you can set up a batch processing job to generate the sketches before sending the data to Druid. You must serialize the sketches in the input dataset to Base64-encoded bytes. Then, specify `HLLSketchMerge` for the input column in the native ingestion `metricsSpec`.
### Post Aggregators
## Post aggregators
#### Estimate
### Estimate
Returns the distinct count estimate as a double.
@ -106,7 +110,7 @@ Returns the distinct count estimate as a double.
}
```
#### Estimate with bounds
### Estimate with bounds
Returns a distinct count estimate and error bounds from an HLL sketch.
The result will be an array containing three double values: estimate, lower bound and upper bound.
@ -122,7 +126,7 @@ This must be an integer value of 1, 2 or 3 corresponding to approximately 68.3%,
}
```
#### Union
### Union
```
{
@ -134,7 +138,7 @@ This must be an integer value of 1, 2 or 3 corresponding to approximately 68.3%,
}
```
#### Sketch to string
### Sketch to string
Human-readable sketch summary for debugging.

View File

@ -37,7 +37,9 @@ To use this aggregator, make sure you [include](../../development/extensions.md#
druid.extensions.loadList=["druid-datasketches"]
```
### Aggregator
For additional sketch types supported in Druid, see [DataSketches extension](datasketches-extension.md).
## Aggregator
The result of the aggregation is a KllFloatsSketch or KllDoublesSketch that is the union of all sketches either built from raw data or read from the segments.
@ -50,17 +52,17 @@ The result of the aggregation is a KllFloatsSketch or KllDoublesSketch that is t
}
```
|property|description|required?|
|Property|Description|Required?|
|--------|-----------|---------|
|type|This String should be "KllFloatsSketch" or "KllDoublesSketch"|yes|
|name|A String for the output (result) name of the calculation.|yes|
|fieldName|A String for the name of the input field (can contain sketches or raw numeric values).|yes|
|k|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be from 8 to 65535. See [KLL Sketch Accuracy and Size](https://datasketches.apache.org/docs/KLL/KLLAccuracyAndSize.html).|no, defaults to 200|
|maxStreamLength|This parameter defines the number of items that can be presented to each sketch before it may need to move from off-heap to on-heap memory. This is relevant to query types that use off-heap memory, including [TopN](../../querying/topnquery.md) and [GroupBy](../../querying/groupbyquery.md). Ideally, should be set high enough such that most sketches can stay off-heap.|no, defaults to 1000000000|
|`type`|Either "KllFloatsSketch" or "KllDoublesSketch"|yes|
|`name`|A String for the output (result) name of the calculation.|yes|
|`fieldName`|String for the name of the input field, which may contain sketches or raw numeric values.|yes|
|`k`|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be from 8 to 65535. See [KLL Sketch Accuracy and Size](https://datasketches.apache.org/docs/KLL/KLLAccuracyAndSize.html).|no, defaults to 200|
|`maxStreamLength`|This parameter defines the number of items that can be presented to each sketch before it may need to move from off-heap to on-heap memory. This is relevant to query types that use off-heap memory, including [TopN](../../querying/topnquery.md) and [GroupBy](../../querying/groupbyquery.md). Ideally, should be set high enough such that most sketches can stay off-heap.|no, defaults to 1000000000|
### Post Aggregators
## Post aggregators
#### Quantile
### Quantile
This returns an approximation to the value that would be preceded by a given fraction of a hypothetical sorted version of the input stream.
@ -73,7 +75,7 @@ This returns an approximation to the value that would be preceded by a given fra
}
```
#### Quantiles
### Quantiles
This returns an array of quantiles corresponding to a given array of fractions
@ -86,7 +88,7 @@ This returns an array of quantiles corresponding to a given array of fractions
}
```
#### Histogram
### Histogram
This returns an approximation to the histogram given an array of split points that define the histogram bins or a number of bins (not both). An array of <i>m</i> unique, monotonically increasing split points divide the real number line into <i>m+1</i> consecutive disjoint intervals. The definition of an interval is inclusive of the left split point and exclusive of the right split point. If the number of bins is specified instead of split points, the interval between the minimum and maximum values is divided into the given number of equally-spaced bins.
@ -100,7 +102,7 @@ This returns an approximation to the histogram given an array of split points th
}
```
#### Rank
### Rank
This returns an approximation to the rank of a given value that is the fraction of the distribution less than that value.
@ -112,7 +114,7 @@ This returns an approximation to the rank of a given value that is the fraction
"value" : <value>
}
```
#### CDF
### CDF
This returns an approximation to the Cumulative Distribution Function given an array of split points that define the edges of the bins. An array of <i>m</i> unique, monotonically increasing split points divide the real number line into <i>m+1</i> consecutive disjoint intervals. The definition of an interval is inclusive of the left split point and exclusive of the right split point. The resulting array of fractions can be viewed as ranks of each split point with one additional rank that is always 1.
@ -125,7 +127,7 @@ This returns an approximation to the Cumulative Distribution Function given an a
}
```
#### Sketch Summary
### Sketch Summary
This returns a summary of the sketch that can be used for debugging. This is the result of calling toString() method.

View File

@ -37,7 +37,9 @@ To use this aggregator, make sure you [include](../../development/extensions.md#
druid.extensions.loadList=["druid-datasketches"]
```
### Aggregator
For additional sketch types supported in Druid, see [DataSketches extension](datasketches-extension.md).
## Aggregator
The result of the aggregation is a DoublesSketch that is the union of all sketches either built from raw data or read from the segments.
@ -50,17 +52,18 @@ The result of the aggregation is a DoublesSketch that is the union of all sketch
}
```
|property|description|required?|
|Property|Description|Required?|
|--------|-----------|---------|
|type|This String should always be "quantilesDoublesSketch"|yes|
|name|A String for the output (result) name of the calculation.|yes|
|fieldName|A String for the name of the input field (can contain sketches or raw numeric values).|yes|
|k|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2 from 2 to 32768. See [accuracy information](https://datasketches.apache.org/docs/Quantiles/OrigQuantilesSketch) in the DataSketches documentation for details.|no, defaults to 128|
|maxStreamLength|This parameter defines the number of items that can be presented to each sketch before it may need to move from off-heap to on-heap memory. This is relevant to query types that use off-heap memory, including [TopN](../../querying/topnquery.md) and [GroupBy](../../querying/groupbyquery.md). Ideally, should be set high enough such that most sketches can stay off-heap.|no, defaults to 1000000000|
|`type`|This string should always be "quantilesDoublesSketch"|yes|
|`name`|String representing the output column to store sketch values.|yes|
|`fieldName`|A string for the name of the input field (can contain sketches or raw numeric values).|yes|
|`k`|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2 from 2 to 32768. See [accuracy information](https://datasketches.apache.org/docs/Quantiles/OrigQuantilesSketch) in the DataSketches documentation for details.|no, defaults to 128|
|`maxStreamLength`|This parameter defines the number of items that can be presented to each sketch before it may need to move from off-heap to on-heap memory. This is relevant to query types that use off-heap memory, including [TopN](../../querying/topnquery.md) and [GroupBy](../../querying/groupbyquery.md). Ideally, should be set high enough such that most sketches can stay off-heap.|no, defaults to 1000000000|
|`shouldFinalize`|Return the final double type representing the estimate rather than the intermediate sketch type itself. In addition to controlling the finalization of this aggregator, you can control whether all aggregators are finalized with the query context parameters [`finalize`](../../querying/query-context.md) and [`sqlFinalizeOuterSketches`](../../querying/sql-query-context.md).|no, defaults to `true`|
### Post Aggregators
## Post aggregators
#### Quantile
### Quantile
This returns an approximation to the value that would be preceded by a given fraction of a hypothetical sorted version of the input stream.
@ -73,7 +76,7 @@ This returns an approximation to the value that would be preceded by a given fra
}
```
#### Quantiles
### Quantiles
This returns an array of quantiles corresponding to a given array of fractions
@ -86,7 +89,7 @@ This returns an array of quantiles corresponding to a given array of fractions
}
```
#### Histogram
### Histogram
This returns an approximation to the histogram given an array of split points that define the histogram bins or a number of bins (not both). An array of <i>m</i> unique, monotonically increasing split points divide the real number line into <i>m+1</i> consecutive disjoint intervals. The definition of an interval is inclusive of the left split point and exclusive of the right split point. If the number of bins is specified instead of split points, the interval between the minimum and maximum values is divided into the given number of equally-spaced bins.
@ -100,7 +103,7 @@ This returns an approximation to the histogram given an array of split points th
}
```
#### Rank
### Rank
This returns an approximation to the rank of a given value that is the fraction of the distribution less than that value.
@ -112,7 +115,7 @@ This returns an approximation to the rank of a given value that is the fraction
"value" : <value>
}
```
#### CDF
### CDF
This returns an approximation to the Cumulative Distribution Function given an array of split points that define the edges of the bins. An array of <i>m</i> unique, monotonically increasing split points divide the real number line into <i>m+1</i> consecutive disjoint intervals. The definition of an interval is inclusive of the left split point and exclusive of the right split point. The resulting array of fractions can be viewed as ranks of each split point with one additional rank that is always 1.
@ -125,7 +128,7 @@ This returns an approximation to the Cumulative Distribution Function given an a
}
```
#### Sketch Summary
### Sketch summary
This returns a summary of the sketch that can be used for debugging. This is the result of calling toString() method.

View File

@ -23,8 +23,11 @@ title: "DataSketches Theta Sketch module"
-->
This module provides Apache Druid aggregators based on Theta sketch from [Apache DataSketches](https://datasketches.apache.org/) library. Note that sketch algorithms are approximate; see details in the "Accuracy" section of the datasketches doc.
At ingestion time, this aggregator creates the Theta sketch objects which get stored in Druid segments. Logically speaking, a Theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. Also, you can use post aggregators to do union, intersection or difference on sketch columns in the same row.
This module provides Apache Druid aggregators based on Theta sketch from [Apache DataSketches](https://datasketches.apache.org/) library.
Sketch algorithms are approximate. For more information, see [Accuracy](https://datasketches.apache.org/docs/Theta/ThetaAccuracy.html) in the DataSketches documentation.
At ingestion time, the Theta sketch aggregator creates Theta sketch objects which are stored in Druid segments. Logically speaking, a Theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated (set unioned) together. In the end, by default, you receive the estimate of the number of unique entries in the sketch object. You can use post aggregators to do union, intersection or difference on sketch columns in the same row.
Note that you can use `thetaSketch` aggregator on columns which were not ingested using the same. It will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster.
To use this aggregator, make sure you [include](../../development/extensions.md#loading-extensions) the extension in your config file:
@ -33,7 +36,9 @@ To use this aggregator, make sure you [include](../../development/extensions.md#
druid.extensions.loadList=["druid-datasketches"]
```
### Aggregators
For additional sketch types supported in Druid, see [DataSketches extension](datasketches-extension.md).
## Aggregator
```json
{
@ -45,17 +50,18 @@ druid.extensions.loadList=["druid-datasketches"]
}
```
|property|description|required?|
|Property|Description|Required?|
|--------|-----------|---------|
|type|This String should always be "thetaSketch"|yes|
|name|A String for the output (result) name of the calculation.|yes|
|fieldName|A String for the name of the aggregator used at ingestion time.|yes|
|isInputThetaSketch|This should only be used at indexing time if your input data contains theta sketch objects. This would be the case if you use datasketches library outside of Druid, say with Pig/Hive, to produce the data that you are ingesting into Druid |no, defaults to false|
|size|Must be a power of 2. Internally, size refers to the maximum number of entries sketch object will retain. Higher size means higher accuracy but more space to store sketches. Note that after you index with a particular size, druid will persist sketch in segments and you will use size greater or equal to that at query time. See the [DataSketches site](https://datasketches.apache.org/docs/Theta/ThetaSize) for details. In general, We recommend just sticking to default size. |no, defaults to 16384|
|`type`|This string should always be "thetaSketch"|yes|
|`name`|String representing the output column to store sketch values.|yes|
|`fieldName`|A string for the name of the aggregator used at ingestion time.|yes|
|`isInputThetaSketch`|Only set this to true at indexing time if your input data contains Theta sketch objects. This applies to cases when you use DataSketches outside of Druid, for example with Pig or Hive, to produce the data to ingest into Druid |no, defaults to false|
|`size`|Must be a power of 2. Internally, size refers to the maximum number of entries sketch object retains. Higher size means higher accuracy but more space to store sketches. After you index with a particular size, Druid persists the sketch in segments. At query time you must use a size greater or equal to the ingested size. See the [DataSketches site](https://datasketches.apache.org/docs/Theta/ThetaSize) for details. The default is recommended for the majority of use cases.|no, defaults to 16384|
|`shouldFinalize`|Return the final double type representing the estimate rather than the intermediate sketch type itself. In addition to controlling the finalization of this aggregator, you can control whether all aggregators are finalized with the query context parameters [`finalize`](../../querying/query-context.md) and [`sqlFinalizeOuterSketches`](../../querying/sql-query-context.md).|no, defaults to `true`|
### Post Aggregators
## Post aggregators
#### Sketch Estimator
### Sketch estimator
```json
{
@ -65,7 +71,7 @@ druid.extensions.loadList=["druid-datasketches"]
}
```
#### Sketch Operations
### Sketch operations
```json
{
@ -77,7 +83,7 @@ druid.extensions.loadList=["druid-datasketches"]
}
```
#### Sketch Summary
### Sketch summary
This returns a summary of the sketch that can be used for debugging. This is the result of calling toString() method.
@ -89,7 +95,7 @@ This returns a summary of the sketch that can be used for debugging. This is the
}
```
### Examples
## Examples
Assuming, you have a dataset containing (timestamp, product, user_id). You want to answer questions like
@ -185,7 +191,7 @@ sample query for, How many unique users visited both product A and B?
}
```
#### Retention Analysis Example
### Retention analysis example
Suppose you want to answer a question like, "How many unique users performed a specific action in a particular time period and also performed another specific action in a different time period?"

View File

@ -31,7 +31,9 @@ To use this aggregator, make sure you [include](../../development/extensions.md#
druid.extensions.loadList=["druid-datasketches"]
```
### Aggregators
For additional sketch types supported in Druid, see [DataSketches extension](datasketches-extension.md).
## Aggregator
```json
{
@ -44,21 +46,21 @@ druid.extensions.loadList=["druid-datasketches"]
}
```
|property|description|required?|
|Property|Description|Required?|
|--------|-----------|---------|
|type|This String should always be "arrayOfDoublesSketch"|yes|
|name|String representing the output column to store sketch values.|yes|
|fieldName|A String for the name of the input field.|yes|
|nominalEntries|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2. See the [Theta sketch accuracy](https://datasketches.apache.org/docs/Theta/ThetaErrorTable) for details. |no, defaults to 16384|
|metricColumns|When building sketches from raw data, an array input column that contain numeric values to associate with each distinct key. If not provided, assumes `fieldName` is an `arrayOfDoublesSketch`|no, if not provided `fieldName` is assumed to be an arrayOfDoublesSketch|
|numberOfValues|Number of values associated with each distinct key. |no, defaults to the length of `metricColumns` if provided and 1 otherwise|
|`type`|This string should always be "arrayOfDoublesSketch"|yes|
|`name`|String representing the output column to store sketch values.|yes|
|`fieldName`|A string for the name of the input field.|yes|
|`nominalEntries`|Parameter that determines the accuracy and size of the sketch. Higher k means higher accuracy but more space to store sketches. Must be a power of 2. See the [Theta sketch accuracy](https://datasketches.apache.org/docs/Theta/ThetaErrorTable) for details. |no, defaults to 16384|
|`metricColumns`|When building sketches from raw data, an array input column that contain numeric values to associate with each distinct key. If not provided, assumes `fieldName` is an `arrayOfDoublesSketch`|no, if not provided `fieldName` is assumed to be an arrayOfDoublesSketch|
|`numberOfValues`|Number of values associated with each distinct key. |no, defaults to the length of `metricColumns` if provided and 1 otherwise|
You can use the `arrayOfDoublesSketch` aggregator to:
- Build a sketch from raw data. In this case, set `metricColumns` to an array.
- Build a sketch from an existing `ArrayOfDoubles` sketch . In this case, leave `metricColumns` unset and set the `fieldName` to an `ArrayOfDoubles` sketch with `numberOfValues` doubles. At ingestion time, you must base64 encode `ArrayOfDoubles` sketches at ingestion time.
#### Example on top of raw data
### Example on top of raw data
Compute a theta of unique users. For each user store the `added` and `deleted` scores. The new sketch column will be called `users_theta`.
@ -72,7 +74,7 @@ Compute a theta of unique users. For each user store the `added` and `deleted` s
}
```
#### Example ingesting a precomputed sketch column
### Example ingesting a precomputed sketch column
Ingest a sketch column called `user_sketches` that has a base64 encoded value of two doubles in its array and store it in a column called `users_theta`.
@ -86,9 +88,9 @@ Ingest a sketch column called `user_sketches` that has a base64 encoded value of
}
```
### Post Aggregators
## Post aggregators
#### Estimate of the number of distinct keys
### Estimate of the number of distinct keys
Returns a distinct count estimate from a given ArrayOfDoublesSketch.
@ -100,7 +102,7 @@ Returns a distinct count estimate from a given ArrayOfDoublesSketch.
}
```
#### Estimate of the number of distinct keys with error bounds
### Estimate of the number of distinct keys with error bounds
Returns a distinct count estimate and error bounds from a given ArrayOfDoublesSketch. The result will be three double values: estimate of the number of distinct keys, lower bound and upper bound. The bounds are provided at the given number of standard deviations (optional, defaults to 1). This must be an integer value of 1, 2 or 3 corresponding to approximately 68.3%, 95.4% and 99.7% confidence intervals.
@ -113,7 +115,7 @@ Returns a distinct count estimate and error bounds from a given ArrayOfDoublesSk
}
```
#### Number of retained entries
### Number of retained entries
Returns the number of retained entries from a given ArrayOfDoublesSketch.
@ -125,7 +127,7 @@ Returns the number of retained entries from a given ArrayOfDoublesSketch.
}
```
#### Mean values for each column
### Mean values for each column
Returns a list of mean values from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.
@ -137,7 +139,7 @@ Returns a list of mean values from a given ArrayOfDoublesSketch. The result will
}
```
#### Variance values for each column
### Variance values for each column
Returns a list of variance values from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.
@ -149,7 +151,7 @@ Returns a list of variance values from a given ArrayOfDoublesSketch. The result
}
```
#### Quantiles sketch from a column
### Quantiles sketch from a column
Returns a quantiles DoublesSketch constructed from a given column of values from a given ArrayOfDoublesSketch using optional parameter k that determines the accuracy and size of the quantiles sketch. See [Quantiles Sketch Module](datasketches-quantiles.md)
@ -167,7 +169,7 @@ Returns a quantiles DoublesSketch constructed from a given column of values from
}
```
#### Set Operations
### Set operations
Returns a result of a specified set operation on the given array of sketches. Supported operations are: union, intersection and set difference (UNION, INTERSECT, NOT).
@ -182,7 +184,7 @@ Returns a result of a specified set operation on the given array of sketches. Su
}
```
#### Student's t-test
### Student's t-test
Performs Student's t-test and returns a list of p-values given two instances of ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key. See [t-test documentation](https://commons.apache.org/proper/commons-math/javadocs/api-3.6.1/org/apache/commons/math3/stat/inference/TTest.html).
@ -194,7 +196,7 @@ Performs Student's t-test and returns a list of p-values given two instances of
}
```
#### Sketch summary
### Sketch summary
Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a string returned by toString() method of the sketch. This can be useful for debugging.