Note that queries with first/last aggregators on a segment created with rollup enabled will return the rolled up value, and not the last value within the raw ingested data.
#### `doubleFirst` aggregator
`doubleFirst` computes the metric value with the minimum timestamp or 0 if no row exist
```json
{
"type" : "doubleFirst",
"name" : <output_name>,
"fieldName" : <metric_name>
}
```
#### `doubleLast` aggregator
`doubleLast` computes the metric value with the maximum timestamp or 0 if no row exist
JavaScript-based functionality is disabled by default. Please refer to the Druid <ahref="../development/javascript.html">JavaScript programming guide</a> for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
The [DataSketches Theta Sketch](../development/extensions-core/datasketches-theta.html) extension-provided aggregator gives distinct count estimates with support for set union, intersection, and difference post-aggregators, using Theta sketches from the [datasketches](http://datasketches.github.io/) library.
The [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.html) extension-provided aggregator gives distinct count estimates using the HyperLogLog algorithm.
Compared to the Theta sketch, the HLL sketch does not support set operations and has slightly slower update and merge speed, but requires significantly less space.
The Cardinality and HyperUnique aggregators are deprecated. Please use <ahref="../development/extensions-core/datasketches-theta.html">DataSketches Theta Sketch</a> or <ahref="../development/extensions-core/datasketches-hll.html">DataSketches HLL Sketch</a> instead.
The [Cardinality and HyperUnique](../querying/hll-old.html) aggregators are older aggregator implementations available by default in Druid that also provide distinct count estimates using the HyperLogLog algorithm. The newer DataSketches Theta and HLL extension-provided aggregators described above have superior accuracy and performance and are recommended instead.
The DataSketches team has published a [comparison study](https://datasketches.github.io/docs/HLL/HllSketchVsDruidHyperLogLogCollector.html) between Druid's original HLL algorithm and the DataSketches HLL algorithm. Based on the demonstrated advantages of the DataSketches implementation, we have deprecated Druid's original HLL aggregator.
Please note that `hyperUnique` aggregators are not mutually compatible with Datasketches HLL or Theta sketches.
##### Multi-column handling
Note the DataSketches Theta and HLL aggregators currently only support single-column inputs. If you were previously using the Cardinality aggregator with multiple-column inputs, equivalent operations using Theta or HLL sketches are described below:
* Multi-column `byValue` Cardinality can be replaced with a union of Theta sketches on the individual input columns
* Multi-column `byRow` Cardinality can be replaced with a Theta or HLL sketch on a single [virtual column]((../querying/virtual-columns.html) that combines the individual input columns.
The [DataSketches Quantiles Sketch](../development/extensions-core/datasketches-quantiles.html) extension-provided aggregator provides quantile estimates and histogram approximations using the numeric quantiles DoublesSketch from the [datasketches](http://datasketches.github.io/) library.
We recommend this aggregator in general for quantiles/histogram use cases, as it provides formal error bounds and has distribution-independent accuracy.
#### Moments Sketch (Experimental)
The [Moments Sketch](../development/extensions-contrib/momentsketch-quantiles.html) extension-provided aggregator is an experimental aggregator that provides quantile estimates using the [Moments Sketch](https://github.com/stanford-futuredata/momentsketch).
The Moments Sketch aggregator is provided as an experimental option. It is optimized for merging speed and it can have higher aggregation performance compared to the DataSketches quantiles aggregator. However, the accuracy of the Moments Sketch is distribution-dependent, so users will need to empirically verify that the aggregator is suitable for their input data.
As a general guideline for experimentation, the [Moments Sketch paper](https://arxiv.org/pdf/1803.01969.pdf) points out that this algorithm works better on inputs with high entropy. In particular, the algorithm is not a good fit when the input data consists of a small number of clustered discrete values.
#### Fixed Buckets Histogram
Druid also provides a [simple histogram implementation]((../development/extensions-core/approxiate-histograms.html#fixed-buckets-histogram) that uses a fixed range and fixed number of buckets with support for quantile estimation, backed by an array of bucket count values.
The fixed buckets histogram can perform well when the distribution of the input data allows a small number of buckets to be used.
We do not recommend the fixed buckets histogram for general use, as its usefulness is extremely data dependent. However, it is made available for users that have already identified use cases where a fixed buckets histogram is suitable.
The [Approximate Histogram](../development/extensions-core/approximate-histograms.html) extension-provided aggregator also provides quantile estimates and histogram approximations, based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf).
The algorithm used by this deprecated aggregator is highly distribution-dependent and its output is subject to serious distortions when the input does not fit within the algorithm's limitations.
A [study published by the DataSketches team](https://datasketches.github.io/docs/Quantiles/DruidApproxHistogramStudy.html) demonstrates some of the known failure modes of this algorithm:
- The algorithm's quantile calculations can fail to provide results for a large range of rank values (all ranks less than 0.89 in the example used in the study), returning all zeroes instead.
- The algorithm can completely fail to record spikes in the tail ends of the distribution
- In general, the histogram produced by the algorithm can deviate significantly from the true histogram, with no bounds on the errors.
It is not possible to determine a priori how well this aggregator will behave for a given input stream, nor does the aggregator provide any indication that serious distortions are present in the output.
For these reasons, we have deprecated this aggregator and do not recommend its use.
A filtered aggregator wraps any given aggregator, but only aggregates the values for which the given dimension filter matches.
This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations.
*Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data.