Merge pull request #7598 from infinivision/bitmap_replace

added bitmapTransform
This commit is contained in:
alexey-milovidov 2019-11-05 02:23:50 +03:00 committed by GitHub
commit 6a871f579f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 378 additions and 156 deletions

View File

@ -581,6 +581,23 @@ public:
return max_val;
}
/**
* Replace value
*/
void rb_replace(const UInt32 * from_vals, const UInt32 * to_vals, size_t num)
{
if (isSmall())
toLarge();
for (size_t i = 0; i < num; ++i)
{
if (from_vals[i] == to_vals[i])
continue;
bool changed = roaring_bitmap_remove_checked(rb, from_vals[i]);
if (changed)
roaring_bitmap_add(rb, to_vals[i]);
}
}
private:
/// To read and write the DB Buffer directly, migrate code from CRoaring
void db_roaring_bitmap_add_many(DB::ReadBuffer & dbBuf, roaring_bitmap_t * r, size_t n_args)

View File

@ -14,6 +14,7 @@ void registerFunctionsBitmap(FunctionFactory & factory)
factory.registerFunction<FunctionBitmapToArray>();
factory.registerFunction<FunctionBitmapSubsetInRange>();
factory.registerFunction<FunctionBitmapSubsetLimit>();
factory.registerFunction<FunctionBitmapTransform>();
factory.registerFunction<FunctionBitmapSelfCardinality>();
factory.registerFunction<FunctionBitmapMin>();

View File

