diff --git a/.github/workflows/revised-its.yml b/.github/workflows/revised-its.yml index 5fd1ff1c9026..14218e47baed 100644 --- a/.github/workflows/revised-its.yml +++ b/.github/workflows/revised-its.yml @@ -67,7 +67,7 @@ jobs: fail-fast: false matrix: jdk: [17] - it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query] + it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query, DruidExactCountBitmap] indexer: [middleManager] uses: ./.github/workflows/reusable-revised-its.yml if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }} diff --git a/distribution/pom.xml b/distribution/pom.xml index 58ab1ac1fff2..ea2be179b325 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -462,6 +462,8 @@ org.apache.druid.extensions.contrib:grpc-query -c org.apache.druid.extensions.contrib:druid-ranger-security + -c + org.apache.druid.extensions.contrib:druid-exact-count-bitmap diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md index fb040188ba38..ae8d5987d24e 100644 --- a/docs/configuration/extensions.md +++ b/docs/configuration/extensions.md @@ -86,6 +86,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati |druid-ddsketch|Support for DDSketch approximate quantiles based on [DDSketch](https://github.com/datadog/sketches-java) | [link](../development/extensions-contrib/ddsketch-quantiles.md)| |druid-deltalake-extensions|Support for ingesting Delta Lake tables.|[link](../development/extensions-contrib/delta-lake.md)| |druid-distinctcount|DistinctCount aggregator|[link](../development/extensions-contrib/distinctcount.md)| +|druid-exact-count-bitmap|Support for exact cardinality counting using Roaring Bitmap over a Long column.|[link](../development/extensions-contrib/druid-exact-count-bitmap.md)| |druid-iceberg-extensions|Support for ingesting Iceberg tables.|[link](../development/extensions-contrib/iceberg.md)| |druid-redis-cache|A cache implementation for Druid based on Redis.|[link](../development/extensions-contrib/redis-cache.md)| |druid-time-min-max|Min/Max aggregator for timestamp.|[link](../development/extensions-contrib/time-min-max.md)| diff --git a/docs/development/extensions-contrib/druid-exact-count-bitmap.md b/docs/development/extensions-contrib/druid-exact-count-bitmap.md new file mode 100644 index 000000000000..b39ed38dd65a --- /dev/null +++ b/docs/development/extensions-contrib/druid-exact-count-bitmap.md @@ -0,0 +1,452 @@ +--- +id: druid-exact-count-bitmap +title: "Exact Count Bitmap" +--- + + + +This extension provides exact cardinality counting functionality for LONG type columns using [Roaring Bitmaps](https://roaringbitmap.org/). Unlike approximate cardinality aggregators like HyperLogLog, this aggregator provides precise distinct counts. + +## Installation + +To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-exact-count-bitmap` in the extensions load list. + +## Comparison with Similar Aggregations + +The [Distinct Count Aggregator](https://druid.apache.org/docs/latest/development/extensions-contrib/distinctcount/) 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) | + +## How it Works + +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: + +### Build Aggregator (Bitmap64ExactCountBuild) + +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: + +```json +{ + "type": "Bitmap64ExactCountBuild", + "name": "unique_values", + "fieldName": "id" +} +``` + +### Merge Aggregator (Bitmap64ExactCountMerge) + +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. +- `Bitmap64ExactCountMerge` aggregator is recommended for use in `timeseries` type queries, though it also works for `topN` and `groupBy` queries. + +Example: + +```json +{ + "type": "Bitmap64ExactCountMerge", + "name": "total_unique_values", + "fieldName": "unique_values" // Must be a pre-computed bitmap +} +``` + +### Typical Workflow + +1. During ingestion, use BUILD to create the initial bitmap: + ```json + { + "type": "index", + "spec": { + "dataSchema": { + "metricsSpec": [ + { + "type": "Bitmap64ExactCountBuild", + "name": "unique_users", + "fieldName": "user_id" + } + ] + } + } + } + ``` + +2. When querying the aggregated data, use MERGE to combine bitmaps: + ```json + { + "queryType": "timeseries", + "aggregations": [ + { + "type": "Bitmap64ExactCountMerge", + "name": "total_unique_users", + "fieldName": "unique_users" + } + ] + } + ``` + +## Usage + +### SQL Query + +You can use the `BITMAP64_EXACT_COUNT` function in SQL queries: + +```sql +SELECT BITMAP64_EXACT_COUNT(column_name) +FROM datasource +WHERE ... +GROUP BY ... +``` + +### Post-Aggregator + +You can also use the post-aggregator for further processing: + +```json +{ + "type": "bitmap64ExactCount", + "name": "", + "fieldName": "" +} +``` + +## Considerations + +- **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. + +## Example Use Cases + +1. **User Analytics**: Count unique users over time + +```sql +-- 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 +``` + +2. **High-Precision Metrics**: When exact counts are required + +```json +{ + "type": "groupBy", + "dimensions": [ + "country" + ], + "aggregations": [ + { + "type": "Bitmap64ExactCountMerge", + "name": "exact_user_count", + "fieldName": "unique_users" + } + ] +} +``` + +## Walkthrough Using Wikipedia datasource + +### Batch Ingestion Task Spec + +```json +{ + "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 + } + } +} +``` + +### Query from datasource with raw bytes + +```json +{ + "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" + } + ] +} +``` + +### Query from datasource with pre-aggregated bitmap + +```json +{ + "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" + } + ] +} +``` + +## Other Examples + +### Kafka ingestion task spec + +```json +{ + "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" + } + } +} +``` + +### Query with Post-aggregator: + +```json +{ + "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 +} +``` diff --git a/extensions-contrib/druid-exact-count-bitmap/pom.xml b/extensions-contrib/druid-exact-count-bitmap/pom.xml new file mode 100644 index 000000000000..711f47d27a68 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/pom.xml @@ -0,0 +1,147 @@ + + + + + 4.0.0 + + org.apache.druid.extensions.contrib + druid-exact-count-bitmap + druid-exact-count-bitmap + Druid Aggregators for counting the exact cardinality of number columns + + + org.apache.druid + druid + 35.0.0-SNAPSHOT + ../../pom.xml + + + + + org.apache.calcite + calcite-core + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided + + + com.google.code.findbugs + jsr305 + provided + + + com.google.guava + guava + provided + + + com.google.inject + guice + provided + + + org.roaringbitmap + RoaringBitmap + 0.9.49 + provided + + + it.unimi.dsi + fastutil-core + 8.5.4 + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + + + junit + junit + test + + + org.junit.jupiter + junit-jupiter-api + test + + + org.easymock + easymock + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + + + org.hamcrest + hamcrest-core + test + + + org.reflections + reflections + test + + + diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64.java new file mode 100644 index 000000000000..52b26b3cc601 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64.java @@ -0,0 +1,63 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import java.nio.ByteBuffer; + +/** + * Interface for a 64-bit bitmap counter used for exact cardinality estimation. + *

+ * Implementations of this interface provide methods to add values, merge counters, retrieve the cardinality, + * and serialize the counter state. This is typically used in scenarios where exact distinct counting is required + * for large datasets, such as analytics or aggregation queries in Druid. + *

+ */ +public interface Bitmap64 +{ + /** + * Adds a value to the bitmap counter. The value is treated as a unique element to be tracked for cardinality. + * + * @param value the value to add to the counter + */ + void add(long value); + + /** + * Returns the exact cardinality (number of unique values) currently tracked by this counter. + * + * @return the number of unique values added to the counter + */ + long getCardinality(); + + /** + * Merges the unique values from another Bitmap64Counter into this one, resulting in a new counter that represents the union of both counters. + * This method modifies and returns the current Bitmap64Counter instance. + * + * @param rhs the other Bitmap64Counter to merge with + * @return a new Bitmap64Counter representing the union of both counters + */ + Bitmap64 fold(Bitmap64 rhs); + + /** + * Serializes the current state of the counter into a ByteBuffer. + * + * @return a ByteBuffer containing the serialized state of the counter + */ + ByteBuffer toByteBuffer(); +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactory.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactory.java new file mode 100644 index 000000000000..073cfb295f69 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactory.java @@ -0,0 +1,204 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.ObjectAggregateCombiner; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + * Base class for both build and merge factories + */ +public abstract class Bitmap64ExactCountAggregatorFactory extends AggregatorFactory +{ + // 1KiB is large enough for bookkeeping plus some future head-room & small enough that we don’t waste direct memory. + static final int MAX_INTERMEDIATE_SIZE = 1024; // 1 KiB + static final Comparator COMPARATOR = + Comparator.nullsFirst(Comparator.comparingLong(Bitmap64::getCardinality)); + + @Nonnull + private final String name; + + @Nonnull + private final String fieldName; + + Bitmap64ExactCountAggregatorFactory( + final String name, + final String fieldName + ) + { + this.name = Objects.requireNonNull(name); + this.fieldName = Objects.requireNonNull(fieldName); + } + + @Override + @Nonnull + @JsonProperty + public String getName() + { + return name; + } + + @Nonnull + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Nonnull + @Override + public List requiredFields() + { + return Collections.singletonList(fieldName); + } + + @Override + public Bitmap64 deserialize(final Object object) + { + return Bitmap64ExactCountMergeComplexMetricSerde.deserializeRoaringBitmap64Counter(object); + } + + @Override + public Bitmap64 combine(final Object objectA, final Object objectB) + { + if (objectB == null) { + return (Bitmap64) objectA; + } + if (objectA == null) { + return (Bitmap64) objectB; + } + return ((Bitmap64) objectA).fold((Bitmap64) objectB); + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new ObjectAggregateCombiner<>() + { + private Bitmap64 union = new RoaringBitmap64Counter(); + + @Override + public void reset(final ColumnValueSelector selector) + { + union = new RoaringBitmap64Counter(); + fold(selector); + } + + @Override + public void fold(final ColumnValueSelector selector) + { + final Bitmap64 bitmap64Counter = (Bitmap64) selector.getObject(); + union.fold(bitmap64Counter); + } + + @Nullable + @Override + public Bitmap64 getObject() + { + return union; + } + + @Override + public Class classOfObject() + { + return Bitmap64.class; + } + }; + } + + @Nullable + @Override + public Object finalizeComputation(@Nullable final Object object) + { + if (object == null) { + return null; + } + return ((Bitmap64) object).getCardinality(); + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new Bitmap64ExactCountMergeAggregatorFactory(getName(), getName()); + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(getCacheTypeId()).appendString(name).appendString(fieldName).build(); + } + + @Override + public int getMaxIntermediateSize() + { + return MAX_INTERMEDIATE_SIZE; + } + + @Override + public boolean equals(Object object) + { + if (this == object) { + return true; + } + if (object == null || !getClass().equals(object.getClass())) { + return false; + } + final Bitmap64ExactCountAggregatorFactory that = (Bitmap64ExactCountAggregatorFactory) object; + if (!name.equals(that.getName())) { + return false; + } + return fieldName.equals(that.getFieldName()); + } + + @Override + public int hashCode() + { + return Objects.hash(name, fieldName); + } + + @Override + public String toString() + { + return getClass().getSimpleName() + " {" + + " name=" + name + + ", fieldName=" + fieldName + + " }"; + } + + protected abstract byte getCacheTypeId(); + +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregator.java new file mode 100644 index 000000000000..ea1204a1525b --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregator.java @@ -0,0 +1,70 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.segment.BaseLongColumnValueSelector; + +import javax.annotation.Nullable; + +public class Bitmap64ExactCountBuildAggregator implements Aggregator +{ + private BaseLongColumnValueSelector selector; + private Bitmap64 bitmap; + + public Bitmap64ExactCountBuildAggregator(BaseLongColumnValueSelector selector) + { + this.selector = selector; + this.bitmap = new RoaringBitmap64Counter(); + } + + @Override + public void aggregate() + { + if (!selector.isNull()) { + bitmap.add(selector.getLong()); + } + } + + @Nullable + @Override + public Object get() + { + return bitmap; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + bitmap = null; + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactory.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactory.java new file mode 100644 index 000000000000..ac54c6b2702a --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactory.java @@ -0,0 +1,97 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; + +public class Bitmap64ExactCountBuildAggregatorFactory extends Bitmap64ExactCountAggregatorFactory +{ + public static final ColumnType TYPE = ColumnType.ofComplex(Bitmap64ExactCountModule.BUILD_TYPE_NAME); + + @JsonCreator + public Bitmap64ExactCountBuildAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, fieldName); + } + + @Override + protected byte getCacheTypeId() + { + return AggregatorUtil.BITMAP64_EXACT_COUNT_BUILD_CACHE_TYPE_ID; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + validateNumericColumn(metricFactory); + return new Bitmap64ExactCountBuildAggregator(metricFactory.makeColumnValueSelector(getFieldName())); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + validateNumericColumn(metricFactory); + return new Bitmap64ExactCountBuildBufferAggregator(metricFactory.makeColumnValueSelector(getFieldName())); + } + + @Override + public ColumnType getIntermediateType() + { + return TYPE; + } + + @Override + public ColumnType getResultType() + { + return ColumnType.LONG; + } + + /** + * Ensures that the column referenced by {@link #getFieldName()} is of a numeric type when this aggregator is used + * in a native Druid query. We must enforce the constraint here to provide a clear and early failure if the + * query references a non-numeric (e.g., STRING) column. + * + * @throws IllegalArgumentException if the column exists and is not numeric. + */ + private void validateNumericColumn(ColumnSelectorFactory metricFactory) + { + final ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(getFieldName()); + if (capabilities != null) { + final ValueType valueType = capabilities.getType(); + if (!valueType.isNumeric()) { + throw new IAE( + "Aggregation [%s] does not support column [%s] of type [%s]. Supported types: numeric.", + Bitmap64ExactCountModule.BUILD_TYPE_NAME, getFieldName(), valueType); + } + } + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregator.java new file mode 100644 index 000000000000..6d2f80286a08 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregator.java @@ -0,0 +1,132 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.BaseLongColumnValueSelector; + +import java.nio.ByteBuffer; +import java.util.IdentityHashMap; + +public class Bitmap64ExactCountBuildBufferAggregator implements BufferAggregator +{ + private final BaseLongColumnValueSelector selector; + private final IdentityHashMap> collectors = new IdentityHashMap<>(); + + public Bitmap64ExactCountBuildBufferAggregator(BaseLongColumnValueSelector selector) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + createNewCollector(buf, position); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + final int oldPosition = buf.position(); + try { + buf.position(position); + Bitmap64 bitmap64Counter = getOrCreateCollector(buf, position); + if (!selector.isNull()) { + bitmap64Counter.add(selector.getLong()); + } + } + finally { + buf.position(oldPosition); + } + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return getOrCreateCollector(buf, position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + + } + + @Override + public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer) + { + createNewCollector(newBuffer, newPosition); + Bitmap64 collector = collectors.get(oldBuffer).get(oldPosition); + putCollectors(newBuffer, newPosition, collector); + Int2ObjectMap collectorMap = collectors.get(oldBuffer); + if (collectorMap != null) { + collectorMap.remove(oldPosition); + if (collectorMap.isEmpty()) { + collectors.remove(oldBuffer); + } + } + } + + private void putCollectors(final ByteBuffer buffer, final int position, final Bitmap64 collector) + { + Int2ObjectMap map = collectors.computeIfAbsent(buffer, buf -> new Int2ObjectOpenHashMap<>()); + map.put(position, collector); + } + + private Bitmap64 getOrCreateCollector(ByteBuffer buf, int position) + { + Int2ObjectMap collectMap = collectors.get(buf); + Bitmap64 bitmap64Counter = collectMap != null ? collectMap.get(position) : null; + if (bitmap64Counter != null) { + return bitmap64Counter; + } + + return createNewCollector(buf, position); + } + + private Bitmap64 createNewCollector(ByteBuffer buf, int position) + { + buf.position(position); + Bitmap64 bitmap64Counter = new RoaringBitmap64Counter(); + Int2ObjectMap collectorMap = collectors.computeIfAbsent(buf, k -> new Int2ObjectOpenHashMap<>()); + collectorMap.put(position, bitmap64Counter); + return bitmap64Counter; + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildComplexMetricSerde.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildComplexMetricSerde.java new file mode 100644 index 000000000000..f3f6aca24676 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildComplexMetricSerde.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.aggregation.exact.count.bitmap64; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.segment.serde.ComplexMetricExtractor; + +import javax.annotation.Nullable; + +public class Bitmap64ExactCountBuildComplexMetricSerde extends Bitmap64ExactCountMergeComplexMetricSerde +{ + private static final ComplexMetricExtractor EXTRACTOR = new ComplexMetricExtractor() + { + + @Override + public Class extractedClass() + { + return Object.class; + } + + @Nullable + @Override + public Object extractValue(InputRow inputRow, String metricName) + { + return inputRow.getRaw(metricName); + } + }; + + @Override + public ComplexMetricExtractor getExtractor() + { + return EXTRACTOR; + } + +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregator.java new file mode 100644 index 000000000000..40e17f7e32b8 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregator.java @@ -0,0 +1,68 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +public class Bitmap64ExactCountMergeAggregator implements Aggregator +{ + private final ColumnValueSelector selector; + private Bitmap64 bitmap; + + public Bitmap64ExactCountMergeAggregator(ColumnValueSelector selector) + { + this.selector = selector; + this.bitmap = new RoaringBitmap64Counter(); + } + + @Override + public void aggregate() + { + bitmap.fold(selector.getObject()); + } + + @Nullable + @Override + public Object get() + { + return bitmap; + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + bitmap = null; + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactory.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactory.java new file mode 100644 index 000000000000..a33d70964637 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactory.java @@ -0,0 +1,75 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; + +public class Bitmap64ExactCountMergeAggregatorFactory extends Bitmap64ExactCountAggregatorFactory +{ + public static final ColumnType TYPE = ColumnType.ofComplex(Bitmap64ExactCountModule.MERGE_TYPE_NAME); + + @JsonCreator + public Bitmap64ExactCountMergeAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + super(name, fieldName); + } + + @Override + protected byte getCacheTypeId() + { + return AggregatorUtil.BITMAP64_EXACT_COUNT_MERGE_CACHE_TYPE_ID; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + ColumnValueSelector selector = metricFactory.makeColumnValueSelector(getFieldName()); + return new Bitmap64ExactCountMergeAggregator(selector); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + ColumnValueSelector selector = metricFactory.makeColumnValueSelector(getFieldName()); + return new Bitmap64ExactCountMergeBufferAggregator(selector); + } + + @Override + public ColumnType getIntermediateType() + { + return TYPE; + } + + @Override + public ColumnType getResultType() + { + return ColumnType.LONG; + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregator.java new file mode 100644 index 000000000000..0d002a33cacd --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregator.java @@ -0,0 +1,107 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; +import java.util.IdentityHashMap; + +public class Bitmap64ExactCountMergeBufferAggregator implements BufferAggregator +{ + private final ColumnValueSelector selector; + private final IdentityHashMap> counterCache = new IdentityHashMap<>(); + + public Bitmap64ExactCountMergeBufferAggregator(ColumnValueSelector selector) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + RoaringBitmap64Counter emptyCounter = new RoaringBitmap64Counter(); + addToCache(buf, position, emptyCounter); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + Object x = selector.getObject(); + if (x == null) { + return; + } + Bitmap64 bitmap64Counter = counterCache.get(buf).get(position); + bitmap64Counter.fold((RoaringBitmap64Counter) x); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return counterCache.get(buf).get(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() + { + counterCache.clear(); + } + + @Override + public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer) + { + Bitmap64 counter = counterCache.get(oldBuffer).get(oldPosition); + addToCache(newBuffer, newPosition, counter); + Int2ObjectMap counterMap = counterCache.get(oldBuffer); + if (counterMap != null) { + counterMap.remove(oldPosition); + if (counterMap.isEmpty()) { + counterCache.remove(oldBuffer); + } + } + } + + private void addToCache(final ByteBuffer buffer, final int position, final Bitmap64 counter) + { + Int2ObjectMap map = counterCache.computeIfAbsent(buffer, buf -> new Int2ObjectOpenHashMap<>()); + map.put(position, counter); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeComplexMetricSerde.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeComplexMetricSerde.java new file mode 100644 index 000000000000..0bce70eae96a --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeComplexMetricSerde.java @@ -0,0 +1,113 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.GenericColumnSerializer; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.ObjectStrategy; +import org.apache.druid.segment.serde.ComplexColumnPartSupplier; +import org.apache.druid.segment.serde.ComplexMetricExtractor; +import org.apache.druid.segment.serde.ComplexMetricSerde; +import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.nio.ByteBuffer; + +public class Bitmap64ExactCountMergeComplexMetricSerde extends ComplexMetricSerde +{ + static RoaringBitmap64Counter deserializeRoaringBitmap64Counter(final Object object) + { + if (object instanceof String) { + return RoaringBitmap64Counter.fromBytes(decodeStringToByteArray((String) object)); + } else if (object instanceof byte[]) { + return RoaringBitmap64Counter.fromBytes((byte[]) object); + } else if (object instanceof RoaringBitmap64Counter) { + return (RoaringBitmap64Counter) object; + } + throw new IAE("Cannot deserialize type[%s] to an RoaringBitmap64Counter:", object.getClass().getName()); + } + + private static byte[] decodeStringToByteArray(String string) + { + try { + return StringUtils.decodeBase64(StringUtils.toUtf8(string)); + } + catch (IllegalArgumentException e) { + throw new IAE("Failed to deserialize to RoaringBitmap64Counter, input is an invalid base64 string"); + } + } + + @Override + public String getTypeName() + { + return Bitmap64ExactCountModule.TYPE_NAME; // must be common type name + } + + @Override + public ObjectStrategy getObjectStrategy() + { + return Bitmap64ExactCountObjectStrategy.STRATEGY; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return RoaringBitmap64Counter.class; + } + + @Override + public RoaringBitmap64Counter extractValue(final InputRow inputRow, final String metricName) + { + final Object object = inputRow.getRaw(metricName); + if (object == null) { + return null; + } + return deserializeRoaringBitmap64Counter(object); + } + }; + } + + @Override + public void deserializeColumn(final ByteBuffer buf, final ColumnBuilder columnBuilder) + { + columnBuilder.setComplexColumnSupplier( + new ComplexColumnPartSupplier( + getTypeName(), + GenericIndexed.read(buf, Bitmap64ExactCountObjectStrategy.STRATEGY, columnBuilder.getFileMapper()) + ) + ); + } + + // support large columns + @Override + public GenericColumnSerializer getSerializer(final SegmentWriteOutMedium segmentWriteOutMedium, final String column) + { + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModule.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModule.java new file mode 100644 index 000000000000..a1db6c5055da --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModule.java @@ -0,0 +1,72 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.annotations.VisibleForTesting; +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.query.aggregation.exact.count.bitmap64.sql.Bitmap64ExactCountSqlAggregator; +import org.apache.druid.segment.serde.ComplexMetrics; +import org.apache.druid.sql.guice.SqlBindings; + +import java.util.Collections; +import java.util.List; + +public class Bitmap64ExactCountModule implements DruidModule +{ + public static final String TYPE_NAME = "Bitmap64ExactCount"; // common type name to be associated with segment data + public static final String BUILD_TYPE_NAME = "Bitmap64ExactCountBuild"; + public static final String MERGE_TYPE_NAME = "Bitmap64ExactCountMerge"; + + @Override + public List getJacksonModules() + { + return Collections.singletonList( + new SimpleModule("Bitmap64ExactCountModule") + .registerSubtypes( + new NamedType(Bitmap64ExactCountMergeAggregatorFactory.class, MERGE_TYPE_NAME), + new NamedType(Bitmap64ExactCountBuildAggregatorFactory.class, BUILD_TYPE_NAME), + new NamedType(Bitmap64ExactCountPostAggregator.class, "bitmap64ExactCount") + ) + .addSerializer( + RoaringBitmap64Counter.class, + new RoaringBitmap64CounterJsonSerializer() + ) + ); + } + + @Override + public void configure(Binder binder) + { + registerSerde(); + SqlBindings.addAggregator(binder, Bitmap64ExactCountSqlAggregator.class); + } + + @VisibleForTesting + public static void registerSerde() + { + ComplexMetrics.registerSerde(TYPE_NAME, new Bitmap64ExactCountMergeComplexMetricSerde()); + ComplexMetrics.registerSerde(BUILD_TYPE_NAME, new Bitmap64ExactCountBuildComplexMetricSerde()); + ComplexMetrics.registerSerde(MERGE_TYPE_NAME, new Bitmap64ExactCountMergeComplexMetricSerde()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountObjectStrategy.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountObjectStrategy.java new file mode 100644 index 000000000000..e3e5dc383db3 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountObjectStrategy.java @@ -0,0 +1,83 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.databind.util.ByteBufferBackedInputStream; +import org.apache.druid.segment.data.ObjectStrategy; + +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; + +public class Bitmap64ExactCountObjectStrategy implements ObjectStrategy +{ + static final Bitmap64ExactCountObjectStrategy STRATEGY = new Bitmap64ExactCountObjectStrategy(); + + @Override + public Class getClazz() + { + return RoaringBitmap64Counter.class; + } + + @Nullable + @Override + public Bitmap64 fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ByteBuffer readOnlyBuf = buffer.asReadOnlyBuffer(); + + if (readOnlyBuf.remaining() < numBytes) { + throw new BufferUnderflowException(); + } + + DataInputStream dataInputStream; + if (readOnlyBuf.hasArray()) { + // Use the underlying array directly without copying array. + dataInputStream = new DataInputStream(new ByteArrayInputStream( + readOnlyBuf.array(), + readOnlyBuf.arrayOffset() + readOnlyBuf.position(), + numBytes + )); + } else { + // Wrap ByteBuffer as DataInput to avoid copying underlying byte array. + ByteBuffer slice = readOnlyBuf.slice(); + slice.limit(numBytes); + dataInputStream = new DataInputStream(new ByteBufferBackedInputStream(slice)); + } + return RoaringBitmap64Counter.fromDataInput(dataInputStream); + } + + @Nullable + @Override + public byte[] toBytes(@Nullable Bitmap64 val) + { + if (val == null) { + return new byte[0]; + } + return val.toByteBuffer().array(); + } + + @Override + public int compare(Bitmap64 o1, Bitmap64 o2) + { + return Bitmap64ExactCountAggregatorFactory.COMPARATOR.compare(o1, o2); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregator.java new file mode 100644 index 000000000000..ee4bb834cc07 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregator.java @@ -0,0 +1,138 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Sets; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.PostAggregatorIds; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.util.Comparator; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +@JsonTypeName("bitmap64ExactCount") +public class Bitmap64ExactCountPostAggregator implements PostAggregator +{ + private final String name; + private final String fieldName; + + @JsonCreator + public Bitmap64ExactCountPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName + ) + { + this.name = name; + this.fieldName = fieldName; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + public Set getDependentFields() + { + return Sets.newHashSet(fieldName); + } + + @Override + public Comparator getComparator() + { + return ArithmeticPostAggregator.DEFAULT_COMPARATOR; + } + + @Override + @JsonProperty + public Object compute(Map combinedAggregators) + { + Object value = combinedAggregators.get(fieldName); + return ((Bitmap64) value).getCardinality(); + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Nullable + @Override + public ColumnType getType(ColumnInspector signature) + { + return ColumnType.LONG; + } + + @Override + public PostAggregator decorate(Map aggregators) + { + return this; + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(PostAggregatorIds.BITMAP64_EXACT_COUNT_TYPE_ID) + .appendString(fieldName) + .build(); + } + + @Override + public String toString() + { + return "Bitmap64ExactCountPostAggregator{" + + "name='" + name + '\'' + + ", field=" + fieldName + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Bitmap64ExactCountPostAggregator that = (Bitmap64ExactCountPostAggregator) o; + return name.equals(that.name) && + fieldName.equals(that.fieldName); + } + + @Override + public int hashCode() + { + return Objects.hash(name, fieldName); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/ExposedByteArrayOutputStream.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/ExposedByteArrayOutputStream.java new file mode 100644 index 000000000000..44d6a1a55086 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/ExposedByteArrayOutputStream.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.aggregation.exact.count.bitmap64; + +import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; + +/** + * This class is used to prevent extra copying of the underlying byte array. + */ +public class ExposedByteArrayOutputStream extends ByteArrayOutputStream +{ + ByteBuffer getBuffer() + { + return ByteBuffer.wrap(this.buf, 0, this.count); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64Counter.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64Counter.java new file mode 100644 index 000000000000..db0d0a611a55 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64Counter.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.aggregation.exact.count.bitmap64; + +import org.apache.druid.java.util.common.logger.Logger; +import org.roaringbitmap.longlong.Roaring64NavigableMap; + +import java.io.ByteArrayInputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.nio.ByteBuffer; + +public class RoaringBitmap64Counter implements Bitmap64 +{ + private static final Logger log = new Logger(RoaringBitmap64Counter.class); + + private final Roaring64NavigableMap bitmap; + + public RoaringBitmap64Counter() + { + this.bitmap = new Roaring64NavigableMap(); + } + + private RoaringBitmap64Counter(Roaring64NavigableMap bitmap) + { + this.bitmap = bitmap; + } + + public static RoaringBitmap64Counter fromBytes(byte[] bytes) + { + ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); + DataInputStream in = new DataInputStream(inputStream); + return fromDataInput(in); + } + + public static RoaringBitmap64Counter fromDataInput(DataInput in) + { + try { + Roaring64NavigableMap bitmap = new Roaring64NavigableMap(); + bitmap.deserialize(in); + return new RoaringBitmap64Counter(bitmap); + } + catch (Exception e) { + log.error(e, "Failed to deserialize RoaringBitmap64Counter from data input"); + throw new RuntimeException(e); + } + } + + @Override + public void add(long value) + { + bitmap.addLong(value); + } + + @Override + public long getCardinality() + { + return bitmap.getLongCardinality(); + } + + @Override + public Bitmap64 fold(Bitmap64 rhs) + { + if (rhs != null) { + bitmap.or(((RoaringBitmap64Counter) rhs).bitmap); + } + + return this; + } + + @Override + public ByteBuffer toByteBuffer() + { + bitmap.runOptimize(); + try { + final ExposedByteArrayOutputStream out = new ExposedByteArrayOutputStream(); + bitmap.serialize(new DataOutputStream(out)); + return out.getBuffer(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64CounterJsonSerializer.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64CounterJsonSerializer.java new file mode 100644 index 000000000000..e2a3ce12df2d --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/RoaringBitmap64CounterJsonSerializer.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.aggregation.exact.count.bitmap64; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.SerializerProvider; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public class RoaringBitmap64CounterJsonSerializer extends JsonSerializer +{ + @Override + public void serialize( + final RoaringBitmap64Counter bitmap64Counter, + final JsonGenerator jgen, + final SerializerProvider provider + ) throws IOException + { + ByteBuffer buffer = bitmap64Counter.toByteBuffer(); + jgen.writeBinary(buffer.array(), buffer.arrayOffset(), buffer.limit()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregator.java b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregator.java new file mode 100644 index 000000000000..b93bce50ca6e --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregator.java @@ -0,0 +1,185 @@ +/* + * 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.aggregation.exact.count.bitmap64.sql; + +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.OperandTypes; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.exact.count.bitmap64.Bitmap64ExactCountBuildAggregatorFactory; +import org.apache.druid.query.aggregation.exact.count.bitmap64.Bitmap64ExactCountMergeAggregatorFactory; +import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.SqlAggregator; +import org.apache.druid.sql.calcite.aggregation.builtin.SimpleSqlAggregator; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.expression.Expressions; +import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.planner.Calcites; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; +import org.apache.druid.sql.calcite.table.RowSignatures; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; + +public class Bitmap64ExactCountSqlAggregator implements SqlAggregator +{ + private static final String NAME = "BITMAP64_EXACT_COUNT"; + private static final SqlAggFunction FUNCTION_INSTANCE + = OperatorConversions.aggregatorBuilder(NAME) + .operandTypeChecker(OperandTypes.or( + OperandTypes.NUMERIC, + RowSignatures.complexTypeChecker(Bitmap64ExactCountMergeAggregatorFactory.TYPE), + RowSignatures.complexTypeChecker(Bitmap64ExactCountBuildAggregatorFactory.TYPE) + )) + .returnTypeNonNull(SqlTypeName.BIGINT) + .functionCategory(SqlFunctionCategory.NUMERIC) + .build(); + + @Override + public SqlAggFunction calciteFunction() + { + return FUNCTION_INSTANCE; + } + + @Nullable + @Override + public Aggregation toDruidAggregation( + PlannerContext plannerContext, + RowSignature rowSignature, + VirtualColumnRegistry virtualColumnRegistry, + RexBuilder rexBuilder, + String name, + AggregateCall aggregateCall, + Project project, + List existingAggregations, + boolean finalizeAggregations + ) + { + // Don't use Aggregations.getArgumentsForSimpleAggregator, since it won't let us use direct column access + // for Bitmap64ExactCountBuild / Bitmap64ExactCountMerge columns. + final RexNode columnRexNode = Expressions.fromFieldAccess( + rexBuilder.getTypeFactory(), + rowSignature, + project, + aggregateCall.getArgList().get(0) + ); + + final DruidExpression columnArg = Expressions.toDruidExpression(plannerContext, rowSignature, columnRexNode); + if (columnArg == null) { + return null; + } + + final AggregatorFactory aggregatorFactory; + final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name; + + // Create Merge Aggregator if is already Bitmap64 complex type, else create Build Aggregator. + if (isBitmap64ComplexType(columnArg, rowSignature)) { + aggregatorFactory = new Bitmap64ExactCountMergeAggregatorFactory(aggregatorName, columnArg.getDirectColumn()); + } else { + rejectImplicitCastsToNumericType(columnRexNode, name); + aggregatorFactory = createBuildAggregatorFactory(columnRexNode, columnArg, virtualColumnRegistry, aggregatorName); + } + + return toAggregation(name, finalizeAggregations, aggregatorFactory); + } + + private boolean isBitmap64ComplexType(DruidExpression columnArg, RowSignature rowSignature) + { + return columnArg.isDirectColumnAccess() + && rowSignature.getColumnType(columnArg.getDirectColumn()) + .map(type -> type.is(ValueType.COMPLEX)) + .orElse(false); + } + + /** + * Rejects SQL queries pertaining to using non-numeric columns that can be implicitly cast to numeric values (e.g. String). + * There is a second layer of protection at {@link Bitmap64ExactCountBuildAggregatorFactory#validateNumericColumn}, + * which checks the column types when given a query in Druid Native. + * + *

+ * This function aims to keep the Exception type consistent with the rest of Druid. + *

+ */ + private void rejectImplicitCastsToNumericType(RexNode columnRexNode, String columnName) + { + if (columnRexNode.isA(SqlKind.CAST)) { + final RelDataType operandType = ((RexCall) columnRexNode).operands.get(0).getType(); + final ColumnType operandDruidType = Calcites.getColumnTypeForRelDataType(operandType); + if (operandDruidType == null || !operandDruidType.isNumeric()) { + throw SimpleSqlAggregator.badTypeException(columnName, NAME, ColumnType.STRING); + } + } + } + + private AggregatorFactory createBuildAggregatorFactory( + final RexNode columnRexNode, + final DruidExpression columnArg, + final VirtualColumnRegistry virtualColumnRegistry, + final String aggregatorName + ) + { + final RelDataType dataType = columnRexNode.getType(); + final ColumnType inputType = Calcites.getColumnTypeForRelDataType(dataType); + if (inputType == null) { + throw new ISE( + "Cannot translate sqlTypeName[%s] to Druid type for field[%s]", + dataType.getSqlTypeName(), + aggregatorName + ); + } + + final DimensionSpec dimensionSpec; + + if (columnArg.isDirectColumnAccess()) { + dimensionSpec = columnArg.getSimpleExtraction().toDimensionSpec(null, inputType); + } else { + String virtualColumnName = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(columnArg, dataType); + dimensionSpec = new DefaultDimensionSpec(virtualColumnName, null, inputType); + } + + return new Bitmap64ExactCountBuildAggregatorFactory(aggregatorName, dimensionSpec.getDimension()); + } + + private Aggregation toAggregation(String name, boolean finalizeAggregations, AggregatorFactory aggregatorFactory) + { + return Aggregation.create( + Collections.singletonList(aggregatorFactory), + finalizeAggregations ? new FinalizingFieldAccessPostAggregator(name, aggregatorFactory.getName()) + : null + ); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/druid-exact-count-bitmap/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..38559d580313 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +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.aggregation.exact.count.bitmap64.Bitmap64ExactCountModule diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactoryTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactoryTest.java new file mode 100644 index 000000000000..16f24a5b048b --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountAggregatorFactoryTest.java @@ -0,0 +1,299 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.query.aggregation.AggregateCombiner; +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; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class Bitmap64ExactCountAggregatorFactoryTest +{ + private static final String NAME = "testName"; + private static final String FIELD_NAME = "testFieldName"; + + private TestBitmap64ExactCountAggregatorFactory factory; + + // Concrete implementation for testing the abstract class + private static class TestBitmap64ExactCountAggregatorFactory extends Bitmap64ExactCountAggregatorFactory + { + private static final byte CACHE_TYPE_ID = 0x1A; // Using a distinct byte for test + + TestBitmap64ExactCountAggregatorFactory(String name, String fieldName) + { + super(name, fieldName); + } + + @Override + protected byte getCacheTypeId() + { + return CACHE_TYPE_ID; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return null; + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return null; + } + + @Override + public ColumnType getIntermediateType() + { + return Bitmap64ExactCountBuildAggregatorFactory.TYPE; + } + + @Override + public AggregatorFactory withName(String newName) + { + return new TestBitmap64ExactCountAggregatorFactory(newName, getFieldName()); + } + } + + @BeforeEach + public void setUp() + { + factory = new TestBitmap64ExactCountAggregatorFactory(NAME, FIELD_NAME); + } + + @Test + public void testConstructor() + { + Assertions.assertEquals(NAME, factory.getName()); + Assertions.assertEquals(FIELD_NAME, factory.getFieldName()); + } + + @Test + public void testRequiredFields() + { + Assertions.assertEquals(Collections.singletonList(FIELD_NAME), factory.requiredFields()); + } + + @Test + public void testFinalizeComputationNull() + { + Assertions.assertNull(factory.finalizeComputation(null)); + } + + @Test + public void testFinalizeComputation() + { + Bitmap64 counter = new RoaringBitmap64Counter(); + counter.add(1L); + counter.add(2L); + Assertions.assertEquals(2L, factory.finalizeComputation(counter)); + } + + @Test + public void testGetComparator() + { + Bitmap64 counter1 = new RoaringBitmap64Counter(); + counter1.add(1L); + + Bitmap64 counter2 = new RoaringBitmap64Counter(); + counter2.add(1L); + counter2.add(2L); + + Assertions.assertTrue(factory.getComparator().compare(counter1, counter2) < 0); + Assertions.assertTrue(factory.getComparator().compare(counter2, counter1) > 0); + Assertions.assertTrue(factory.getComparator().compare(null, counter1) < 0); + Assertions.assertTrue(factory.getComparator().compare(counter1, null) > 0); + } + + @Test + public void testGetCombiningFactory() + { + AggregatorFactory combiningFactory = factory.getCombiningFactory(); + Assertions.assertInstanceOf(Bitmap64ExactCountMergeAggregatorFactory.class, combiningFactory); + Assertions.assertEquals(NAME, combiningFactory.getName()); + Assertions.assertEquals(NAME, ((Bitmap64ExactCountMergeAggregatorFactory) combiningFactory).getFieldName()); + } + + @Test + public void testGetMaxIntermediateSize() + { + Assertions.assertEquals( + Bitmap64ExactCountAggregatorFactory.MAX_INTERMEDIATE_SIZE, + factory.getMaxIntermediateSize() + ); + } + + @Test + public void testEqualsAndHashCode() + { + // Test symmetry + TestBitmap64ExactCountAggregatorFactory factory2 = new TestBitmap64ExactCountAggregatorFactory( + NAME, + FIELD_NAME + ); + Assertions.assertEquals(factory, factory2); + Assertions.assertEquals(factory.hashCode(), factory2.hashCode()); + + // Test different name + TestBitmap64ExactCountAggregatorFactory factoryDiffName = new TestBitmap64ExactCountAggregatorFactory( + NAME + "_diff", + FIELD_NAME + ); + Assertions.assertNotEquals(factory, factoryDiffName); + + // Test different fieldName + TestBitmap64ExactCountAggregatorFactory factoryDiffFieldName = new TestBitmap64ExactCountAggregatorFactory( + NAME, + FIELD_NAME + "_diff" + ); + Assertions.assertNotEquals(factory, factoryDiffFieldName); + + // Test different class (even if fields match, if getEffectiveClass is used in parent equals) + // For Bitmap64ExactCountAggregatorFactory, getClass() is used in equals. + Bitmap64ExactCountAggregatorFactory anotherConcreteFactory = new Bitmap64ExactCountBuildAggregatorFactory( + NAME, + FIELD_NAME + ); + Assertions.assertNotEquals( + factory, + anotherConcreteFactory, + "Test factory should not be equal to Build factory due to different class" + ); + } + + @Test + public void testToString() + { + String expected = "TestBitmap64ExactCountAggregatorFactory { name=" + + NAME + + ", fieldName=" + + FIELD_NAME + + " }"; + Assertions.assertEquals(expected, factory.toString()); + } + + @Test + public void testGetCacheKey() + { + byte[] cacheKey1 = factory.getCacheKey(); + TestBitmap64ExactCountAggregatorFactory factory2 = new TestBitmap64ExactCountAggregatorFactory( + NAME, + FIELD_NAME + ); + byte[] cacheKey2 = factory2.getCacheKey(); + Assertions.assertArrayEquals(cacheKey1, cacheKey2); + + TestBitmap64ExactCountAggregatorFactory factoryDiffName = new TestBitmap64ExactCountAggregatorFactory( + NAME + "_diff", + FIELD_NAME + ); + byte[] cacheKeyDiffName = factoryDiffName.getCacheKey(); + Assertions.assertFalse(Arrays.equals(cacheKey1, cacheKeyDiffName)); + + TestBitmap64ExactCountAggregatorFactory factoryDiffFieldName = new TestBitmap64ExactCountAggregatorFactory( + NAME, + FIELD_NAME + "_diff" + ); + byte[] cacheKeyDiffFieldName = factoryDiffFieldName.getCacheKey(); + Assertions.assertFalse(Arrays.equals(cacheKey1, cacheKeyDiffFieldName)); + } + + @Test + public void testCombine() + { + Bitmap64 counter1 = new RoaringBitmap64Counter(); + counter1.add(1L); + counter1.add(2L); + + Bitmap64 counter2 = new RoaringBitmap64Counter(); + counter2.add(2L); + counter2.add(3L); + + Bitmap64 result = factory.combine(counter1, counter2); + Assertions.assertEquals(3L, result.getCardinality()); + Assertions.assertSame(counter1, result); + + Bitmap64 counter3 = new RoaringBitmap64Counter(); + counter3.add(4L); + Bitmap64 resultNullB = factory.combine(counter3, null); + Assertions.assertSame(counter3, resultNullB); + Assertions.assertEquals(1L, resultNullB.getCardinality()); + + Bitmap64 counter4 = new RoaringBitmap64Counter(); + counter4.add(5L); + Bitmap64 resultNullA = factory.combine(null, counter4); + Assertions.assertSame(counter4, resultNullA); + Assertions.assertEquals(1L, resultNullA.getCardinality()); + + Assertions.assertNull(factory.combine(null, null)); + } + + @Test + public void testMakeAggregateCombiner() + { + AggregateCombiner combiner = factory.makeAggregateCombiner(); + Assertions.assertNotNull(combiner); + + ColumnValueSelector selector = + EasyMock.createMock(ColumnValueSelector.class); + + Bitmap64 counter1 = new RoaringBitmap64Counter(); + counter1.add(10L); + counter1.add(20L); + + Bitmap64 counter2 = new RoaringBitmap64Counter(); + counter2.add(20L); + counter2.add(30L); + + EasyMock.expect(selector.getObject()).andReturn(counter1).times(1); + EasyMock.replay(selector); + combiner.fold(selector); + EasyMock.verify(selector); + Assertions.assertEquals(2L, combiner.getObject().getCardinality()); + + EasyMock.reset(selector); + EasyMock.expect(selector.getObject()).andReturn(counter2).times(1); + EasyMock.replay(selector); + combiner.fold(selector); + EasyMock.verify(selector); + Assertions.assertEquals(3L, combiner.getObject().getCardinality()); + + EasyMock.reset(selector); + Bitmap64 counter3 = new RoaringBitmap64Counter(); + counter3.add(40L); + EasyMock.expect(selector.getObject()).andReturn(counter3).times(1); + EasyMock.replay(selector); + combiner.reset(selector); + EasyMock.verify(selector); + Assertions.assertEquals(1L, combiner.getObject().getCardinality()); + + Assertions.assertEquals(Bitmap64.class, combiner.classOfObject()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactoryTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactoryTest.java new file mode 100644 index 000000000000..07295cc81cdc --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorFactoryTest.java @@ -0,0 +1,156 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.TestObjectColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class Bitmap64ExactCountBuildAggregatorFactoryTest +{ + private static final String NAME = "exactCountBuildTestName"; + private static final String FIELD_NAME = "exactCountBuildTestFieldName"; + + private Bitmap64ExactCountBuildAggregatorFactory factory; + + @BeforeEach + public void setUp() + { + factory = new Bitmap64ExactCountBuildAggregatorFactory(NAME, FIELD_NAME); + } + + @Test + public void testConstructor() + { + Assertions.assertEquals(NAME, factory.getName()); + Assertions.assertEquals(FIELD_NAME, factory.getFieldName()); + } + + @Test + public void testGetCacheTypeId() + { + Assertions.assertEquals(AggregatorUtil.BITMAP64_EXACT_COUNT_BUILD_CACHE_TYPE_ID, factory.getCacheTypeId()); + } + + @Test + public void testFactorize() + { + ColumnSelectorFactory selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(selectorFactory.getColumnCapabilities(FIELD_NAME)).andReturn(null); + EasyMock.expect(selectorFactory.makeColumnValueSelector(FIELD_NAME)) + .andReturn(new TestObjectColumnSelector<>(null)); // Return a dummy selector + EasyMock.replay(selectorFactory); + + Assertions.assertInstanceOf(Bitmap64ExactCountBuildAggregator.class, factory.factorize(selectorFactory)); + EasyMock.verify(selectorFactory); + } + + @Test + public void testFactorizeWithNonNumericColumnThrowsIAE() + { + ColumnSelectorFactory selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + ColumnCapabilities capabilities = EasyMock.createMock(ColumnCapabilities.class); + + EasyMock.expect(selectorFactory.getColumnCapabilities(FIELD_NAME)).andReturn(capabilities); + EasyMock.expect(capabilities.getType()).andReturn(ValueType.STRING); + + EasyMock.replay(selectorFactory, capabilities); + + Assertions.assertThrows( + IAE.class, + () -> factory.factorize(selectorFactory) + ); + + EasyMock.verify(selectorFactory, capabilities); + } + + @Test + public void testFactorizeBuffered() + { + ColumnSelectorFactory selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(selectorFactory.getColumnCapabilities(FIELD_NAME)).andReturn(null); + EasyMock.expect(selectorFactory.makeColumnValueSelector(FIELD_NAME)) + .andReturn(new TestObjectColumnSelector<>(null)); // Return a dummy selector + EasyMock.replay(selectorFactory); + + Assertions.assertInstanceOf( + Bitmap64ExactCountBuildBufferAggregator.class, + factory.factorizeBuffered(selectorFactory) + ); + EasyMock.verify(selectorFactory); + } + + @Test + public void testGetIntermediateType() + { + Assertions.assertEquals(Bitmap64ExactCountBuildAggregatorFactory.TYPE, factory.getIntermediateType()); + } + + @Test + public void testGetResultType() + { + Assertions.assertEquals(ColumnType.LONG, factory.getResultType()); + } + + @Test + public void testEqualsAndHashCode() + { + Bitmap64ExactCountBuildAggregatorFactory factory1 = new Bitmap64ExactCountBuildAggregatorFactory( + NAME, + FIELD_NAME + ); + Bitmap64ExactCountBuildAggregatorFactory factory2 = new Bitmap64ExactCountBuildAggregatorFactory( + NAME, + FIELD_NAME + ); + Bitmap64ExactCountBuildAggregatorFactory factoryDiffName = new Bitmap64ExactCountBuildAggregatorFactory( + NAME + "_diff", + FIELD_NAME + ); + Bitmap64ExactCountBuildAggregatorFactory factoryDiffFieldName = new Bitmap64ExactCountBuildAggregatorFactory( + NAME, + FIELD_NAME + "_diff" + ); + + Assertions.assertEquals(factory1, factory2); + Assertions.assertEquals(factory1.hashCode(), factory2.hashCode()); + + Assertions.assertNotEquals(factory1, factoryDiffName); + Assertions.assertNotEquals(factory1.hashCode(), factoryDiffName.hashCode()); + + Assertions.assertNotEquals(factory1, factoryDiffFieldName); + Assertions.assertNotEquals(factory1.hashCode(), factoryDiffFieldName.hashCode()); + } + + @Test + public void testToString() + { + String expected = "Bitmap64ExactCountBuildAggregatorFactory { name=" + NAME + ", fieldName=" + FIELD_NAME + " }"; + Assertions.assertEquals(expected, factory.toString()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorTest.java new file mode 100644 index 000000000000..5ea06b1f0297 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildAggregatorTest.java @@ -0,0 +1,115 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class Bitmap64ExactCountBuildAggregatorTest +{ + private BaseLongColumnValueSelector mockSelector; + private Bitmap64ExactCountBuildAggregator aggregator; + + @BeforeEach + public void setUp() + { + mockSelector = EasyMock.createMock(BaseLongColumnValueSelector.class); + EasyMock.expect(mockSelector.isNull()).andReturn(false).anyTimes(); + aggregator = new Bitmap64ExactCountBuildAggregator(mockSelector); + } + + @Test + public void testAggregateSingleValue() + { + EasyMock.expect(mockSelector.getLong()).andReturn(123L).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); + + Bitmap64 counter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(counter); + Assertions.assertEquals(1, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleDistinctValues() + { + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(20L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(30L).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); + aggregator.aggregate(); + aggregator.aggregate(); + + Bitmap64 counter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(counter); + Assertions.assertEquals(3, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleValuesWithDuplicates() + { + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(20L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); // Duplicate + EasyMock.replay(mockSelector); + + aggregator.aggregate(); + aggregator.aggregate(); + aggregator.aggregate(); + + Bitmap64 counter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(counter); + Assertions.assertEquals(2, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testGetInitialState() + { + Bitmap64 counter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(counter); + Assertions.assertEquals(0, counter.getCardinality()); + } + + @Test + public void testClose() + { + aggregator.close(); + Assertions.assertNull(aggregator.get(), "Bitmap should be null after close"); + } + + @Test + public void testUnsupportedGetOperations() + { + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getFloat()); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getLong()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregatorTest.java new file mode 100644 index 000000000000..8df083706802 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountBuildBufferAggregatorTest.java @@ -0,0 +1,224 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +public class Bitmap64ExactCountBuildBufferAggregatorTest +{ + private BaseLongColumnValueSelector mockSelector; + private Bitmap64ExactCountBuildBufferAggregator aggregator; + private ByteBuffer buffer; + private static final int BUFFER_CAPACITY = 1024; + private static final int POSITION_1 = 0; + private static final int POSITION_2 = 100; // Another distinct position + + @BeforeEach + public void setUp() + { + mockSelector = EasyMock.createMock(BaseLongColumnValueSelector.class); + EasyMock.expect(mockSelector.isNull()).andReturn(false).anyTimes(); + aggregator = new Bitmap64ExactCountBuildBufferAggregator(mockSelector); + buffer = ByteBuffer.allocate(BUFFER_CAPACITY); + } + + @Test + public void testInit() + { + aggregator.init(buffer, POSITION_1); + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counter); + Assertions.assertEquals(0, counter.getCardinality()); + } + + @Test + public void testAggregateSingleValue() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(123L).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateCreatesCollectorIfNotExists() + { + // No init call, aggregate should create it + EasyMock.expect(mockSelector.getLong()).andReturn(456L).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counter); + Assertions.assertEquals(1, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleDistinctValues() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(20L).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(2, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateWithDuplicates() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(20L).once(); + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); // Duplicate + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(2, counter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateAtDifferentPositions() + { + aggregator.init(buffer, POSITION_1); + aggregator.init(buffer, POSITION_2); + + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); // For POSITION_1 + EasyMock.expect(mockSelector.getLong()).andReturn(20L).once(); // For POSITION_2 + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_2); + + Bitmap64 counter1 = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, counter1.getCardinality()); + + Bitmap64 counter2 = (Bitmap64) aggregator.get(buffer, POSITION_2); + Assertions.assertEquals(1, counter2.getCardinality()); + + Assertions.assertNotSame(counter1, counter2); + + EasyMock.verify(mockSelector); + } + + @Test + public void testGetWithoutInitOrAggregateReturnsNewCounter() + { + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counter); + Assertions.assertEquals(0, counter.getCardinality(), "Getting for a new position should return an empty counter"); + } + + @Test + public void testCloseIsNoOp() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(10L).once(); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + + aggregator.close(); // Should be a no-op + + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counter, "Counter should still exist after close"); + Assertions.assertEquals(1, counter.getCardinality()); + EasyMock.verify(mockSelector); + } + + @Test + public void testRelocateSameBuffer() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(123L).times(1); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + EasyMock.verify(mockSelector); + + Bitmap64 originalCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, originalCounter.getCardinality()); + + aggregator.relocate(POSITION_1, POSITION_2, buffer, buffer); + + Bitmap64 newCounter = (Bitmap64) aggregator.get(buffer, POSITION_2); + Assertions.assertNotNull(newCounter); + Assertions.assertEquals(1, newCounter.getCardinality()); + Assertions.assertSame(originalCounter, newCounter, "Relocate in same buffer should move the same counter instance"); + } + + @Test + public void testRelocateDifferentBuffers() + { + aggregator.init(buffer, POSITION_1); + EasyMock.expect(mockSelector.getLong()).andReturn(456L).times(1); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + EasyMock.verify(mockSelector); + + Bitmap64 originalCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, originalCounter.getCardinality()); + + ByteBuffer newBuffer = ByteBuffer.allocate(BUFFER_CAPACITY); + aggregator.relocate(POSITION_1, POSITION_1, buffer, newBuffer); // new buffer, same position offset + + Bitmap64 newCounter = (Bitmap64) aggregator.get(newBuffer, POSITION_1); + Assertions.assertNotNull(newCounter); + Assertions.assertEquals(1, newCounter.getCardinality()); + Assertions.assertSame( + originalCounter, + newCounter, + "Relocate to different buffer should move the same counter instance" + ); + } + + @Test + public void testUnsupportedGetOperations() + { + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getLong(buffer, POSITION_1)); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getDouble(buffer, POSITION_1)); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getFloat(buffer, POSITION_1)); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactoryTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactoryTest.java new file mode 100644 index 000000000000..8f63a1804067 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorFactoryTest.java @@ -0,0 +1,136 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.TestObjectColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnType; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class Bitmap64ExactCountMergeAggregatorFactoryTest +{ + private static final String NAME = "exactCountMergeTestName"; + private static final String FIELD_NAME = "exactCountMergeTestFieldName"; + + private Bitmap64ExactCountMergeAggregatorFactory factory; + + @BeforeEach + public void setUp() + { + factory = new Bitmap64ExactCountMergeAggregatorFactory(NAME, FIELD_NAME); + } + + @Test + public void testConstructor() + { + Assertions.assertEquals(NAME, factory.getName()); + Assertions.assertEquals(FIELD_NAME, factory.getFieldName()); + } + + @Test + public void testGetCacheTypeId() + { + Assertions.assertEquals(AggregatorUtil.BITMAP64_EXACT_COUNT_MERGE_CACHE_TYPE_ID, factory.getCacheTypeId()); + } + + @Test + public void testFactorize() + { + ColumnSelectorFactory selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(selectorFactory.makeColumnValueSelector(FIELD_NAME)) + .andReturn(new TestObjectColumnSelector(null)); // Return a dummy selector + EasyMock.replay(selectorFactory); + + Assertions.assertInstanceOf(Bitmap64ExactCountMergeAggregator.class, factory.factorize(selectorFactory)); + EasyMock.verify(selectorFactory); + } + + @Test + public void testFactorizeBuffered() + { + ColumnSelectorFactory selectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(selectorFactory.makeColumnValueSelector(FIELD_NAME)) + .andReturn(new TestObjectColumnSelector(null)); // Return a dummy selector + EasyMock.replay(selectorFactory); + + Assertions.assertInstanceOf( + Bitmap64ExactCountMergeBufferAggregator.class, + factory.factorizeBuffered(selectorFactory) + ); + EasyMock.verify(selectorFactory); + } + + @Test + public void testGetIntermediateType() + { + Assertions.assertEquals(Bitmap64ExactCountMergeAggregatorFactory.TYPE, factory.getIntermediateType()); + } + + @Test + public void testGetResultType() + { + Assertions.assertEquals(ColumnType.LONG, factory.getResultType()); + } + + @Test + public void testEqualsAndHashCode() + { + Bitmap64ExactCountMergeAggregatorFactory factory1 = new Bitmap64ExactCountMergeAggregatorFactory( + NAME, + FIELD_NAME + ); + Bitmap64ExactCountMergeAggregatorFactory factory2 = new Bitmap64ExactCountMergeAggregatorFactory( + NAME, + FIELD_NAME + ); + Bitmap64ExactCountMergeAggregatorFactory factoryDiffName = new Bitmap64ExactCountMergeAggregatorFactory( + NAME + "_diff", + FIELD_NAME + ); + Bitmap64ExactCountMergeAggregatorFactory factoryDiffFieldName = new Bitmap64ExactCountMergeAggregatorFactory( + NAME, + FIELD_NAME + "_diff" + ); + + Assertions.assertEquals(factory1, factory2); + Assertions.assertEquals(factory1.hashCode(), factory2.hashCode()); + + Assertions.assertNotEquals(factory1, factoryDiffName); + Assertions.assertNotEquals(factory1.hashCode(), factoryDiffName.hashCode()); + + Assertions.assertNotEquals(factory1, factoryDiffFieldName); + Assertions.assertNotEquals(factory1.hashCode(), factoryDiffFieldName.hashCode()); + } + + @Test + public void testToString() + { + String expected = "Bitmap64ExactCountMergeAggregatorFactory { name=" + + NAME + + ", fieldName=" + + FIELD_NAME + + " }"; + Assertions.assertEquals(expected, factory.toString()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorTest.java new file mode 100644 index 000000000000..89060ff5be20 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeAggregatorTest.java @@ -0,0 +1,147 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class Bitmap64ExactCountMergeAggregatorTest +{ + private ColumnValueSelector mockSelector; + private Bitmap64ExactCountMergeAggregator aggregator; + + @BeforeEach + public void setUp() + { + mockSelector = EasyMock.createMock(ColumnValueSelector.class); + aggregator = new Bitmap64ExactCountMergeAggregator(mockSelector); + } + + @Test + public void testAggregateWithNonNullCounter() + { + RoaringBitmap64Counter inputCounter = new RoaringBitmap64Counter(); + inputCounter.add(123L); + inputCounter.add(456L); + + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(resultCounter); + Assertions.assertEquals(2, resultCounter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateWithNullCounter() + { + EasyMock.expect(mockSelector.getObject()).andReturn(null).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); // Should not throw NPE, RoaringBitmap64Counter.fold handles null + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(resultCounter); + Assertions.assertEquals(0, resultCounter.getCardinality(), "Aggregating null should not change cardinality count"); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleCounters() + { + RoaringBitmap64Counter inputCounter1 = new RoaringBitmap64Counter(); + inputCounter1.add(10L); + inputCounter1.add(20L); + + RoaringBitmap64Counter inputCounter2 = new RoaringBitmap64Counter(); + inputCounter2.add(20L); // Duplicate with counter1 + inputCounter2.add(30L); + + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter1).once(); + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter2).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); + aggregator.aggregate(); + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(resultCounter); + Assertions.assertEquals(3, resultCounter.getCardinality()); // 10, 20, 30 + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleCountersIncludingNull() + { + RoaringBitmap64Counter inputCounter1 = new RoaringBitmap64Counter(); + inputCounter1.add(10L); + inputCounter1.add(20L); + + RoaringBitmap64Counter inputCounter3 = new RoaringBitmap64Counter(); + inputCounter3.add(20L); + inputCounter3.add(30L); + + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter1).once(); + EasyMock.expect(mockSelector.getObject()).andReturn(null).once(); // Null counter + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter3).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(); // counter1 + aggregator.aggregate(); // null + aggregator.aggregate(); // counter3 + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(resultCounter); + Assertions.assertEquals(3, resultCounter.getCardinality()); // 10, 20, 30 + + EasyMock.verify(mockSelector); + } + + @Test + public void testGetInitialState() + { + Bitmap64 counter = (Bitmap64) aggregator.get(); + Assertions.assertNotNull(counter); + Assertions.assertEquals(0, counter.getCardinality()); + } + + @Test + public void testClose() + { + aggregator.close(); + Assertions.assertNull(aggregator.get(), "Bitmap should be null after close"); + } + + @Test + public void testUnsupportedGetOperations() + { + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getFloat()); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getLong()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregatorTest.java new file mode 100644 index 000000000000..b55ab3d18157 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountMergeBufferAggregatorTest.java @@ -0,0 +1,228 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.nio.ByteBuffer; + +public class Bitmap64ExactCountMergeBufferAggregatorTest +{ + private ColumnValueSelector mockSelector; + private Bitmap64ExactCountMergeBufferAggregator aggregator; + private ByteBuffer buffer; + private static final int BUFFER_CAPACITY = 1024; + private static final int POSITION_1 = 0; + private static final int POSITION_2 = 100; + + @BeforeEach + public void setUp() + { + mockSelector = EasyMock.createMock(ColumnValueSelector.class); + aggregator = new Bitmap64ExactCountMergeBufferAggregator(mockSelector); + buffer = ByteBuffer.allocate(BUFFER_CAPACITY); + } + + @Test + public void testInit() + { + aggregator.init(buffer, POSITION_1); + Bitmap64 counter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counter); + Assertions.assertEquals(0, counter.getCardinality()); + } + + @Test + public void testAggregateWithNonNullCounter() + { + aggregator.init(buffer, POSITION_1); + RoaringBitmap64Counter inputCounter = new RoaringBitmap64Counter(); + inputCounter.add(123L); + + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, resultCounter.getCardinality()); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateWithNullCounterFromSelector() + { + aggregator.init(buffer, POSITION_1); + Bitmap64 initialCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + long initialCardinality = initialCounter.getCardinality(); + + EasyMock.expect(mockSelector.getObject()).andReturn(null).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); // Should not throw NPE, and not change the counter + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals( + initialCardinality, + resultCounter.getCardinality(), + "Aggregating null from selector should not change cardinality count" + ); + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateMultipleCounters() + { + aggregator.init(buffer, POSITION_1); + RoaringBitmap64Counter inputCounter1 = new RoaringBitmap64Counter(); + inputCounter1.add(10L); + RoaringBitmap64Counter inputCounter2 = new RoaringBitmap64Counter(); + inputCounter2.add(20L); + inputCounter2.add(10L); // duplicate + + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter1).once(); + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter2).once(); + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_1); + + Bitmap64 resultCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(2, resultCounter.getCardinality()); // 10, 20 + + EasyMock.verify(mockSelector); + } + + @Test + public void testAggregateAtDifferentPositions() + { + aggregator.init(buffer, POSITION_1); + aggregator.init(buffer, POSITION_2); + + RoaringBitmap64Counter counterForPos1 = new RoaringBitmap64Counter(); + counterForPos1.add(1L); + RoaringBitmap64Counter counterForPos2 = new RoaringBitmap64Counter(); + counterForPos2.add(2L); + + EasyMock.expect(mockSelector.getObject()).andReturn(counterForPos1).once(); // For POSITION_1 + EasyMock.expect(mockSelector.getObject()).andReturn(counterForPos2).once(); // For POSITION_2 + EasyMock.replay(mockSelector); + + aggregator.aggregate(buffer, POSITION_1); + aggregator.aggregate(buffer, POSITION_2); + + Bitmap64 result1 = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, result1.getCardinality()); + + Bitmap64 result2 = (Bitmap64) aggregator.get(buffer, POSITION_2); + Assertions.assertEquals(1, result2.getCardinality()); + + Assertions.assertNotSame(result1, result2); + EasyMock.verify(mockSelector); + } + + @Test + public void testCloseClearsCache() + { + aggregator.init(buffer, POSITION_1); + RoaringBitmap64Counter inputCounter = new RoaringBitmap64Counter(); + inputCounter.add(1L); + EasyMock.expect(mockSelector.getObject()).andReturn(inputCounter).once(); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + EasyMock.verify(mockSelector); + + Bitmap64 counterBeforeClose = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertNotNull(counterBeforeClose); + + aggregator.close(); + + // After close, accessing the same buffer and position might throw NPE or return null + // depending on how IdentityHashMap and Int2ObjectMap behave after clearing. + // The current implementation of get() would lead to NPE if buffer is not in counterCache. + Assertions.assertThrows( + NullPointerException.class, + () -> aggregator.get(buffer, POSITION_1), + "Accessing counter from cleared cache should fail or return null" + ); + } + + @Test + public void testRelocateSameBuffer() + { + aggregator.init(buffer, POSITION_1); + RoaringBitmap64Counter initialCounterVal = new RoaringBitmap64Counter(); + initialCounterVal.add(123L); + + EasyMock.expect(mockSelector.getObject()).andReturn(initialCounterVal).once(); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + EasyMock.verify(mockSelector); + + Bitmap64 originalCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, originalCounter.getCardinality()); + + aggregator.relocate(POSITION_1, POSITION_2, buffer, buffer); + + Bitmap64 newCounter = (Bitmap64) aggregator.get(buffer, POSITION_2); + Assertions.assertNotNull(newCounter); + Assertions.assertEquals(1, newCounter.getCardinality()); + Assertions.assertSame(originalCounter, newCounter); + } + + @Test + public void testRelocateDifferentBuffers() + { + aggregator.init(buffer, POSITION_1); + RoaringBitmap64Counter initialCounterVal = new RoaringBitmap64Counter(); + initialCounterVal.add(456L); + + EasyMock.expect(mockSelector.getObject()).andReturn(initialCounterVal).once(); + EasyMock.replay(mockSelector); + aggregator.aggregate(buffer, POSITION_1); + EasyMock.verify(mockSelector); + + Bitmap64 originalCounter = (Bitmap64) aggregator.get(buffer, POSITION_1); + Assertions.assertEquals(1, originalCounter.getCardinality()); + + ByteBuffer newBuffer = ByteBuffer.allocate(BUFFER_CAPACITY); + aggregator.relocate(POSITION_1, POSITION_1, buffer, newBuffer); + + Bitmap64 newCounter = (Bitmap64) aggregator.get(newBuffer, POSITION_1); + Assertions.assertNotNull(newCounter); + Assertions.assertEquals(1, newCounter.getCardinality()); + Assertions.assertSame(originalCounter, newCounter); + } + + @Test + public void testUnsupportedGetOperations() + { + aggregator.init(buffer, POSITION_1); // Ensure the entry exists in cache to avoid NPE on get() + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getLong(buffer, POSITION_1)); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getDouble(buffer, POSITION_1)); + Assertions.assertThrows(UnsupportedOperationException.class, () -> aggregator.getFloat(buffer, POSITION_1)); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModuleTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModuleTest.java new file mode 100644 index 000000000000..697400d79b84 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountModuleTest.java @@ -0,0 +1,46 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import org.apache.druid.segment.serde.ComplexMetricSerde; +import org.apache.druid.segment.serde.ComplexMetrics; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class Bitmap64ExactCountModuleTest +{ + @Test + public void testRegisterSerde() + { + Bitmap64ExactCountModule.registerSerde(); + + ComplexMetricSerde typeNameSerde = ComplexMetrics.getSerdeForType(Bitmap64ExactCountModule.TYPE_NAME); + Assertions.assertNotNull(typeNameSerde); + Assertions.assertInstanceOf(Bitmap64ExactCountMergeComplexMetricSerde.class, typeNameSerde); + + ComplexMetricSerde buildTypeNameSerde = ComplexMetrics.getSerdeForType(Bitmap64ExactCountModule.BUILD_TYPE_NAME); + Assertions.assertNotNull(buildTypeNameSerde); + Assertions.assertInstanceOf(Bitmap64ExactCountBuildComplexMetricSerde.class, buildTypeNameSerde); + + ComplexMetricSerde mergeTypeNameSerde = ComplexMetrics.getSerdeForType(Bitmap64ExactCountModule.MERGE_TYPE_NAME); + Assertions.assertNotNull(mergeTypeNameSerde); + Assertions.assertInstanceOf(Bitmap64ExactCountMergeComplexMetricSerde.class, mergeTypeNameSerde); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregatorTest.java new file mode 100644 index 000000000000..bd84f174cc37 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/Bitmap64ExactCountPostAggregatorTest.java @@ -0,0 +1,159 @@ +/* + * 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.aggregation.exact.count.bitmap64; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.PostAggregatorIds; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +public class Bitmap64ExactCountPostAggregatorTest +{ + private static final String NAME = "postAggTestName"; + private static final String FIELD_NAME = "postAggTestFieldName"; + + private Bitmap64ExactCountPostAggregator postAggregator; + + @BeforeEach + public void setUp() + { + postAggregator = new Bitmap64ExactCountPostAggregator(NAME, FIELD_NAME); + } + + @Test + public void testConstructor() + { + Assertions.assertEquals(NAME, postAggregator.getName()); + Assertions.assertEquals(FIELD_NAME, postAggregator.getFieldName()); + } + + @Test + public void testGetDependentFields() + { + Assertions.assertEquals(Collections.singleton(FIELD_NAME), postAggregator.getDependentFields()); + } + + @Test + public void testGetComparator() + { + Assertions.assertEquals(ArithmeticPostAggregator.DEFAULT_COMPARATOR, postAggregator.getComparator()); + } + + @Test + public void testCompute() + { + Bitmap64 counter = new RoaringBitmap64Counter(); + counter.add(1L); + counter.add(2L); + counter.add(3L); + + Map combinedAggregators = ImmutableMap.of(FIELD_NAME, counter); + Assertions.assertEquals(3L, postAggregator.compute(combinedAggregators)); + + Bitmap64 emptyCounter = new RoaringBitmap64Counter(); + Map combinedAggregatorsEmpty = ImmutableMap.of(FIELD_NAME, emptyCounter); + Assertions.assertEquals(0L, postAggregator.compute(combinedAggregatorsEmpty)); + } + + @Test + public void testGetType() + { + Assertions.assertEquals(ColumnType.LONG, postAggregator.getType(null)); + } + + @Test + public void testDecorate() + { + Assertions.assertSame(postAggregator, postAggregator.decorate(Collections.emptyMap())); + } + + @Test + public void testGetCacheKey() + { + byte[] expectedKey = new CacheKeyBuilder(PostAggregatorIds.BITMAP64_EXACT_COUNT_TYPE_ID) + .appendString(FIELD_NAME) + .build(); + Assertions.assertArrayEquals(expectedKey, postAggregator.getCacheKey()); + + Bitmap64ExactCountPostAggregator postAggregator2 = new Bitmap64ExactCountPostAggregator( + NAME, + FIELD_NAME + ); + Assertions.assertArrayEquals(postAggregator.getCacheKey(), postAggregator2.getCacheKey()); + + Bitmap64ExactCountPostAggregator postAggregatorDiffFieldName = new Bitmap64ExactCountPostAggregator( + NAME, + FIELD_NAME + + "_diff" + ); + Assertions.assertFalse(Arrays.equals(postAggregator.getCacheKey(), postAggregatorDiffFieldName.getCacheKey())); + + Bitmap64ExactCountPostAggregator postAggregatorDiffName = new Bitmap64ExactCountPostAggregator( + NAME + + "_diff", + FIELD_NAME + ); + // Cache key for PostAggregator does not include its own name, only dependent fieldName + Assertions.assertArrayEquals(postAggregator.getCacheKey(), postAggregatorDiffName.getCacheKey()); + } + + @Test + public void testEqualsAndHashCode() + { + Bitmap64ExactCountPostAggregator pa1 = new Bitmap64ExactCountPostAggregator(NAME, FIELD_NAME); + Bitmap64ExactCountPostAggregator pa2 = new Bitmap64ExactCountPostAggregator(NAME, FIELD_NAME); + Bitmap64ExactCountPostAggregator paDiffName = new Bitmap64ExactCountPostAggregator( + NAME + "_diff", + FIELD_NAME + ); + Bitmap64ExactCountPostAggregator paDiffFieldName = new Bitmap64ExactCountPostAggregator( + NAME, + FIELD_NAME + + "_diff" + ); + + Assertions.assertEquals(pa1, pa2); + Assertions.assertEquals(pa1.hashCode(), pa2.hashCode()); + + Assertions.assertNotEquals(null, pa1); // Null comparison + Assertions.assertNotEquals(new Object(), pa1); // Different type + + Assertions.assertNotEquals(pa1, paDiffName); + Assertions.assertNotEquals(pa1.hashCode(), paDiffName.hashCode()); + + Assertions.assertNotEquals(pa1, paDiffFieldName); + Assertions.assertNotEquals(pa1.hashCode(), paDiffFieldName.hashCode()); + } + + @Test + public void testToString() + { + String expected = "Bitmap64ExactCountPostAggregator{name='" + NAME + "', field=" + FIELD_NAME + "}"; + Assertions.assertEquals(expected, postAggregator.toString()); + } +} diff --git a/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregatorTest.java b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregatorTest.java new file mode 100644 index 000000000000..7f9bf0c4db03 --- /dev/null +++ b/extensions-contrib/druid-exact-count-bitmap/src/test/java/org/apache/druid/query/aggregation/exact/count/bitmap64/sql/Bitmap64ExactCountSqlAggregatorTest.java @@ -0,0 +1,244 @@ +/* + * 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.aggregation.exact.count.bitmap64.sql; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.exact.count.bitmap64.Bitmap64ExactCountBuildAggregatorFactory; +import org.apache.druid.query.aggregation.exact.count.bitmap64.Bitmap64ExactCountMergeAggregatorFactory; +import org.apache.druid.query.aggregation.exact.count.bitmap64.Bitmap64ExactCountModule; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; +import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; +import org.apache.druid.sql.calcite.TempDirProducer; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.calcite.util.DruidModuleCollection; +import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; +import org.apache.druid.sql.calcite.util.TestDataBuilder; +import org.apache.druid.sql.calcite.util.datasets.TestDataSet; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +@SqlTestFrameworkConfig.ComponentSupplier(Bitmap64ExactCountSqlAggregatorTest.Bitmap64ExactCountSqlAggComponentSupplier.class) +public class Bitmap64ExactCountSqlAggregatorTest extends BaseCalciteQueryTest +{ + private static final String DATA_SOURCE = "numfoo"; + + public static class Bitmap64ExactCountSqlAggComponentSupplier extends StandardComponentSupplier + { + public Bitmap64ExactCountSqlAggComponentSupplier(TempDirProducer tempFolderProducer) + { + super(tempFolderProducer); + } + + @Override + public DruidModule getCoreModule() + { + return DruidModuleCollection.of(super.getCoreModule(), new Bitmap64ExactCountModule()); + } + + @Override + public SpecificSegmentsQuerySegmentWalker addSegmentsToWalker(SpecificSegmentsQuerySegmentWalker walker) + { + Bitmap64ExactCountModule.registerSerde(); + final QueryableIndex index = + IndexBuilder.create(CalciteTests.getJsonMapper()) + .tmpDir(tempDirProducer.newTempFolder()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema( + IncrementalIndexSchema.builder() + .withDimensionsSpec(TestDataSet.NUMFOO.getInputRowSchema() + .getDimensionsSpec()) + .withMetrics( + new Bitmap64ExactCountBuildAggregatorFactory( + "unique_m1_values", + "m1" + ), + new HyperUniquesAggregatorFactory( + "hyper_unique_m1", + "m1" + ) + ) + .withRollup(false) + .build() + ) + .rows(TestDataBuilder.ROWS1_WITH_NUMERIC_DIMS) + .buildMMappedIndex(); + + return walker.add( + DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(index.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + index + ); + } + } + + @Test + public void testExactCountOnStringColumnTypeThrowsError() + { + cannotVectorize(); + DruidException e = Assertions.assertThrows( + DruidException.class, + () -> testQuery( + "SELECT BITMAP64_EXACT_COUNT(dim1) FROM " + DATA_SOURCE, + ImmutableList.of(), + ImmutableList.of() + ) + ); + + Assertions.assertTrue(e.getMessage().contains("Aggregation [BITMAP64_EXACT_COUNT] does not support type [STRING]")); + } + + @Test + public void testExactCountOnHyperUniqueColumnTypeThrowsError() + { + DruidException e = Assertions.assertThrows( + DruidException.class, + () -> testQuery( + "SELECT BITMAP64_EXACT_COUNT(hyper_unique_m1) FROM " + DATA_SOURCE, + ImmutableList.of(), + ImmutableList.of() + ) + ); + + Assertions.assertTrue(e.getMessage().contains( + "Cannot apply 'BITMAP64_EXACT_COUNT' to arguments of type 'BITMAP64_EXACT_COUNT(>)'")); + } + + @Test + public void testExactCountOnPreAggregatedColumn() + { + cannotVectorize(); + String sql = "SELECT BITMAP64_EXACT_COUNT(unique_m1_values) FROM " + DATA_SOURCE; + testQuery( + sql, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators( + ImmutableList.of( + new Bitmap64ExactCountMergeAggregatorFactory( + "a0", + "unique_m1_values" + ) + ) + ) + .context(Collections.emptyMap()) + .build() + ), + ImmutableList.of( + new Object[]{6L} // m1 string inputs: "1.0"-"6.0" are 6 unique values + ) + ); + } + + @Test + public void testExactCountWithGroupBy() + { + cannotVectorize(); + String sql = "SELECT __time, BITMAP64_EXACT_COUNT(l1) FROM " + DATA_SOURCE + " " + + "GROUP BY __time " + + "ORDER BY __time"; + testQuery( + sql, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setDimensions(ImmutableList.of(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) + .setAggregatorSpecs( + ImmutableList.of( + new Bitmap64ExactCountBuildAggregatorFactory("a0", "l1") + ) + ) + .setContext(Collections.emptyMap()) + .build() + ), + ImmutableList.of( + new Object[]{946684800000L, 1L}, // 2000-01-01, l1=7L + new Object[]{946771200000L, 1L}, // 2000-01-02, l1=325323L + new Object[]{946857600000L, 1L}, // 2000-01-03, l1=0L + new Object[]{978307200000L, 0L}, // 2001-01-01, l1 is null + new Object[]{978393600000L, 0L}, // 2001-01-02, l1 is null + new Object[]{978480000000L, 0L} // 2001-01-03, l1 is null + ) + ); + } + + @Test + public void testExactCountOnPreAggregatedWithGroupBy() + { + cannotVectorize(); + String sql = "SELECT __time, BITMAP64_EXACT_COUNT(unique_m1_values) FROM " + DATA_SOURCE + " " + + "GROUP BY __time " + + "ORDER BY __time"; + + testQuery( + sql, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE) + .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .setGranularity(Granularities.ALL) + .setDimensions(ImmutableList.of(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) + .setAggregatorSpecs( + ImmutableList.of( + new Bitmap64ExactCountMergeAggregatorFactory("a0", "unique_m1_values") + ) + ) + .setContext(Collections.emptyMap()) + .build() + ), + ImmutableList.of( + new Object[]{946684800000L, 1L}, // 2000-01-01, m1="1.0" + new Object[]{946771200000L, 1L}, // 2000-01-02, m1="2.0" + new Object[]{946857600000L, 1L}, // 2000-01-03, m1="3.0" + new Object[]{978307200000L, 1L}, // 2001-01-01, m1="4.0" + new Object[]{978393600000L, 1L}, // 2001-01-02, m1="5.0" + new Object[]{978480000000L, 1L} // 2001-01-03, m1="6.0" + ) + ); + } +} diff --git a/integration-tests-ex/cases/cluster/DruidExactCountBitmap/docker-compose.py b/integration-tests-ex/cases/cluster/DruidExactCountBitmap/docker-compose.py new file mode 100644 index 000000000000..8cb9b56fddff --- /dev/null +++ b/integration-tests-ex/cases/cluster/DruidExactCountBitmap/docker-compose.py @@ -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. + +from template import BaseTemplate, generate +from template import COORDINATOR, MIDDLE_MANAGER + +class Template(BaseTemplate): + def extend_druid_service(self, service): + self.add_env(service, 'druid_test_loadList', 'druid-exact-count-bitmap') + + def define_coordinator(self): + service_name = COORDINATOR + service = self.define_master_service(service_name, COORDINATOR) + self.add_env(service, 'druid_host', service_name) + self.add_env(service, 'druid_manager_segments_pollDuration', 'PT5S') + self.add_env(service, 'druid_coordinator_period', 'PT10S') + + def define_indexer(self): + ''' + Override the indexer to MIDDLE_MANAGER + ''' + return self.define_std_indexer(MIDDLE_MANAGER) + +generate(__file__, Template()) diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 23ebaccb4064..36ed8b439086 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -259,6 +259,12 @@ ${project.parent.version} provided + + org.apache.druid.extensions.contrib + druid-exact-count-bitmap + ${project.parent.version} + provided + org.apache.commons commons-lang3 @@ -282,6 +288,11 @@ junit test + + org.junit.jupiter + junit-jupiter-api + test + pl.pragmatists JUnitParams @@ -552,5 +563,14 @@ Security + + IT-DruidExactCountBitmap + + false + + + DruidExactCountBitmap + + diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/DruidExactCountBitmapTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/DruidExactCountBitmapTest.java new file mode 100644 index 000000000000..74446232ced5 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/DruidExactCountBitmapTest.java @@ -0,0 +1,205 @@ +/* + * 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.testsEx.DruidExactCountBitmap; + +import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.utils.ITRetryUtil; +import org.apache.druid.testing.utils.KafkaAdminClient; +import org.apache.druid.testing.utils.KafkaEventWriter; +import org.apache.druid.testing.utils.KafkaUtil; +import org.apache.druid.testing.utils.StreamEventWriter; +import org.apache.druid.testsEx.indexer.AbstractIndexerTest; +import org.joda.time.Interval; +import org.junit.Test; + +import java.io.BufferedReader; +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.Properties; +import java.util.UUID; +import java.util.function.Function; + +public class DruidExactCountBitmapTest extends AbstractIndexerTest +{ + private static final Logger LOG = new Logger(DruidExactCountBitmapTest.class); + private static final String SUPERVISOR_TEMPLATE = "/druid-exact-count-bitmap/kafka_supervisor_template.json"; + private static final String QUERY_FILE = "/druid-exact-count-bitmap/queries.json"; + private static final String DATA_FILE = "/druid-exact-count-bitmap/data.json"; + private static final String DATASOURCE = "wikipedia_index_test"; + + @Test + public void testQuery() throws Exception + { + final String fullDatasourceName = DATASOURCE + config.getExtraDatasourceNameSuffix(); + String baseName = fullDatasourceName + UUID.randomUUID(); + KafkaAdminClient streamAdminClient = new KafkaAdminClient(config); + List supervisors = new ArrayList<>(); + + final int numDatasources = 3; + for (int i = 0; i < numDatasources; i++) { + String datasource = baseName + "-" + i; + streamAdminClient.createStream(datasource, 1, Collections.emptyMap()); + ITRetryUtil.retryUntil( + () -> streamAdminClient.isStreamActive(datasource), + true, + 10000, + 30, + "Wait for stream active" + ); + String supervisorSpec = generateStreamIngestionPropsTransform( + datasource, + datasource, + config + ).apply(getResourceAsString(SUPERVISOR_TEMPLATE)); + LOG.info("supervisorSpec: [%s]\n", supervisorSpec); + // Start supervisor + String specResponse = indexer.submitSupervisor(supervisorSpec); + LOG.info("Submitted supervisor [%s]", specResponse); + supervisors.add(specResponse); + + int ctr = 0; + try ( + StreamEventWriter streamEventWriter = new KafkaEventWriter(config, false); + BufferedReader reader = new BufferedReader( + new InputStreamReader(getResourceAsStream(DATA_FILE), StandardCharsets.UTF_8) + ) + ) { + String line; + while ((line = reader.readLine()) != null) { + streamEventWriter.write(datasource, StringUtils.toUtf8(line)); + ctr++; + } + } + final int numWritten = ctr; + + LOG.info("Waiting for stream indexing tasks to consume events"); + + ITRetryUtil.retryUntilTrue( + () -> + numWritten == this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + StringUtils.format( + "dataSource[%s] consumed [%,d] events, expected [%,d]", + datasource, + this.queryHelper.countRows( + datasource, + Intervals.ETERNITY, + name -> new LongSumAggregatorFactory(name, "count") + ), + numWritten + ) + ); + } + + String queryResponseTemplate = StringUtils.replace( + getResourceAsString(QUERY_FILE), + "%%DATASOURCE%%", + baseName + ); + + queryHelper.testQueriesFromString(queryResponseTemplate); + + + for (int i = 0; i < numDatasources; i++) { + indexer.terminateSupervisor(supervisors.get(i)); + streamAdminClient.deleteStream(baseName + "-" + i); + } + + for (int i = 0; i < numDatasources; i++) { + final int datasourceNumber = i; + ITRetryUtil.retryUntil( + () -> coordinator.areSegmentsLoaded(baseName + "-" + datasourceNumber), + true, + 10000, + 10, + "Kafka segments loaded" + ); + } + + queryHelper.testQueriesFromString(queryResponseTemplate); + + for (int i = 0; i < numDatasources; i++) { + final String datasource = baseName + "-" + i; + List intervals = coordinator.getSegmentIntervals(datasource); + + Collections.sort(intervals); + String first = intervals.get(0).split("/")[0]; + String last = intervals.get(intervals.size() - 1).split("/")[1]; + Interval interval = Intervals.of(first + "/" + last); + coordinator.unloadSegmentsForDataSource(baseName + "-" + i); + ITRetryUtil.retryUntilFalse( + () -> coordinator.areSegmentsLoaded(datasource), + "Segment Unloading" + ); + coordinator.deleteSegmentsDataSource(baseName + "-" + i, interval); + } + } + + /** + * sad version of + * {@link org.apache.druid.tests.indexer.AbstractKafkaIndexingServiceTest#generateStreamIngestionPropsTransform} + */ + private Function generateStreamIngestionPropsTransform( + String streamName, + String fullDatasourceName, + IntegrationTestingConfig config + ) + { + final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); + final Properties consumerProperties = new Properties(); + consumerProperties.putAll(consumerConfigs); + consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); + KafkaUtil.addPropertiesFromTestConfig(config, consumerProperties); + return spec -> { + try { + spec = StringUtils.replace( + spec, + "%%DATASOURCE%%", + fullDatasourceName + ); + spec = StringUtils.replace( + spec, + "%%TOPIC_VALUE%%", + streamName + ); + return StringUtils.replace( + spec, + "%%STREAM_PROPERTIES_VALUE%%", + jsonMapper.writeValueAsString(consumerProperties) + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + }; + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/ITDruidExactCountBitmapTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/ITDruidExactCountBitmapTest.java new file mode 100644 index 000000000000..212415156578 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/DruidExactCountBitmap/ITDruidExactCountBitmapTest.java @@ -0,0 +1,31 @@ +/* + * 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.testsEx.DruidExactCountBitmap; + +import org.apache.druid.testsEx.categories.DruidExactCountBitmap; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +@RunWith(DruidTestRunner.class) +@Category(DruidExactCountBitmap.class) +public class ITDruidExactCountBitmapTest extends DruidExactCountBitmapTest +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/DruidExactCountBitmap.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/DruidExactCountBitmap.java new file mode 100644 index 000000000000..42799ff2b485 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/DruidExactCountBitmap.java @@ -0,0 +1,24 @@ +/* + * 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.testsEx.categories; + +public class DruidExactCountBitmap +{ +} diff --git a/integration-tests-ex/cases/src/test/resources/cluster/DruidExactCountBitmap/docker.yaml b/integration-tests-ex/cases/src/test/resources/cluster/DruidExactCountBitmap/docker.yaml new file mode 100644 index 000000000000..b001a96db08a --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/cluster/DruidExactCountBitmap/docker.yaml @@ -0,0 +1,76 @@ +# 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. +#------------------------------------------------------------------------- + +# Definition of the query test cluster. +# See https://yaml.org/spec/1.2.2 for more about YAML +include: + - /cluster/Common/zk-metastore.yaml + - /cluster/Common/kafka.yaml + +druid: + coordinator: + instances: + - port: 8081 + overlord: + instances: + - port: 8090 + broker: + instances: + - port: 8082 + router: + instances: + - port: 8888 + historical: + instances: + - port: 8083 + middlemanager: + instances: + - port: 8091 + +# Metastore initialization queries. +# REPLACE is used so that the statements are idempotent +# The fancy formatting is for human consumption, it is compacted internally +metastoreInit: + - sql: | + REPLACE INTO druid_segments ( + id, dataSource, created_date, start, end, partitioned, version, used, payload + ) + VALUES ( + 'wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z', + 'wikipedia', + '2013-08-08T21:26:23.799Z', + '2013-08-01T00:00:00.000Z', + '2013-08-02T00:00:00.000Z', + 0, + '2013-08-08T21:22:48.989Z', + 1, + '{"dataSource": "wikipedia", + "interval": "2013-08-01T00:00:00.000Z/2013-08-02T00:00:00.000Z", + "version": "2013-08-08T21:22:48.989Z", + "loadSpec": { + "type": "s3_zip", + "bucket": "static.druid.io", + "key": "data/segments/wikipedia/20130801T000000.000Z_20130802T000000.000Z/2013-08-08T21_22_48.989Z/0/index.zip" + }, + "dimensions": "dma_code,continent_code,geo,area_code,robot,country_name,network,city,namespace, + anonymous,unpatrolled,page,postal_code,language,newpage,user,region_lookup", + "metrics": "count,delta,variation,added,deleted", + "shardSpec": {"type": "none"}, + "binaryVersion": 9, + "size": 24664730, + "identifier": "wikipedia_2013-08-01T00:00:00.000Z_2013-08-02T00:00:00.000Z_2013-08-08T21:22:48.989Z" + }' + ) diff --git a/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/data.json b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/data.json new file mode 100644 index 000000000000..f39dcd7c1041 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/data.json @@ -0,0 +1,10 @@ +{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 459, "deleted": 200, "delta": -143} +{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} +{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 459, "deleted": 200, "delta": -143} +{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330} +{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111} +{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900} +{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9} diff --git a/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/kafka_supervisor_template.json b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/kafka_supervisor_template.json new file mode 100644 index 000000000000..bd9dd3f111f9 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/kafka_supervisor_template.json @@ -0,0 +1,74 @@ +{ + "type": "kafka", + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "page", + "language", + "user", + "unpatrolled", + "newPage", + "robot", + "anonymous", + "namespace", + "continent", + "country", + "region", + "city" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + }, + { + "type": "Bitmap64ExactCountBuild", + "name": "unique_deleted", + "fieldName": "deleted" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "DAY", + "queryGranularity": "second" + } + }, + "tuningConfig": { + "type": "kafka", + "intermediatePersistPeriod": "PT1H", + "maxRowsPerSegment": 5000000, + "maxRowsInMemory": 500000 + }, + "ioConfig": { + "topic": "%%TOPIC_VALUE%%", + "consumerProperties": %%STREAM_PROPERTIES_VALUE%%, + "taskCount": 2, + "replicas": 1, + "taskDuration": "PT120S", + "useEarliestOffset": true, + "inputFormat" : {"type": "json"} + } +} diff --git a/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/queries.json b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/queries.json new file mode 100644 index 000000000000..cd7e90af3fe8 --- /dev/null +++ b/integration-tests-ex/cases/src/test/resources/druid-exact-count-bitmap/queries.json @@ -0,0 +1,115 @@ +[ + { + "description": "Timeseries query on non-rolled-up columns", + "query": { + "queryType": "timeseries", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "added_count", + "fieldName": "added" + }, + { + "type": "Bitmap64ExactCountBuild", + "name": "added_cardinality", + "fieldName": "added" + } + ] + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "added_count": 15, + "added_cardinality": 4 + } + } + ] + }, + { + "description": "Timeseries query on rolled-up columns", + "query": { + "queryType": "timeseries", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "deleted_count", + "fieldName": "deleted" + }, + { + "type": "Bitmap64ExactCountBuild", + "name": "deleted_cardinality", + "fieldName": "deleted" + }, + { + "type": "Bitmap64ExactCountMerge", + "name": "deleted_cardinality_from_rollup_columns", + "fieldName": "unique_deleted" + } + ] + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "deleted_count": 15, + "deleted_cardinality": 5, + "deleted_cardinality_from_rollup_columns": 5 + } + } + ] + }, + { + "description": "Bitmap64 works on time column", + "query": { + "queryType": "timeseries", + "dataSource": { + "type": "union", + "dataSources": [ + "%%DATASOURCE%%-1", "%%DATASOURCE%%-2", "%%DATASOURCE%%-3", + "%%DATASOURCE%%-0" + ] + }, + "intervals": ["2013-08-31/2013-09-01"], + "granularity": "all", + "aggregations": [ + { + "type": "count", + "name": "time_count" + }, + { + "type": "Bitmap64ExactCountBuild", + "name": "time_cardinality", + "fieldName": "__time" + } + ] + }, + "expectedResults": [ + { + "timestamp": "2013-08-31T01:02:33.000Z", + "result": { + "time_count": 15, + "time_cardinality": 5 + } + } + ] + } +] diff --git a/it.sh b/it.sh index e322954827ba..2e13329a14ac 100755 --- a/it.sh +++ b/it.sh @@ -229,13 +229,13 @@ case $CMD in usage ;; "ci" ) - mvn -q clean install -P dist $MAVEN_IGNORE + mvn -q clean install -P dist,bundle-contrib-exts $MAVEN_IGNORE ;; "build" ) - mvn -B clean install -P dist $MAVEN_IGNORE -T1.0C $* + mvn -B clean install -P dist,bundle-contrib-exts $MAVEN_IGNORE -T1.0C $* ;; "dist" ) - mvn -B install -P dist $MAVEN_IGNORE -pl :distribution + mvn -B install -P dist,bundle-contrib-exts $MAVEN_IGNORE -pl :distribution ;; "tools" ) mvn -B install -pl :druid-it-tools diff --git a/pom.xml b/pom.xml index 1e45bf7f605d..4ba1872de6f5 100644 --- a/pom.xml +++ b/pom.xml @@ -235,6 +235,7 @@ extensions-contrib/cloudfiles-extensions extensions-contrib/graphite-emitter extensions-contrib/distinctcount + extensions-contrib/druid-exact-count-bitmap extensions-contrib/statsd-emitter extensions-contrib/time-min-max extensions-contrib/virtual-columns diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index c4c9a7875ef0..91d39402e778 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -167,6 +167,10 @@ public class AggregatorUtil // DDSketch aggregator public static final byte DDSKETCH_CACHE_TYPE_ID = 0x50; + // Bitmap64 exact cardinality count aggregator + public static final byte BITMAP64_EXACT_COUNT_BUILD_CACHE_TYPE_ID = 0x60; + public static final byte BITMAP64_EXACT_COUNT_MERGE_CACHE_TYPE_ID = 0x61; + /** * Given a list of PostAggregators and the name of an output column, returns the minimal list of PostAggregators * required to compute the output column. diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java index 9d097b6e4f29..44203e92702c 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java @@ -70,4 +70,5 @@ public class PostAggregatorIds public static final byte SPECTATOR_HISTOGRAM_SKETCH_PERCENTILES_CACHE_TYPE_ID = 46; public static final byte DDSKETCH_QUANTILES_TYPE_ID = 51; public static final byte DDSKETCH_QUANTILE_TYPE_ID = 52; + public static final byte BITMAP64_EXACT_COUNT_TYPE_ID = 53; } diff --git a/website/.spelling b/website/.spelling index 65d99797fe3e..d4dc2069f087 100644 --- a/website/.spelling +++ b/website/.spelling @@ -470,6 +470,7 @@ pre-aggregating pre-aggregation pre-computation pre-compute +pre-computed pre-computing preconfigured pre-existing @@ -2436,6 +2437,10 @@ quantileFromDDSketch collapsingLowestDense snapshotVersion +- ../docs/development/extensions-contrib/druid-exact-count-bitmap.md +Bitmap64ExactCountBuild +Bitmap64ExactCountMerge + - ../docs/development/extensions-core/catalog.md ColumnSpec PropertyKeyName