-
Notifications
You must be signed in to change notification settings - Fork 1.1k
Add APPROX_COUNT_DISTINCT Function #15338
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
JackieTien97
merged 18 commits into
apache:master
from
FearfulTomcat27:approx_count_distinct
Apr 25, 2025
Merged
Changes from all commits
Commits
Show all changes
18 commits
Select commit
Hold shift + click to select a range
edf8fff
style: add approx_count_distinct support
FearfulTomcat27 03a9421
style: add approx_count_distinct BE
FearfulTomcat27 8d5406b
fix: Error message for specification approx_count_distinct
FearfulTomcat27 951cc42
feat: lazy load for hyperloglog
FearfulTomcat27 7fcc907
feat: Add support for the addIntermediate function to the Approx_coun…
FearfulTomcat27 eb080af
Add APPROX_COUNT_DISTINCT IT
FearfulTomcat27 7bb99ac
fix: Checksum Approx_count_distinct The second argument must be a Num…
FearfulTomcat27 4aca109
fix: Add HyperLogLogBigArray and delete unused class
FearfulTomcat27 4498846
fix: Modify the type validation of the second parameter for Approx_co…
FearfulTomcat27 91c0a9f
fix: Fix calculation errors in Approx_count_distinct when different d…
FearfulTomcat27 96b7eb2
fix: Add approx_count_distinct in GROUP BY situation for IT.
FearfulTomcat27 b1484bb
fix: Optimize HyperLogLog merging and state management in ApproxCount…
FearfulTomcat27 22de508
fix: Remove unused constructors and initial value handling in BinaryB…
FearfulTomcat27 2d9713e
fix: Refactor ApproxCountDistinctAccumulator and related classes for …
FearfulTomcat27 a57d289
fix: Improve variable initialization and documentation formatting in …
FearfulTomcat27 abcaab4
fix: Clean up code by removing unnecessary print statements and simpl…
FearfulTomcat27 79ebed7
fix: Fix the issue where HyperLogLogBigArray does not update sizeOfHy…
FearfulTomcat27 aff9346
fix: Update SQL queries in IoTDBTableAggregationIT to order results b…
FearfulTomcat27 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
265 changes: 265 additions & 0 deletions
265
...gine/execution/operator/source/relational/aggregation/ApproxCountDistinctAccumulator.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,265 @@ | ||
| /* | ||
| * Licensed 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.iotdb.db.queryengine.execution.operator.source.relational.aggregation; | ||
|
|
||
| import org.apache.tsfile.block.column.Column; | ||
| import org.apache.tsfile.block.column.ColumnBuilder; | ||
| import org.apache.tsfile.enums.TSDataType; | ||
| import org.apache.tsfile.file.metadata.statistics.Statistics; | ||
| import org.apache.tsfile.read.common.block.column.BinaryColumnBuilder; | ||
| import org.apache.tsfile.utils.Binary; | ||
| import org.apache.tsfile.utils.RamUsageEstimator; | ||
| import org.apache.tsfile.write.UnSupportedDataTypeException; | ||
|
|
||
| import static com.google.common.base.Preconditions.checkArgument; | ||
| import static org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.HyperLogLog.DEFAULT_STANDARD_ERROR; | ||
|
|
||
| public class ApproxCountDistinctAccumulator implements TableAccumulator { | ||
| private static final long INSTANCE_SIZE = | ||
| RamUsageEstimator.shallowSizeOfInstance(ApproxCountDistinctAccumulator.class); | ||
| private final TSDataType seriesDataType; | ||
| private final HyperLogLogStateFactory.SingleHyperLogLogState state = | ||
| HyperLogLogStateFactory.createSingleState(); | ||
|
|
||
| private static final int DEFAULT_HYPERLOGLOG_BUCKET_SIZE = 2048; | ||
|
|
||
| public ApproxCountDistinctAccumulator(TSDataType seriesDataType) { | ||
| this.seriesDataType = seriesDataType; | ||
| } | ||
|
|
||
| @Override | ||
| public long getEstimatedSize() { | ||
| return INSTANCE_SIZE | ||
| + RamUsageEstimator.shallowSizeOfInstance(HyperLogLog.class) | ||
| + Integer.BYTES * DEFAULT_HYPERLOGLOG_BUCKET_SIZE; | ||
| } | ||
|
|
||
| @Override | ||
| public TableAccumulator copy() { | ||
| return new ApproxCountDistinctAccumulator(seriesDataType); | ||
| } | ||
|
|
||
| @Override | ||
| public void addInput(Column[] arguments, AggregationMask mask) { | ||
| double maxStandardError = | ||
| arguments.length == 1 ? DEFAULT_STANDARD_ERROR : arguments[1].getDouble(0); | ||
| HyperLogLog hll = getOrCreateHyperLogLog(state, maxStandardError); | ||
|
|
||
| switch (seriesDataType) { | ||
| case INT32: | ||
| case DATE: | ||
| addIntInput(arguments[0], mask, hll); | ||
| return; | ||
| case INT64: | ||
| case TIMESTAMP: | ||
| addLongInput(arguments[0], mask, hll); | ||
| return; | ||
| case FLOAT: | ||
| addFloatInput(arguments[0], mask, hll); | ||
| return; | ||
| case DOUBLE: | ||
| addDoubleInput(arguments[0], mask, hll); | ||
| return; | ||
| case TEXT: | ||
| case STRING: | ||
| case BLOB: | ||
| addBinaryInput(arguments[0], mask, hll); | ||
| return; | ||
| case BOOLEAN: | ||
| addBooleanInput(arguments[0], mask, hll); | ||
| return; | ||
| default: | ||
| throw new UnSupportedDataTypeException( | ||
| String.format( | ||
| "Unsupported data type in APPROX_COUNT_DISTINCT Aggregation: %s", seriesDataType)); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void addIntermediate(Column argument) { | ||
|
|
||
| for (int i = 0; i < argument.getPositionCount(); i++) { | ||
| if (!argument.isNull(i)) { | ||
| HyperLogLog current = new HyperLogLog(argument.getBinary(i).getValues()); | ||
| state.merge(current); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void evaluateIntermediate(ColumnBuilder columnBuilder) { | ||
| checkArgument( | ||
| columnBuilder instanceof BinaryColumnBuilder, | ||
| "intermediate input and output of APPROX_COUNT_DISTINCT should be BinaryColumn"); | ||
| columnBuilder.writeBinary(new Binary(state.getHyperLogLog().serialize())); | ||
| } | ||
|
|
||
| @Override | ||
| public void evaluateFinal(ColumnBuilder columnBuilder) { | ||
| columnBuilder.writeLong(state.getHyperLogLog().cardinality()); | ||
| } | ||
|
|
||
| @Override | ||
| public boolean hasFinalResult() { | ||
| return false; | ||
| } | ||
|
|
||
| @Override | ||
| public void addStatistics(Statistics[] statistics) { | ||
| throw new UnsupportedOperationException( | ||
| "ApproxCountDistinctAccumulator does not support statistics"); | ||
| } | ||
|
|
||
| @Override | ||
| public void reset() { | ||
| state.getHyperLogLog().reset(); | ||
| } | ||
|
|
||
| public void addBooleanInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getBoolean(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getBoolean(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void addIntInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getInt(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getInt(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void addLongInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getLong(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getLong(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void addFloatInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getFloat(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getFloat(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void addDoubleInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getDouble(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getDouble(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public void addBinaryInput(Column valueColumn, AggregationMask mask, HyperLogLog hll) { | ||
| int positionCount = mask.getPositionCount(); | ||
|
|
||
| if (mask.isSelectAll()) { | ||
| for (int i = 0; i < valueColumn.getPositionCount(); i++) { | ||
| if (!valueColumn.isNull(i)) { | ||
| hll.add(valueColumn.getBinary(i)); | ||
| } | ||
| } | ||
| } else { | ||
| int[] selectedPositions = mask.getSelectedPositions(); | ||
| int position; | ||
| for (int i = 0; i < positionCount; i++) { | ||
| position = selectedPositions[i]; | ||
| if (!valueColumn.isNull(position)) { | ||
| hll.add(valueColumn.getBinary(position)); | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public static HyperLogLog getOrCreateHyperLogLog( | ||
| HyperLogLogStateFactory.SingleHyperLogLogState state, double maxStandardError) { | ||
| HyperLogLog hll = state.getHyperLogLog(); | ||
| if (hll == null) { | ||
| hll = new HyperLogLog(maxStandardError); | ||
| state.setHyperLogLog(hll); | ||
| } | ||
| return hll; | ||
| } | ||
| } | ||
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Don't need to use
SingleHyperLogLogStateto wrap theHyperLogLog, you can refer toTableModeAccumulator