Skip to content
Merged
Show file tree
Hide file tree
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 Apr 1, 2025
03a9421
style: add approx_count_distinct BE
FearfulTomcat27 Apr 1, 2025
8d5406b
fix: Error message for specification approx_count_distinct
FearfulTomcat27 Apr 1, 2025
951cc42
feat: lazy load for hyperloglog
FearfulTomcat27 Apr 11, 2025
7fcc907
feat: Add support for the addIntermediate function to the Approx_coun…
FearfulTomcat27 Apr 14, 2025
eb080af
Add APPROX_COUNT_DISTINCT IT
FearfulTomcat27 Apr 17, 2025
7bb99ac
fix: Checksum Approx_count_distinct The second argument must be a Num…
FearfulTomcat27 Apr 24, 2025
4aca109
fix: Add HyperLogLogBigArray and delete unused class
FearfulTomcat27 Apr 24, 2025
4498846
fix: Modify the type validation of the second parameter for Approx_co…
FearfulTomcat27 Apr 24, 2025
91c0a9f
fix: Fix calculation errors in Approx_count_distinct when different d…
FearfulTomcat27 Apr 24, 2025
96b7eb2
fix: Add approx_count_distinct in GROUP BY situation for IT.
FearfulTomcat27 Apr 24, 2025
b1484bb
fix: Optimize HyperLogLog merging and state management in ApproxCount…
FearfulTomcat27 Apr 24, 2025
22de508
fix: Remove unused constructors and initial value handling in BinaryB…
FearfulTomcat27 Apr 24, 2025
2d9713e
fix: Refactor ApproxCountDistinctAccumulator and related classes for …
FearfulTomcat27 Apr 24, 2025
a57d289
fix: Improve variable initialization and documentation formatting in …
FearfulTomcat27 Apr 24, 2025
abcaab4
fix: Clean up code by removing unnecessary print statements and simpl…
FearfulTomcat27 Apr 24, 2025
79ebed7
fix: Fix the issue where HyperLogLogBigArray does not update sizeOfHy…
FearfulTomcat27 Apr 24, 2025
aff9346
fix: Update SQL queries in IoTDBTableAggregationIT to order results b…
FearfulTomcat27 Apr 24, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -4105,6 +4105,59 @@ public void modeTest() {
DATABASE_NAME);
}

@Test
public void approxCountDistinctTest() {
String[] expectedHeader = buildHeaders(17);
String[] retArray = new String[] {"10,2,2,4,16,2,2,5,5,5,5,2,24,32,5,10,1,"};
tableResultSetEqualTest(
"select approx_count_distinct(time), approx_count_distinct(province), approx_count_distinct(city), approx_count_distinct(region), approx_count_distinct(device_id), approx_count_distinct(color), approx_count_distinct(type), approx_count_distinct(s1), approx_count_distinct(s2), approx_count_distinct(s3), approx_count_distinct(s4), approx_count_distinct(s5), approx_count_distinct(s6), approx_count_distinct(s7), approx_count_distinct(s8), approx_count_distinct(s9), approx_count_distinct(s10) from table1",
expectedHeader,
retArray,
DATABASE_NAME);

tableResultSetEqualTest(
"select approx_count_distinct(time, 0.02), approx_count_distinct(province, 0.02), approx_count_distinct(city, 0.02), approx_count_distinct(region, 0.02), approx_count_distinct(device_id, 0.02), approx_count_distinct(color, 0.02), approx_count_distinct(type, 0.02), approx_count_distinct(s1, 0.02), approx_count_distinct(s2, 0.02), approx_count_distinct(s3, 0.02), approx_count_distinct(s4, 0.02), approx_count_distinct(s5, 0.02), approx_count_distinct(s6, 0.02), approx_count_distinct(s7, 0.02), approx_count_distinct(s8, 0.02), approx_count_distinct(s9, 0.02), approx_count_distinct(s10, 0.02) from table1",
expectedHeader,
retArray,
DATABASE_NAME);

retArray =
new String[] {
"2024-09-24T06:15:30.000Z,beijing,2,2,",
"2024-09-24T06:15:31.000Z,beijing,0,0,",
"2024-09-24T06:15:35.000Z,beijing,2,2,",
"2024-09-24T06:15:36.000Z,beijing,2,4,",
"2024-09-24T06:15:40.000Z,beijing,0,4,",
"2024-09-24T06:15:41.000Z,beijing,2,0,",
"2024-09-24T06:15:46.000Z,beijing,0,2,",
"2024-09-24T06:15:50.000Z,beijing,0,2,",
"2024-09-24T06:15:51.000Z,beijing,2,0,",
"2024-09-24T06:15:55.000Z,beijing,2,0,",
"2024-09-24T06:15:30.000Z,shanghai,2,2,",
"2024-09-24T06:15:31.000Z,shanghai,0,0,",
"2024-09-24T06:15:35.000Z,shanghai,2,2,",
"2024-09-24T06:15:36.000Z,shanghai,2,4,",
"2024-09-24T06:15:40.000Z,shanghai,0,4,",
"2024-09-24T06:15:41.000Z,shanghai,2,0,",
"2024-09-24T06:15:46.000Z,shanghai,0,2,",
"2024-09-24T06:15:50.000Z,shanghai,0,2,",
"2024-09-24T06:15:51.000Z,shanghai,2,0,",
"2024-09-24T06:15:55.000Z,shanghai,2,0,",
};

tableResultSetEqualTest(
"select time,province,approx_count_distinct(s6),approx_count_distinct(s7) from table1 group by 1,2 order by 2,1",
new String[] {"time", "province", "_col2", "_col3"},
retArray,
DATABASE_NAME);

tableResultSetEqualTest(
"select time,province,approx_count_distinct(s6,0.02),approx_count_distinct(s7,0.02) from table1 group by 1,2 order by 2,1",
new String[] {"time", "province", "_col2", "_col3"},
retArray,
DATABASE_NAME);
}

