bitmap function for all native integers

This commit is contained in:
Amos Bird 2021-02-07 11:32:54 +08:00
parent 298a29ef52
commit 1b32292dd0
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
5 changed files with 154 additions and 28 deletions

View File

@ -16,6 +16,22 @@ namespace ErrorCodes
namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::UInt8) return new AggregateFunctionTemplate<UInt8, Data<UInt8>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::UInt16) return new AggregateFunctionTemplate<UInt16, Data<UInt16>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::UInt32) return new AggregateFunctionTemplate<UInt32, Data<UInt32>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::UInt64) return new AggregateFunctionTemplate<UInt64, Data<UInt64>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Int8) return new AggregateFunctionTemplate<Int8, Data<Int8>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Int16) return new AggregateFunctionTemplate<Int16, Data<Int16>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Int32) return new AggregateFunctionTemplate<Int32, Data<Int32>>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Int64) return new AggregateFunctionTemplate<Int64, Data<Int64>>(std::forward<TArgs>(args)...);
return nullptr;
}
template <template <typename> class Data>
AggregateFunctionPtr createAggregateFunctionBitmap(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
@ -28,7 +44,7 @@ namespace
+ " is illegal, because it cannot be used in Bitmap operations",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
AggregateFunctionPtr res(createWithUnsignedIntegerType<AggregateFunctionBitmap, Data>(*argument_types[0], argument_types[0]));
AggregateFunctionPtr res(createWithIntegerType<AggregateFunctionBitmap, Data>(*argument_types[0], argument_types[0]));
if (!res)
throw Exception(
@ -55,7 +71,7 @@ namespace
const DataTypeAggregateFunction & datatype_aggfunc = dynamic_cast<const DataTypeAggregateFunction &>(*argument_type_ptr);
AggregateFunctionPtr aggfunc = datatype_aggfunc.getFunction();
argument_type_ptr = aggfunc->getArgumentTypes()[0];
AggregateFunctionPtr res(createWithUnsignedIntegerType<AggregateFunctionTemplate, AggregateFunctionGroupBitmapData>(
AggregateFunctionPtr res(createWithIntegerType<AggregateFunctionTemplate, AggregateFunctionGroupBitmapData>(
*argument_type_ptr, argument_type_ptr));
if (!res)
throw Exception(

View File

@ -32,6 +32,7 @@ template <typename T, UInt8 small_set_size>
class RoaringBitmapWithSmallSet : private boost::noncopyable
{
private:
using UnsignedT = std::make_unsigned_t<T>;
SmallSet<T, small_set_size> small;
using ValueBuffer = std::vector<T>;
using RoaringBitmap = std::conditional_t<sizeof(T) >= 8, roaring::Roaring64Map, roaring::Roaring>;
@ -363,6 +364,7 @@ public:
/**
* Check whether the argument is the subset of this set.
* Empty set is a subset of any other set (consistent with hasAll).
* It's used in subset and currently only support comparing same type
*/
UInt8 rb_is_subset(const RoaringBitmapWithSmallSet & r1) const
{
@ -486,6 +488,7 @@ public:
/**
* Return new set with specified range (not include the range_end)
* It's used in subset and currently only support UInt32
*/
UInt64 rb_range(UInt64 range_start, UInt64 range_end, RoaringBitmapWithSmallSet & r1) const
{
@ -525,6 +528,7 @@ public:
/**
* Return new set of the smallest `limit` values in set which is no less than `range_start`.
* It's used in subset and currently only support UInt32
*/
UInt64 rb_limit(UInt64 range_start, UInt64 limit, RoaringBitmapWithSmallSet & r1) const
{
@ -578,10 +582,10 @@ public:
{
if (small.empty())
return 0;
auto min_val = std::numeric_limits<std::make_unsigned_t<T>>::max();
auto min_val = std::numeric_limits<UnsignedT>::max();
for (const auto & x : small)
{
auto val = x.getValue();
UnsignedT val = x.getValue();
if (val < min_val)
min_val = val;
}
@ -597,10 +601,10 @@ public:
{
if (small.empty())
return 0;
auto max_val = std::numeric_limits<std::make_unsigned_t<T>>::min();
UnsignedT max_val = 0;
for (const auto & x : small)
{
auto val = x.getValue();
UnsignedT val = x.getValue();
if (val > max_val)
max_val = val;
}
@ -611,7 +615,8 @@ public:
}
/**
* Replace value
* Replace value.
* It's used in transform and currently can only support UInt32
*/
void rb_replace(const UInt64 * from_vals, const UInt64 * to_vals, size_t num)
{

View File

@ -116,8 +116,35 @@ public:
DataTypes argument_types = {nested_type};
Array params_row;
AggregateFunctionProperties properties;
AggregateFunctionPtr bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionPtr bitmap_function;
WhichDataType which(nested_type);
if (which.isUInt8())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<UInt8>::name(), argument_types, params_row, properties);
else if (which.isUInt16())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<UInt16>::name(), argument_types, params_row, properties);
else if (which.isUInt32())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row, properties);
else if (which.isUInt64())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<UInt64>::name(), argument_types, params_row, properties);
else if (which.isInt8())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<Int8>::name(), argument_types, params_row, properties);
else if (which.isInt16())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<Int16>::name(), argument_types, params_row, properties);
else if (which.isInt32())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<Int32>::name(), argument_types, params_row, properties);
else if (which.isInt64())
bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<Int64>::name(), argument_types, params_row, properties);
else
throw Exception(
"Unexpected type " + array_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeAggregateFunction>(bitmap_function, argument_types, params_row);
}
@ -141,6 +168,14 @@ public:
return executeBitmapData<UInt32>(argument_types, arguments);
else if (which.isUInt64())
return executeBitmapData<UInt64>(argument_types, arguments);
else if (which.isInt8())
return executeBitmapData<Int8>(argument_types, arguments);
else if (which.isInt16())
return executeBitmapData<Int16>(argument_types, arguments);
else if (which.isInt32())
return executeBitmapData<Int32>(argument_types, arguments);
else if (which.isInt64())
return executeBitmapData<Int64>(argument_types, arguments);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -161,7 +196,7 @@ private:
Array params_row;
AggregateFunctionProperties properties;
AggregateFunctionPtr bitmap_function = AggregateFunctionFactory::instance().get(
AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row, properties);
AggregateFunctionGroupBitmapData<T>::name(), argument_types, params_row, properties);
auto col_to = ColumnAggregateFunction::create(bitmap_function);
col_to->reserve(offsets.size());
@ -197,7 +232,7 @@ public:
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()))
if (!(bitmap_type && bitmap_type->getFunctionName() =="groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -230,6 +265,14 @@ public:
executeIntType<UInt32>(arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt64())
executeIntType<UInt64>(arguments, input_rows_count, res_data, res_offsets);
else if (which.isInt8())
executeIntType<Int8>(arguments, input_rows_count, res_data, res_offsets);
else if (which.isInt16())
executeIntType<Int16>(arguments, input_rows_count, res_data, res_offsets);
else if (which.isInt32())
executeIntType<Int32>(arguments, input_rows_count, res_data, res_offsets);
else if (which.isInt64())
executeIntType<Int64>(arguments, input_rows_count, res_data, res_offsets);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -279,7 +322,7 @@ public:
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()))
if (!(bitmap_type && bitmap_type->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -312,6 +355,14 @@ public:
return executeIntType<UInt32>(arguments, input_rows_count);
else if (which.isUInt64())
return executeIntType<UInt64>(arguments, input_rows_count);
else if (which.isInt8())
return executeIntType<Int8>(arguments, input_rows_count);
else if (which.isInt16())
return executeIntType<Int16>(arguments, input_rows_count);
else if (which.isInt32())
return executeIntType<Int32>(arguments, input_rows_count);
else if (which.isInt64())
return executeIntType<Int64>(arguments, input_rows_count);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -384,7 +435,11 @@ struct BitmapSubsetInRangeImpl
public:
static constexpr auto name = "bitmapSubsetInRange";
template <typename T>
static void apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_0, UInt64 range_start, UInt64 range_end, AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
static void apply(
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0,
UInt64 range_start,
UInt64 range_end,
AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
bitmap_data_0.rbs.rb_range(range_start, range_end, bitmap_data_2.rbs);
}
@ -395,7 +450,11 @@ struct BitmapSubsetLimitImpl
public:
static constexpr auto name = "bitmapSubsetLimit";
template <typename T>
static void apply(const AggregateFunctionGroupBitmapData<T> & bitmap_data_0, UInt64 range_start, UInt64 range_end, AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
static void apply(
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0,
UInt64 range_start,
UInt64 range_end,
AggregateFunctionGroupBitmapData<T> & bitmap_data_2)
{
bitmap_data_0.rbs.rb_limit(range_start, range_end, bitmap_data_2.rbs);
}
@ -421,7 +480,7 @@ public:
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()))
if (!(bitmap_type && bitmap_type->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -456,6 +515,14 @@ public:
return executeIntType<UInt32>(arguments, input_rows_count);
else if (which.isUInt64())
return executeIntType<UInt64>(arguments, input_rows_count);
else if (which.isInt8())
return executeIntType<Int8>(arguments, input_rows_count);
else if (which.isInt16())
return executeIntType<Int16>(arguments, input_rows_count);
else if (which.isInt32())
return executeIntType<Int32>(arguments, input_rows_count);
else if (which.isInt64())
return executeIntType<Int64>(arguments, input_rows_count);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -579,7 +646,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * bitmap_type = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type && bitmap_type->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type && bitmap_type->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a bitmap but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -604,6 +671,14 @@ public:
executeIntType<UInt32>(arguments, input_rows_count, vec_to);
else if (which.isUInt64())
executeIntType<UInt64>(arguments, input_rows_count, vec_to);
else if (which.isInt8())
executeIntType<Int8>(arguments, input_rows_count, vec_to);
else if (which.isInt16())
executeIntType<Int16>(arguments, input_rows_count, vec_to);
else if (which.isInt32())
executeIntType<Int32>(arguments, input_rows_count, vec_to);
else if (which.isInt64())
executeIntType<Int64>(arguments, input_rows_count, vec_to);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -743,15 +818,15 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a 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);
WhichDataType which(arguments[1].get());
if (!(which.isUInt8() || which.isUInt16() || which.isUInt32() || which.isUInt64()))
if (!which.isNativeInt() && !which.isNativeUInt())
throw Exception(
"Second argument for function " + getName() + " must be one of [UInt8, UInt16, UInt32, UInt64] but it has type " + arguments[1]->getName() + ".",
"Second argument for function " + getName() + " must be an native integer type but it has type " + arguments[1]->getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<UInt8>>();
@ -775,6 +850,14 @@ public:
executeIntType<UInt32>(arguments, input_rows_count, vec_to);
else if (which.isUInt64())
executeIntType<UInt64>(arguments, input_rows_count, vec_to);
else if (which.isInt8())
executeIntType<Int8>(arguments, input_rows_count, vec_to);
else if (which.isInt16())
executeIntType<Int16>(arguments, input_rows_count, vec_to);
else if (which.isInt32())
executeIntType<Int32>(arguments, input_rows_count, vec_to);
else if (which.isInt64())
executeIntType<Int64>(arguments, input_rows_count, vec_to);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -839,15 +922,15 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a 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 auto * bitmap_type1 = typeid_cast<const DataTypeAggregateFunction *>(arguments[1].get());
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == "groupBitmap"))
throw Exception(
"Second argument for function " + getName() + " must be a 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())
@ -877,6 +960,14 @@ public:
executeIntType<UInt32>(arguments, input_rows_count, vec_to);
else if (which.isUInt64())
executeIntType<UInt64>(arguments, input_rows_count, vec_to);
else if (which.isInt8())
executeIntType<Int8>(arguments, input_rows_count, vec_to);
else if (which.isInt16())
executeIntType<Int16>(arguments, input_rows_count, vec_to);
else if (which.isInt32())
executeIntType<Int32>(arguments, input_rows_count, vec_to);
else if (which.isInt64())
executeIntType<Int64>(arguments, input_rows_count, vec_to);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -974,15 +1065,15 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const auto * bitmap_type0 = typeid_cast<const DataTypeAggregateFunction *>(arguments[0].get());
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type0 && bitmap_type0->getFunctionName() == "groupBitmap"))
throw Exception(
"First argument for function " + getName() + " must be a 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 auto * bitmap_type1 = typeid_cast<const DataTypeAggregateFunction *>(arguments[1].get());
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == AggregateFunctionGroupBitmapData<UInt32>::name()))
if (!(bitmap_type1 && bitmap_type1->getFunctionName() == "groupBitmap"))
throw Exception(
"Second argument for function " + getName() + " must be a 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())
@ -1009,6 +1100,14 @@ public:
return executeBitmapData<UInt32>(arguments, input_rows_count);
else if (which.isUInt64())
return executeBitmapData<UInt64>(arguments, input_rows_count);
else if (which.isUInt8())
return executeBitmapData<UInt8>(arguments, input_rows_count);
else if (which.isUInt16())
return executeBitmapData<UInt16>(arguments, input_rows_count);
else if (which.isUInt32())
return executeBitmapData<UInt32>(arguments, input_rows_count);
else if (which.isUInt64())
return executeBitmapData<UInt64>(arguments, input_rows_count);
else
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -0,0 +1 @@
251 65531 4294967291 18446744073709551611 255 65535 4294967295 18446744073709551615

View File

@ -0,0 +1,5 @@
drop table if exists t;
create table t(i8 Int8, i16 Int16, i32 Int32, i64 Int64) engine Memory;
insert into t values (-1, -1, -1, -1), (-2, -2, -2, -2), (-3, -3, -3, -3), (-4, -4, -4, -4), (-5, -5, -5, -5);
select * apply bitmapMin, * apply bitmapMax from (select * apply groupBitmapState from t);
drop table t;