| id | druid-exact-count-bitmap |
|---|---|
| title | Exact Count Bitmap |
This extension provides exact cardinality counting functionality for LONG type columns using Roaring Bitmaps. Unlike approximate cardinality aggregators like HyperLogLog, this aggregator provides precise distinct counts.
To use this Apache Druid extension, include druid-exact-count-bitmap in the extensions load list.
The Distinct Count Aggregator works in a similar way to the Exact Count Aggregator. Hence, it is important to understand the difference between the behavior of these two aggregators.
| Exact Count | Distinct Count |
|---|---|
| No prerequisites needed (e.g. configuring hash partition, segment granularity) | Prerequisites needed to perform aggregation |
| Works on 64-bit number columns only (BIGINT) | Works on dimension columns (Including Strings, Complex Types, etc) |
The extension uses Roaring64NavigableMap as its underlying data structure to efficiently store and compute exact cardinality of 64-bit integers. It provides two types of aggregators that serve different purposes:
The BUILD aggregator is used when you want to compute cardinality directly from raw LONG values:
- Used during ingestion or when querying raw data
- Must be used on columns of type LONG.
Example:
{
"type": "Bitmap64ExactCountBuild",
"name": "unique_values",
"fieldName": "id"
}The MERGE aggregator is used when working with pre-computed bitmaps:
- Used for querying pre-aggregated data (columns that were previously aggregated using BUILD)
- Combines multiple bitmaps using bitwise operations.
- Must be used on columns that are aggregated using BUILD, or by a previous MERGE.
Bitmap64ExactCountMergeaggregator is recommended for use intimeseriestype queries, though it also works fortopNandgroupByqueries.
Example:
{
"type": "Bitmap64ExactCountMerge",
"name": "total_unique_values",
"fieldName": "unique_values" // Must be a pre-computed bitmap
}-
During ingestion, use BUILD to create the initial bitmap:
{ "type": "index", "spec": { "dataSchema": { "metricsSpec": [ { "type": "Bitmap64ExactCountBuild", "name": "unique_users", "fieldName": "user_id" } ] } } } -
When querying the aggregated data, use MERGE to combine bitmaps:
{ "queryType": "timeseries", "aggregations": [ { "type": "Bitmap64ExactCountMerge", "name": "total_unique_users", "fieldName": "unique_users" } ] }
You can use the BITMAP64_EXACT_COUNT function in SQL queries:
SELECT BITMAP64_EXACT_COUNT(column_name)
FROM datasource
WHERE ...
GROUP BY ...You can also use the post-aggregator for further processing:
{
"type": "bitmap64ExactCount",
"name": "<output_name>",
"fieldName": "<aggregator_name>"
}- Memory Usage: While Roaring Bitmaps are efficient, storing exact unique values will generally consume more memory than approximate algorithms like HyperLogLog.
- Input Type: This aggregator only works with LONG (64-bit integer) columns. String or other data types must be converted to longs before using this aggregator.
- Build vs Merge: Always use BUILD for raw numeric data and MERGE for pre-aggregated data. Using BUILD on pre-aggregated data or MERGE on raw data will not work correctly.
- User Analytics: Count unique users over time
-- First ingest with BUILD aggregator
-- Then query with:
SELECT
TIME_FLOOR(__time, 'PT1H') AS hour,
BITMAP64_EXACT_COUNT(unique_users) as distinct_users
FROM user_metrics
GROUP BY 1- High-Precision Metrics: When exact counts are required
{
"type": "groupBy",
"dimensions": [
"country"
],
"aggregations": [
{
"type": "Bitmap64ExactCountMerge",
"name": "exact_user_count",
"fieldName": "unique_users"
}
]
}{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "wikipedia_metrics",
"timestampSpec": {
"column": "__time",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"channel",
"namespace",
"page",
"user",
"cityName",
"countryName",
"regionName",
"isRobot",
"isUnpatrolled",
"isNew",
"isAnonymous"
]
},
"metricsSpec": [
{
"type": "Bitmap64ExactCountBuild",
"name": "unique_added_values",
"fieldName": "added"
},
{
"type": "Bitmap64ExactCountBuild",
"name": "unique_delta_values",
"fieldName": "delta"
},
{
"type": "Bitmap64ExactCountBuild",
"name": "unique_comment_lengths",
"fieldName": "commentLength"
},
{
"name": "count",
"type": "count"
},
{
"name": "sum_added",
"type": "longSum",
"fieldName": "added"
},
{
"name": "sum_delta",
"type": "longSum",
"fieldName": "delta"
}
],
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "DAY",
"queryGranularity": "HOUR",
"rollup": true,
"intervals": [
"2016-06-27/2016-06-28"
]
}
},
"ioConfig": {
"type": "index",
"inputSource": {
"type": "druid",
"dataSource": "wikipedia",
"interval": "2016-06-27/2016-06-28"
},
"inputFormat": {
"type": "tsv",
"findColumnsFromHeader": true
}
},
"tuningConfig": {
"type": "index",
"maxRowsPerSegment": 5000000,
"maxRowsInMemory": 25000
}
}
}{
"queryType": "timeseries",
"dataSource": {
"type": "table",
"name": "wikipedia_metrics"
},
"intervals": {
"type": "intervals",
"intervals": [
"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
]
},
"granularity": {
"type": "all"
},
"aggregations": [
{
"type": "Bitmap64ExactCountBuild",
"name": "a0",
"fieldName": "unique_added_values"
}
]
}{
"queryType": "timeseries",
"dataSource": {
"type": "table",
"name": "wikipedia_metrics"
},
"intervals": {
"type": "intervals",
"intervals": [
"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
]
},
"granularity": {
"type": "all"
},
"aggregations": [
{
"type": "Bitmap64ExactCountMerge",
"name": "a0",
"fieldName": "unique_added_values"
}
]
}{
"type": "kafka",
"spec": {
"dataSchema": {
"dataSource": "ticker_event_bitmap64_exact_count_rollup",
"timestampSpec": {
"column": "timestamp",
"format": "millis",
"missingValue": null
},
"dimensionsSpec": {
"dimensions": [
{
"type": "string",
"name": "key"
}
],
"dimensionExclusions": []
},
"metricsSpec": [
{
"type": "Bitmap64ExactCountBuild",
"name": "count",
"fieldName": "value"
}
],
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "HOUR",
"queryGranularity": "HOUR",
"rollup": true,
"intervals": null
},
"transformSpec": {
"filter": null,
"transforms": []
}
},
"ioConfig": {
"topic": "ticker_event",
"inputFormat": {
"type": "json",
"flattenSpec": {
"useFieldDiscovery": true,
"fields": []
},
"featureSpec": {}
},
"replicas": 1,
"taskCount": 1,
"taskDuration": "PT3600S",
"consumerProperties": {
"bootstrap.servers": "localhost:9092"
},
"pollTimeout": 100,
"startDelay": "PT5S",
"period": "PT30S",
"useEarliestOffset": false,
"completionTimeout": "PT1800S",
"lateMessageRejectionPeriod": null,
"earlyMessageRejectionPeriod": null,
"lateMessageRejectionStartDateTime": null,
"stream": "ticker_event",
"useEarliestSequenceNumber": false,
"type": "kafka"
}
}
}{
"queryType": "timeseries",
"dataSource": {
"type": "table",
"name": "ticker_event_bitmap64_exact_count_rollup"
},
"intervals": {
"type": "intervals",
"intervals": [
"2020-09-13T06:35:35.000Z/146140482-04-24T15:36:27.903Z"
]
},
"descending": false,
"virtualColumns": [],
"filter": null,
"granularity": {
"type": "all"
},
"aggregations": [
{
"type": "count",
"name": "cnt"
},
{
"type": "Bitmap64ExactCountMerge",
"name": "a0",
"fieldName": "count"
}
],
"postAggregations": [
{
"type": "arithmetic",
"fn": "/",
"fields": [
{
"type": "bitmap64ExactCount",
"name": "a0",
"fieldName": "a0"
},
{
"type": "fieldAccess",
"name": "cnt",
"fieldName": "cnt"
}
],
"name": "rollup_rate"
}
],
"limit": 2147483647
}