@ -33,11 +33,20 @@ namespace ErrorCodes
* Convert bitmap to integer array:
* bitmapToArray: bitmap -> integer[]
*
* Retrun the smallest value in the set:
* bitmapMin: bitmap -> integer
*
* Retrun the greatest value in the set:
* bitmapMax: bitmap -> integer
*
* Return subset in specified range (not include the range_end):
* bitmapSubsetInRange: bitmap,integer,integer -> bitmap
*
* Return subset of the smallest `limit` values in set which is no smaller than `range_start`.
* bitmapSubsetInRange: bitmap,integer,integer -> bitmap
* bitmapSubsetLimit: bitmap,integer,integer -> bitmap
*
* Transform an array of values in a bitmap to another array of values, the result is a new bitmap.
* bitmapTransform: bitmap,integer[],integer[] -> bitmap
*
* Two bitmap and calculation:
* bitmapAnd: bitmap,bitmap -> bitmap
@ -54,12 +63,6 @@ namespace ErrorCodes
* Retrun bitmap cardinality:
* bitmapCardinality: bitmap -> integer
*
* Retrun the smallest value in the set:
* bitmapMin: bitmap -> integer
*
* Retrun the greatest value in the set:
* bitmapMax: bitmap -> integer
*
* Two bitmap and calculation, return cardinality:
* bitmapAndCardinality: bitmap,bitmap -> integer
*
@ -192,7 +195,7 @@ public:
const DataTypeAggregateFunction * bitmap_type = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type && bitmap_type->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypePtr data_type = bitmap_type->getArgumentsDataTypes()[0];
@ -246,9 +249,9 @@ private:
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T> & bd1
const AggregateFunctionGroupBitmapData<T> & bitmap_data_1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(column->getData()[i]);
UInt64 count = bd1.rbs.rb_to_array(res_data);
UInt64 count = bitmap_data_1.rbs.rb_to_array(res_data);
res_offset += count;
res_offsets.emplace_back(res_offset);
}
@ -274,7 +277,7 @@ public:
const DataTypeAggregateFunction * bitmap_type = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type && bitmap_type->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto arg_type1 = typeid_cast<const DataTypeNumber<UInt32> *>(arguments[1].get());
@ -322,7 +325,7 @@ private:
{
const IColumn * columns[3];
bool is_column_const[3];
const ColumnAggregateFunction * colAggFunc;
const ColumnAggregateFunction * col_agg_func;
const PaddedPODArray<AggregateDataPtr> * container0;
const PaddedPODArray<UInt32> * container1, * container2;
@ -332,14 +335,11 @@ private:
is_column_const[i] = isColumnConst(*columns[i]);
}
if (is_column_const[0])
{
colAggFunc = typeid_cast<const ColumnAggregateFunction*>(typeid_cast<const ColumnConst*>(columns[0])->getDataColumnPtr().get());
}
col_agg_func = typeid_cast<const ColumnAggregateFunction*>(typeid_cast<const ColumnConst*>(columns[0])->getDataColumnPtr().get());
else
{
colAggFunc = typeid_cast<const ColumnAggregateFunction*>(columns[0]);
}
container0 = &colAggFunc->getData();
col_agg_func = typeid_cast<const ColumnAggregateFunction*>(columns[0]);
container0 = &col_agg_func->getData();
if (is_column_const[1])
container1 = &typeid_cast<const ColumnUInt32*>(typeid_cast<const ColumnConst*>(columns[1])->getDataColumnPtr().get())->getData();
else
@ -349,21 +349,21 @@ private:
else
container2 = &typeid_cast<const ColumnUInt32*>(columns[2])->getData();
auto col_to = ColumnAggregateFunction::create(colAggFunc->getAggregateFunction());
auto col_to = ColumnAggregateFunction::create(col_agg_func->getAggregateFunction());
col_to->reserve(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateDataPtr dataPtr0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
const AggregateFunctionGroupBitmapData<T>& bd0
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
const AggregateDataPtr data_ptr_0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(data_ptr_0);
const UInt32 range_start = is_column_const[1] ? (*container1)[0] : (*container1)[i];
const UInt32 range_end = is_column_const[2] ? (*container2)[0] : (*container2)[i];
col_to->insertDefault();
AggregateFunctionGroupBitmapData<T> & bd2
AggregateFunctionGroupBitmapData<T> & bitmap_data_2
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
Impl::apply(bd0, range_start, range_end, bd2);
Impl::apply(bitmap_data_0, range_start, range_end, bitmap_data_2);
}
block.getByPosition(result).column = std::move(col_to);
}
@ -374,9 +374,9 @@ struct BitmapSubsetInRangeImpl
public:
static constexpr auto name = "bitmapSubsetInRange";
template <typename T>
static void apply(const AggregateFunctionGroupBitmapData<T> & bd0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
bd0.rbs.rb_range(range_start, range_end, bd2.rbs);
bitmap_data_0.rbs.rb_range(range_start, range_end, bitmap_data_2.rbs);
}
};
@ -385,15 +385,161 @@ struct BitmapSubsetLimitImpl
public:
static constexpr auto name = "bitmapSubsetLimit";
template <typename T>
static void apply(const AggregateFunctionGroupBitmapData<T> & bd0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
bd0.rbs.rb_limit(range_start, range_end, bd2.rbs);
bitmap_data_0.rbs.rb_limit(range_start, range_end, bitmap_data_2.rbs);
}
};
using FunctionBitmapSubsetInRange = FunctionBitmapSubset<BitmapSubsetInRangeImpl>;
using FunctionBitmapSubsetLimit = FunctionBitmapSubset<BitmapSubsetLimitImpl>;
class FunctionBitmapTransform : public IFunction
{
public:
static constexpr auto name = "bitmapTransform";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitmapTransform>(); }
String getName() const override { return name; }
bool isVariadic() const override { return false; }
size_t getNumberOfArguments() const override { return 3; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeAggregateFunction * bitmap_type = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type && bitmap_type->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (size_t i = 0; i < 2; ++i)
{
auto array_type = typeid_cast<const DataTypeArray *>(arguments[i + 1].get());
String msg(i == 0 ? "Second" : "Third");
msg += " argument for function " + getName() + " must be an UInt32 array but it has type " + arguments[i + 1]->getName() + ".";
if (!array_type)
throw Exception(msg, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto nested_type = array_type->getNestedType();
WhichDataType which(nested_type);
if (!which.isUInt32())
throw Exception(msg, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return arguments[0];
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8())
executeIntType<UInt8>(block, arguments, result, input_rows_count);
else if (which.isUInt16())
executeIntType<UInt16>(block, arguments, result, input_rows_count);
else if (which.isUInt32())
executeIntType<UInt32>(block, arguments, result, input_rows_count);
else if (which.isUInt64())
executeIntType<UInt64>(block, arguments, result, input_rows_count);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
using ToType = UInt64;
template <typename T>
void executeIntType(
Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
{
const IColumn * columns[3];
bool is_column_const[3];
const ColumnAggregateFunction * col_agg_func;
const PaddedPODArray<AggregateDataPtr> * container0;
const ColumnArray * array;
for (size_t i = 0; i < 3; ++i)
{
columns[i] = block.getByPosition(arguments[i]).column.get();
is_column_const[i] = isColumnConst(*columns[i]);
}
if (is_column_const[0])
{
col_agg_func = typeid_cast<const ColumnAggregateFunction*>(typeid_cast<const ColumnConst*>(columns[0])->getDataColumnPtr().get());
}
else
{
col_agg_func = typeid_cast<const ColumnAggregateFunction*>(columns[0]);
}
container0 = &col_agg_func->getData();
if (is_column_const[1])
array = typeid_cast<const ColumnArray*>(typeid_cast<const ColumnConst*>(columns[1])->getDataColumnPtr().get());
else
{
array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[1]).column.get());
}
const ColumnArray::Offsets & from_offsets = array->getOffsets();
const ColumnVector<UInt32>::Container & from_container = typeid_cast<const ColumnVector<UInt32> *>(&array->getData())->getData();
if (is_column_const[2])
array = typeid_cast<const ColumnArray*>(typeid_cast<const ColumnConst*>(columns[2])->getDataColumnPtr().get());
else
array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[2]).column.get());
const ColumnArray::Offsets & to_offsets = array->getOffsets();
const ColumnVector<UInt32>::Container & to_container = typeid_cast<const ColumnVector<UInt32> *>(&array->getData())->getData();
auto col_to = ColumnAggregateFunction::create(col_agg_func->getAggregateFunction());
col_to->reserve(input_rows_count);
size_t from_start;
size_t from_end;
size_t to_start;
size_t to_end;
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateDataPtr data_ptr_0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(data_ptr_0);
if (is_column_const[1])
{
from_start = 0;
from_end = from_container.size();
}
else
{
from_start = i == 0 ? 0 : from_offsets[i - 1];
from_end = from_offsets[i];
}
if (is_column_const[2])
{
to_start = 0;
to_end = to_container.size();
}
else
{
to_start = i == 0 ? 0 : to_offsets[i - 1];
to_end = to_offsets[i];
}
if (from_end - from_start != to_end - to_start)
throw Exception("From array size and to array size mismatch", ErrorCodes::LOGICAL_ERROR);
col_to->insertDefault();
AggregateFunctionGroupBitmapData<T> & bitmap_data_2
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
bitmap_data_2.rbs.merge(bitmap_data_0.rbs);
bitmap_data_2.rbs.rb_replace(&from_container[from_start], &to_container[to_start], from_end - from_start);
}
block.getByPosition(result).column = std::move(col_to);
}
};
template <typename Impl>
class FunctionBitmapSelfCardinalityImpl : public IFunction
{
@ -413,7 +559,7 @@ public:
auto bitmap_type = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type && bitmap_type->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<ToType>>();
}
@ -454,9 +600,9 @@ private:
= typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T> & bd
const AggregateFunctionGroupBitmapData<T> & bitmap_data
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(column->getData()[i]);
vec_to[i] = Impl::apply(bd);
vec_to[i] = Impl::apply(bitmap_data);
}
}
};
@ -466,9 +612,9 @@ struct BitmapCardinalityImpl
public:
static constexpr auto name = "bitmapCardinality";
template <typename T>
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data)
{
return bd.rbs.size();
return bitmap_data.rbs.size();
}
};
@ -477,9 +623,9 @@ struct BitmapMinImpl
public:
static constexpr auto name = "bitmapMin";
template <typename T>
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data)
{
return bd.rbs.rb_min();
return bitmap_data.rbs.rb_min();
}
};
@ -488,9 +634,9 @@ struct BitmapMaxImpl
public:
static constexpr auto name = "bitmapMax";
template <typename T>
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data)
{
return bd.rbs.rb_max();
return bitmap_data.rbs.rb_max();
}
};
@ -498,10 +644,10 @@ template <typename T>
struct BitmapAndCardinalityImpl
{
using ReturnType = UInt64;
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
// roaring_bitmap_and_cardinality( rb1, rb2 );
return bd1.rbs.rb_and_cardinality(bd2.rbs);
return bitmap_data_1.rbs.rb_and_cardinality(bitmap_data_2.rbs);
}
};
@ -510,10 +656,10 @@ template <typename T>
struct BitmapOrCardinalityImpl
{
using ReturnType = UInt64;
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
// return roaring_bitmap_or_cardinality( rb1, rb2 );
return bd1.rbs.rb_or_cardinality(bd2.rbs);
return bitmap_data_1.rbs.rb_or_cardinality(bitmap_data_2.rbs);
}
};
@ -521,10 +667,10 @@ template <typename T>
struct BitmapXorCardinalityImpl
{
using ReturnType = UInt64;
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
// return roaring_bitmap_xor_cardinality( rb1, rb2 );
return bd1.rbs.rb_xor_cardinality(bd2.rbs);
return bitmap_data_1.rbs.rb_xor_cardinality(bitmap_data_2.rbs);
}
};
@ -532,10 +678,10 @@ template <typename T>
struct BitmapAndnotCardinalityImpl
{
using ReturnType = UInt64;
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt64 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
// roaring_bitmap_andnot_cardinality( rb1, rb2 );
return bd1.rbs.rb_andnot_cardinality(bd2.rbs);
return bitmap_data_1.rbs.rb_andnot_cardinality(bitmap_data_2.rbs);
}
};
@ -543,9 +689,9 @@ template <typename T>
struct BitmapHasAllImpl
{
using ReturnType = UInt8;
static UInt8 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt8 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
return bd1.rbs.rb_is_subset(bd2.rbs);
return bitmap_data_1.rbs.rb_is_subset(bitmap_data_2.rbs);
}
};
@ -553,9 +699,9 @@ template <typename T>
struct BitmapHasAnyImpl
{
using ReturnType = UInt8;
static UInt8 apply(const AggregateFunctionGroupBitmapData<T> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
static UInt8 apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
return bd1.rbs.rb_intersect(bd2.rbs);
return bitmap_data_1.rbs.rb_intersect(bitmap_data_2.rbs);
}
};
@ -577,7 +723,7 @@ public:
auto bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto arg_type1 = typeid_cast<const DataTypeNumber<UInt32> *>(arguments[1].get());
if (!(arg_type1))
@ -639,11 +785,11 @@ private:
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateDataPtr dataPtr0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
const AggregateDataPtr data_ptr_0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
const UInt32 data1 = is_column_const[1] ? (*container1)[0] : (*container1)[i];
const AggregateFunctionGroupBitmapData<T>& bd0
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
vec_to[i] = bd0.rbs.rb_contains(data1);
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(data_ptr_0);
vec_to[i] = bitmap_data_0.rbs.rb_contains(data1);
}
}
};
@ -667,13 +813,13 @@ public:
auto bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto bitmap_type1 = typeid_cast<const DataTypeAggregateFunction *>(arguments[1].get());
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"Second argument for function " + getName() + " must be an bitmap but it has type " + arguments[1]->getName() + ".",
"Second argument for function " + getName() + " must be a bitmap but it has type " + arguments[1]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (bitmap_type0->getArgumentsDataTypes()[0]->getTypeId() != bitmap_type1->getArgumentsDataTypes()[0]->getTypeId())
@ -736,13 +882,13 @@ private:
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateDataPtr dataPtr0 = is_column_const[0] ? container0[0] : container0[i];
const AggregateDataPtr dataPtr1 = is_column_const[1] ? container1[0] : container1[i];
const AggregateFunctionGroupBitmapData<T> & bd1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
const AggregateFunctionGroupBitmapData<T> & bd2
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr1);
vec_to[i] = Impl<T>::apply(bd1, bd2);
const AggregateDataPtr data_ptr_0 = is_column_const[0] ? container0[0] : container0[i];
const AggregateDataPtr data_ptr_1 = is_column_const[1] ? container1[0] : container1[i];
const AggregateFunctionGroupBitmapData<T> & bitmap_data_1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(data_ptr_0);
const AggregateFunctionGroupBitmapData<T> & bitmap_data_2
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(data_ptr_1);
vec_to[i] = Impl<T>::apply(bitmap_data_1, bitmap_data_2);
}
}
};
@ -750,36 +896,36 @@ private:
template <typename T>
struct BitmapAndImpl
{
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
toBd.rbs.rb_and(bd2.rbs);
bitmap_data_1.rbs.rb_and(bitmap_data_2.rbs);
}
};
template <typename T>
struct BitmapOrImpl
{
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
toBd.rbs.rb_or(bd2.rbs);
bitmap_data_1.rbs.rb_or(bitmap_data_2.rbs);
}
};
template <typename T>
struct BitmapXorImpl
{
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
toBd.rbs.rb_xor(bd2.rbs);
bitmap_data_1.rbs.rb_xor(bitmap_data_2.rbs);
}
};
template <typename T>
struct BitmapAndnotImpl
{
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
static void apply(AggregateFunctionGroupBitmapData<T> & bitmap_data_1, const AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
toBd.rbs.rb_andnot(bd2.rbs);
bitmap_data_1.rbs.rb_andnot(bitmap_data_2.rbs);
}
};
@ -802,13 +948,13 @@ public:
auto bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"First argument for function " + getName() + " must be an bitmap but it has type " + arguments[0]->getName() + ".",
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto bitmap_type1 = typeid_cast<const DataTypeAggregateFunction *>(arguments[1].get());
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
throw Exception(
"Second argument for function " + getName() + " must be an bitmap but it has type " + arguments[1]->getName() + ".",
"Second argument for function " + getName() + " must be a bitmap but it has type " + arguments[1]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (bitmap_type0->getArgumentsDataTypes()[0]->getTypeId() != bitmap_type1->getArgumentsDataTypes()[0]->getTypeId())
@ -860,10 +1006,10 @@ private:
for (size_t i = 0; i < input_rows_count; ++i)
{
col_to->insertFrom(columns[0]->getData()[i]);
AggregateFunctionGroupBitmapData<T> & toBd = *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
const AggregateFunctionGroupBitmapData<T> & bd2
AggregateFunctionGroupBitmapData<T> & bitmap_data_1 = *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
const AggregateFunctionGroupBitmapData<T> & bitmap_data_2
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(columns[1]->getData()[i]);
Impl<T>::apply(toBd, bd2);
Impl<T>::apply(bitmap_data_1, bitmap_data_2);
}
block.getByPosition(result).column = std::move(col_to);
}