@Test
public void exceptionTest() {
tableAssertTestFail(
Expand Down Expand Up @@ -4135,6 +4188,22 @@ public void exceptionTest() {
"select last_by() from table1",
"701: Aggregate functions [last_by] should only have two or three arguments",
DATABASE_NAME);
tableAssertTestFail(
"select approx_count_distinct() from table1",
"701: Aggregate functions [approx_count_distinct] should only have two arguments",
DATABASE_NAME);
tableAssertTestFail(
"select approx_count_distinct(province, 0.3) from table1",
"750: Max Standard Error must be in [0.0040625, 0.26]: 0.3",
DATABASE_NAME);
tableAssertTestFail(
"select approx_count_distinct(province, 0.3) from table1",
"750: Max Standard Error must be in [0.0040625, 0.26]: 0.3",
DATABASE_NAME);
tableAssertTestFail(
"select approx_count_distinct(province, 'test') from table1",
"701: Second argument of Aggregate functions [approx_count_distinct] should be numberic type and do not use expression",
DATABASE_NAME);
}

// ==================================================================
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.iotdb.commons.udf.utils.UDFDataTypeTransformer;
import org.apache.iotdb.db.queryengine.execution.aggregation.VarianceAccumulator;
import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.GroupedAccumulator;
import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.GroupedApproxCountDistinctAccumulator;
import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.GroupedAvgAccumulator;
import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.GroupedCountAccumulator;
import org.apache.iotdb.db.queryengine.execution.operator.source.relational.aggregation.grouped.GroupedCountIfAccumulator;
Expand Down Expand Up @@ -240,6 +241,8 @@ private static GroupedAccumulator createBuiltinGroupedAccumulator(
case VAR_POP:
return new GroupedVarianceAccumulator(
inputDataTypes.get(0), VarianceAccumulator.VarianceType.VAR_POP);
case APPROX_COUNT_DISTINCT:
return new GroupedApproxCountDistinctAccumulator(inputDataTypes.get(0));
default:
throw new IllegalArgumentException("Invalid Aggregation function: " + aggregationType);
}
Expand Down Expand Up @@ -305,6 +308,8 @@ public static TableAccumulator createBuiltinAccumulator(
case VAR_POP:
return new TableVarianceAccumulator(
inputDataTypes.get(0), VarianceAccumulator.VarianceType.VAR_POP);
case APPROX_COUNT_DISTINCT:
return new ApproxCountDistinctAccumulator(inputDataTypes.get(0));
default:
throw new IllegalArgumentException("Invalid Aggregation function: " + aggregationType);
}
Expand Down
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 =
Copy link
Copy Markdown
Contributor

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 SingleHyperLogLogState to wrap the HyperLogLog, you can refer to TableModeAccumulator

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;
}
}
Loading
Loading