View File

@ -34,6 +34,10 @@
[6,8,10]
10
[1,3,5,6,8,10,11,13,14,15]
tag1 [0,1,2,3,4,5,6,7,8,9] [] [] [0,1,2,3,4,5,6,7,8,9]
tag2 [0,1,2,3,4,5,6,7,8,9] [0] [2] [1,2,3,4,5,6,7,8,9]
tag3 [0,1,2,3,4,5,6,7,8,9] [0,7] [3,101] [1,2,3,4,5,6,8,9,101]
tag4 [0,1,2,3,4,5,6,7,8,9] [5,999,2] [2,888,20] [0,1,3,4,6,7,8,9,20]
0
0
0

View File

@ -128,10 +128,37 @@ SELECT arraySort(bitmapToArray(groupBitmapAndState(z))) FROM bitmap_column_expr_
SELECT groupBitmapXor(z) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%');
SELECT arraySort(bitmapToArray(groupBitmapXorState(z))) FROM bitmap_column_expr_test2 WHERE like(tag_id, 'tag%');
DROP TABLE IF EXISTS bitmap_column_expr_test3;
CREATE TABLE bitmap_column_expr_test3
(
tag_id String,
z AggregateFunction(groupBitmap, UInt32),
replace Nested (
from UInt32,
to UInt32
)
)
ENGINE = MergeTree
ORDER BY tag_id;
DROP TABLE IF EXISTS numbers10;
CREATE VIEW numbers10 AS SELECT number FROM system.numbers LIMIT 10;
INSERT INTO bitmap_column_expr_test3(tag_id, z, replace.from, replace.to) SELECT 'tag1', groupBitmapState(toUInt32(number)), cast([] as Array(UInt32)), cast([] as Array(UInt32)) FROM numbers10;
INSERT INTO bitmap_column_expr_test3(tag_id, z, replace.from, replace.to) SELECT 'tag2', groupBitmapState(toUInt32(number)), cast([0] as Array(UInt32)), cast([2] as Array(UInt32)) FROM numbers10;
INSERT INTO bitmap_column_expr_test3(tag_id, z, replace.from, replace.to) SELECT 'tag3', groupBitmapState(toUInt32(number)), cast([0,7] as Array(UInt32)), cast([3,101] as Array(UInt32)) FROM numbers10;
INSERT INTO bitmap_column_expr_test3(tag_id, z, replace.from, replace.to) SELECT 'tag4', groupBitmapState(toUInt32(number)), cast([5,999,2] as Array(UInt32)), cast([2,888,20] as Array(UInt32)) FROM numbers10;
SELECT tag_id, bitmapToArray(z), replace.from, replace.to, bitmapToArray(bitmapTransform(z, replace.from, replace.to)) FROM bitmap_column_expr_test3 ORDER BY tag_id;
DROP TABLE IF EXISTS bitmap_test;
DROP TABLE IF EXISTS bitmap_state_test;
DROP TABLE IF EXISTS bitmap_column_expr_test;
DROP TABLE IF EXISTS bitmap_column_expr_test2;
DROP TABLE IF EXISTS numbers10;
DROP TABLE IF EXISTS bitmap_column_expr_test3;
-- bitmapHasAny:
---- Empty

View File

@ -46,7 +46,7 @@ bitmapToArray(bitmap)
**Example**
``` sql
```sql
SELECT bitmapToArray(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
@ -98,7 +98,7 @@ bitmapSubsetLimit(bitmap, range_start, limit)
**Example**
``` sql
```sql
SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res
```
@ -130,7 +130,7 @@ Type: `UInt8`.
**Example**
``` sql
```sql
SELECT bitmapContains(bitmapBuild([1,5,7,9]), toUInt32(9)) AS res
```
```text
@ -196,6 +196,108 @@ SELECT bitmapHasAll(bitmapBuild([1,2,3]),bitmapBuild([3,4,5])) AS res
```
## bitmapCardinality
Retrun bitmap cardinality of type UInt64.
```sql
bitmapCardinality(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
```sql
SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```text
┌─res─┐
│ 5 │
└─────┘
```
## bitmapMin
Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is empty.
```
bitmapMin(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
```sql
SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```
┌─res─┐
│ 1 │
└─────┘
```
## bitmapMax
Retrun the greatest value of type UInt64 in the set, 0 if the set is empty.
```
bitmapMax(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
```sql
SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```
┌─res─┐
│ 5 │
└─────┘
```
## bitmapTransform
Transform an array of values in a bitmap to another array of values, the result is a new bitmap.
```
bitmapTransform(bitmap, from_array, to_array)
```
**Parameters**
- `bitmap` bitmap object.
- `from_array` UInt32 array. For idx in range [0, from_array.size()), if bitmap contains from_array[idx], then replace it with to_array[idx]. Note that the result depends on array ordering if there are common elements between from_array and to_array.
- `to_array` UInt32 array, its size shall be the same to from_array.
**Example**
```sql
SELECT bitmapToArray(bitmapTransform(bitmapBuild([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), cast([5,999,2] as Array(UInt32)), cast([2,888,20] as Array(UInt32)))) AS res
```
```
┌─res───────────────────┐
│ [1,3,4,6,7,8,9,10,20] │
└───────────────────────┘
```
## bitmapAnd
Two bitmap and calculation, the result is a new bitmap.
@ -210,7 +312,7 @@ bitmapAnd(bitmap,bitmap)
**Example**
``` sql
```sql
SELECT bitmapToArray(bitmapAnd(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS res
```
@ -293,81 +395,6 @@ SELECT bitmapToArray(bitmapAndnot(bitmapBuild([1,2,3]),bitmapBuild([3,4,5]))) AS
└───────┘
```
## bitmapCardinality
Retrun bitmap cardinality of type UInt64.
```sql
bitmapCardinality(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
```sql
SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```text
┌─res─┐
│ 5 │
└─────┘
```
## bitmapMin
Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is empty.
```
bitmapMin(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
``` sql
SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```
┌─res─┐
│ 1 │
└─────┘
```
## bitmapMax
Retrun the greatest value of type UInt64 in the set, 0 if the set is empty.
```
bitmapMax(bitmap)
```
**Parameters**
- `bitmap` bitmap object.
**Example**
``` sql
SELECT bitmapMax(bitmapBuild([1, 2, 3, 4, 5])) AS res
```
```
┌─res─┐
│ 5 │
└─────┘
```
## bitmapAndCardinality
Two bitmap and calculation, return cardinality of type UInt64.