Added bitmap function feature, fixed test errors and code styles

This commit is contained in:
Andy Yang 2019-02-09 14:33:09 +08:00
parent 7e7a864fd4
commit 1a28ba01c1
3 changed files with 373 additions and 234 deletions

View File

@ -17,15 +17,30 @@ enum {
*
*/
#if defined(__clang__)
#pragma clang diagnostic ignored "-Wold-style-cast"
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
#pragma clang diagnostic ignored "-Wold-style-cast"
#pragma clang diagnostic ignored "-Wcast-align"
#pragma clang diagnostic ignored "-Wcast-qual"
#pragma clang diagnostic ignored "-Wundef"
#endif
#ifndef INCLUDE_PORTABILITY_H_
#define INCLUDE_PORTABILITY_H_
#ifdef __cplusplus
extern "C" {
#endif
#ifndef _GNU_SOURCE
#define _GNU_SOURCE
#endif
#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS 1
#endif
//#ifndef __STDC_FORMAT_MACROS
//#define __STDC_FORMAT_MACROS 1
//#endif
#if !(defined(_POSIX_C_SOURCE)) || (_POSIX_C_SOURCE < 200809L)
#define _POSIX_C_SOURCE 200809L
@ -250,6 +265,10 @@ static inline int hamming(uint64_t x) {
#define UINT32_C(c) (c##UL)
#endif
#ifdef __cplusplus
}
#endif
#endif /* INCLUDE_PORTABILITY_H_ */
/* end file /opt/bitmap/CRoaring-0.2.57/include/roaring/portability.h */
/* begin file /opt/bitmap/CRoaring-0.2.57/include/roaring/containers/perfparameters.h */
@ -1534,6 +1553,7 @@ inline bool array_container_contains(const array_container_t *arr,
}
//* Check whether a range of values from range_start (included) to range_end (excluded) is present. */
static inline bool array_container_contains_range(const array_container_t *arr,
uint32_t range_start, uint32_t range_end) {
@ -7163,4 +7183,5 @@ uint32_t roaring_read_uint32_iterator(roaring_uint32_iterator_t *it, uint32_t* b
#endif
#endif
/* end file /opt/bitmap/CRoaring-0.2.57/include/roaring/roaring.h */

View File

@ -1,15 +1,14 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <boost/noncopyable.hpp>
#include <Common/HashTable/SmallTable.h>
#include <roaring.hh>
#include <roaring.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <boost/noncopyable.hpp>
#include <roaring.hh>
#include <Common/HashTable/SmallTable.h>
namespace DB
{
/**
* For a small number of values - an array of fixed size "on the stack".
* For large, roaring_bitmap_t is allocated.
@ -43,7 +42,8 @@ public:
roaring_bitmap_free(rb);
}
void add(T value) {
void add(T value)
{
if (isSmall())
{
if (small.find(value) == small.end())
@ -61,10 +61,7 @@ public:
roaring_bitmap_add(rb, value);
}
UInt64 size() const
{
return isSmall() ? small.size() : roaring_bitmap_get_cardinality( rb );
}
UInt64 size() const { return isSmall() ? small.size() : roaring_bitmap_get_cardinality(rb); }
void merge(const RoaringBitmapWithSmallSet & r1)
{
@ -73,7 +70,7 @@ public:
if (isSmall())
toLarge();
roaring_bitmap_or_inplace( rb, r1.rb );
roaring_bitmap_or_inplace(rb, r1.rb);
}
else
{
@ -91,8 +88,8 @@ public:
{
toLarge();
UInt32 cardinality;
readBinary( cardinality, in );
db_roaring_bitmap_add_many( in, rb, cardinality );
readBinary(cardinality, in);
db_roaring_bitmap_add_many(in, rb, cardinality);
}
else
small.read(in);
@ -105,22 +102,17 @@ public:
if (isLarge())
{
UInt32 cardinality = roaring_bitmap_get_cardinality(rb);
writePODBinary( cardinality, out );
db_ra_to_uint32_array( out, &rb->high_low_container );
} else
writePODBinary(cardinality, out);
db_ra_to_uint32_array(out, &rb->high_low_container);
}
else
small.write(out);
}
roaring_bitmap_t * getRb() const
{
return rb;
}
roaring_bitmap_t * getRb() const { return rb; }
Small& getSmall() const
{
return small;
}
Small & getSmall() const { return small; }
/**
* Get a new roaring_bitmap_t from elements of small
@ -139,11 +131,12 @@ public:
void rb_and(const RoaringBitmapWithSmallSet & r1)
{
ValueBuffer buffer;
if(isSmall() && r1.isSmall())
if (isSmall() && r1.isSmall())
{
// intersect
for (const auto & value : this->small)
if (r1.small.find(value) != r1.small.end()) buffer.push_back(value);
if (r1.small.find(value) != r1.small.end())
buffer.push_back(value);
// Clear out the original values
this->small.clear();
@ -152,9 +145,12 @@ public:
this->small.insert(value);
buffer.clear();
} else if(isSmall() && r1.isLarge()){
}
else if (isSmall() && r1.isLarge())
{
for (const auto & value : this->small)
if( roaring_bitmap_contains( r1.rb, value ) ) buffer.push_back( value );
if (roaring_bitmap_contains(r1.rb, value))
buffer.push_back(value);
// Clear out the original values
this->small.clear();
@ -163,30 +159,32 @@ public:
this->small.insert(value);
buffer.clear();
} else {
}
else
{
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
roaring_bitmap_and_inplace( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
roaring_bitmap_and_inplace(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
}
}
/**
* Computes the union between two bitmaps.
*/
void rb_or(const RoaringBitmapWithSmallSet & r1)
{
this->merge( r1 );
}
void rb_or(const RoaringBitmapWithSmallSet & r1) { this->merge(r1); }
/**
* Computes the symmetric difference (xor) between two bitmaps.
*/
void rb_xor(const RoaringBitmapWithSmallSet & r1)
{
if( this->isSmall() ) toLarge();
if (this->isSmall())
toLarge();
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
roaring_bitmap_xor_inplace( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
roaring_bitmap_xor_inplace(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
}
/**
@ -195,11 +193,12 @@ public:
void rb_andnot(const RoaringBitmapWithSmallSet & r1)
{
ValueBuffer buffer;
if(isSmall() && r1.isSmall())
if (isSmall() && r1.isSmall())
{
// subtract
for (const auto & value : this->small)
if (r1.small.find(value) == r1.small.end()) buffer.push_back(value);
if (r1.small.find(value) == r1.small.end())
buffer.push_back(value);
// Clear out the original values
this->small.clear();
@ -208,9 +207,12 @@ public:
this->small.insert(value);
buffer.clear();
} else if(isSmall() && r1.isLarge()){
}
else if (isSmall() && r1.isLarge())
{
for (const auto & value : this->small)
if( !roaring_bitmap_contains( r1.rb, value ) ) buffer.push_back( value );
if (!roaring_bitmap_contains(r1.rb, value))
buffer.push_back(value);
// Clear out the original values
this->small.clear();
@ -219,10 +221,13 @@ public:
this->small.insert(value);
buffer.clear();
} else {
}
else
{
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
roaring_bitmap_andnot_inplace( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
roaring_bitmap_andnot_inplace(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
}
}
@ -232,16 +237,24 @@ public:
UInt64 rb_and_cardinality(const RoaringBitmapWithSmallSet & r1) const
{
UInt64 retSize = 0;
if( isSmall() && r1.isSmall() ){
if (isSmall() && r1.isSmall())
{
for (const auto & value : this->small)
if (r1.small.find( value ) != r1.small.end()) retSize++;
} else if( isSmall() && r1.isLarge() ){
if (r1.small.find(value) != r1.small.end())
retSize++;
}
else if (isSmall() && r1.isLarge())
{
for (const auto & value : this->small)
if( roaring_bitmap_contains( r1.rb, value ) ) retSize++;
} else {
if (roaring_bitmap_contains(r1.rb, value))
retSize++;
}
else
{
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
retSize = roaring_bitmap_and_cardinality( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
retSize = roaring_bitmap_and_cardinality(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
}
return retSize;
}
@ -253,7 +266,7 @@ public:
{
UInt64 c1 = this->size();
UInt64 c2 = r1.size();
UInt64 inter = this->rb_and_cardinality( r1 );
UInt64 inter = this->rb_and_cardinality(r1);
return c1 + c2 - inter;
}
@ -264,7 +277,7 @@ public:
{
UInt64 c1 = this->size();
UInt64 c2 = r1.size();
UInt64 inter = this->rb_and_cardinality( r1 );
UInt64 inter = this->rb_and_cardinality(r1);
return c1 + c2 - 2 * inter;
}
@ -274,7 +287,7 @@ public:
UInt64 rb_andnot_cardinality(const RoaringBitmapWithSmallSet & r1) const
{
UInt64 c1 = this->size();
UInt64 inter = this->rb_and_cardinality( r1 );
UInt64 inter = this->rb_and_cardinality(r1);
return c1 - inter;
}
@ -283,10 +296,12 @@ public:
*/
UInt8 rb_equals(const RoaringBitmapWithSmallSet & r1)
{
if( this->isSmall() ) toLarge();
if (this->isSmall())
toLarge();
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
UInt8 is_true = roaring_bitmap_equals( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
UInt8 is_true = roaring_bitmap_equals(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
return is_true;
}
@ -295,10 +310,12 @@ public:
*/
UInt8 rb_intersect(const RoaringBitmapWithSmallSet & r1)
{
if( this->isSmall() ) toLarge();
if (this->isSmall())
toLarge();
roaring_bitmap_t * rb1 = r1.isSmall() ? r1.getNewRbFromSmall() : r1.getRb();
UInt8 is_true = roaring_bitmap_intersect( rb, rb1 );
if ( r1.isSmall() ) roaring_bitmap_free(rb1);
UInt8 is_true = roaring_bitmap_intersect(rb, rb1);
if (r1.isSmall())
roaring_bitmap_free(rb1);
return is_true;
}
@ -307,8 +324,9 @@ public:
*/
void rb_remove(UInt64 offsetid)
{
if( this->isSmall() ) toLarge();
roaring_bitmap_remove( rb, offsetid );
if (this->isSmall())
toLarge();
roaring_bitmap_remove(rb, offsetid);
}
/**
@ -319,8 +337,9 @@ public:
*/
void rb_flip(UInt64 offsetstart, UInt64 offsetend)
{
if( this->isSmall() ) toLarge();
roaring_bitmap_flip_inplace( rb, offsetstart, offsetend );
if (this->isSmall())
toLarge();
roaring_bitmap_flip_inplace(rb, offsetstart, offsetend);
}
/**
@ -328,8 +347,9 @@ public:
*/
UInt64 rb_rank(UInt64 offsetid)
{
if( this->isSmall() ) toLarge();
return roaring_bitmap_rank( rb, offsetid );
if (this->isSmall())
toLarge();
return roaring_bitmap_rank(rb, offsetid);
}
/**
@ -346,10 +366,12 @@ public:
res_data.emplace_back(x);
count++;
}
} else {
}
else
{
roaring_uint32_iterator_t iterator;
roaring_init_iterator(rb, &iterator);
while(iterator.has_value)
while (iterator.has_value)
{
res_data.emplace_back(iterator.current_value);
roaring_advance_uint32_iterator(&iterator);
@ -361,78 +383,80 @@ public:
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 ) {
void *container = NULL; // hold value of last container touched
uint8_t typecode = 0; // typecode of last container touched
uint32_t prev = 0; // previous valued inserted
size_t i = 0; // index of value
void db_roaring_bitmap_add_many(DB::ReadBuffer & dbBuf, roaring_bitmap_t * r, size_t n_args)
{
void * container = NULL; // hold value of last container touched
uint8_t typecode = 0; // typecode of last container touched
uint32_t prev = 0; // previous valued inserted
size_t i = 0; // index of value
int containerindex = 0;
if (n_args == 0) return;
if (n_args == 0)
return;
uint32_t val;
readBinary( val, dbBuf );
readBinary(val, dbBuf);
container = containerptr_roaring_bitmap_add(r, val, &typecode, &containerindex);
prev = val;
i++;
for (; i < n_args; i++) {
readBinary( val, dbBuf );
if (((prev ^ val) >> 16) ==
0) { // no need to seek the container, it is at hand
for (; i < n_args; i++)
{
readBinary(val, dbBuf);
if (((prev ^ val) >> 16) == 0)
{ // no need to seek the container, it is at hand
// because we already have the container at hand, we can do the
// insertion
// automatically, bypassing the roaring_bitmap_add call
uint8_t newtypecode = typecode;
void * container2 =
container_add(container, val & 0xFFFF, typecode, &newtypecode);
void * container2 = container_add(container, val & 0xFFFF, typecode, &newtypecode);
// rare instance when we need to
if (container2 != container)
{
// change the container type
container_free(container, typecode);
ra_set_container_at_index(&r->high_low_container,
containerindex, container2,
newtypecode);
ra_set_container_at_index(&r->high_low_container, containerindex, container2, newtypecode);
typecode = newtypecode;
container = container2;
}
} else {
container = containerptr_roaring_bitmap_add(r, val, &typecode,
&containerindex);
}
else
{
container = containerptr_roaring_bitmap_add(r, val, &typecode, &containerindex);
}
prev = val;
}
}
void db_ra_to_uint32_array( DB::WriteBuffer & dbBuf, roaring_array_t * ra ) const {
void db_ra_to_uint32_array(DB::WriteBuffer & dbBuf, roaring_array_t * ra) const
{
size_t ctr = 0;
for (Int32 i = 0; i < ra->size; ++i)
{
Int32 num_added = db_container_to_uint32_array( dbBuf, ra->containers[i], ra->typecodes[i],
((UInt32)ra->keys[i]) << 16);
Int32 num_added = db_container_to_uint32_array(dbBuf, ra->containers[i], ra->typecodes[i], ((UInt32)ra->keys[i]) << 16);
ctr += num_added;
}
}
UInt32 db_container_to_uint32_array( DB::WriteBuffer & dbBuf, const void * container, UInt8 typecode, UInt32 base) const {
UInt32 db_container_to_uint32_array(DB::WriteBuffer & dbBuf, const void * container, UInt8 typecode, UInt32 base) const
{
container = container_unwrap_shared(container, &typecode);
switch (typecode) {
switch (typecode)
{
case BITSET_CONTAINER_TYPE_CODE:
return db_bitset_container_to_uint32_array( dbBuf,
(const bitset_container_t *)container, base);
return db_bitset_container_to_uint32_array(dbBuf, (const bitset_container_t *)container, base);
case ARRAY_CONTAINER_TYPE_CODE:
return db_array_container_to_uint32_array( dbBuf,
(const array_container_t *)container, base);
return db_array_container_to_uint32_array(dbBuf, (const array_container_t *)container, base);
case RUN_CONTAINER_TYPE_CODE:
return db_run_container_to_uint32_array( dbBuf,
(const run_container_t *)container, base);
return db_run_container_to_uint32_array(dbBuf, (const run_container_t *)container, base);
}
return 0;
}
UInt32 db_bitset_container_to_uint32_array( DB::WriteBuffer & dbBuf, const bitset_container_t * cont, UInt32 base) const {
return (UInt32) db_bitset_extract_setbits( dbBuf, cont->array, BITSET_CONTAINER_SIZE_IN_WORDS, base);
UInt32 db_bitset_container_to_uint32_array(DB::WriteBuffer & dbBuf, const bitset_container_t * cont, UInt32 base) const
{
return (UInt32)db_bitset_extract_setbits(dbBuf, cont->array, BITSET_CONTAINER_SIZE_IN_WORDS, base);
}
size_t db_bitset_extract_setbits( DB::WriteBuffer & dbBuf, UInt64 * bitset, size_t length, UInt32 base) const {
size_t db_bitset_extract_setbits(DB::WriteBuffer & dbBuf, UInt64 * bitset, size_t length, UInt32 base) const
{
UInt32 outpos = 0;
for (size_t i = 0; i < length; ++i)
{
@ -442,7 +466,7 @@ private:
UInt64 t = w & (~w + 1); // on x64, should compile to BLSI (careful: the Intel compiler seems to fail)
UInt32 r = __builtin_ctzll(w); // on x64, should compile to TZCNT
UInt32 val = r + base;
writePODBinary( val, dbBuf );
writePODBinary(val, dbBuf);
outpos++;
w ^= t;
}
@ -451,18 +475,20 @@ private:
return outpos;
}
int db_array_container_to_uint32_array( DB::WriteBuffer & dbBuf, const array_container_t * cont, UInt32 base) const {
int db_array_container_to_uint32_array(DB::WriteBuffer & dbBuf, const array_container_t * cont, UInt32 base) const
{
UInt32 outpos = 0;
for ( Int32 i = 0; i < cont->cardinality; ++i)
for (Int32 i = 0; i < cont->cardinality; ++i)
{
const UInt32 val = base + cont->array[i];
writePODBinary( val, dbBuf );
writePODBinary(val, dbBuf);
outpos++;
}
return outpos;
}
int db_run_container_to_uint32_array( DB::WriteBuffer & dbBuf, const run_container_t * cont, UInt32 base) const {
int db_run_container_to_uint32_array(DB::WriteBuffer & dbBuf, const run_container_t * cont, UInt32 base) const
{
UInt32 outpos = 0;
for (Int32 i = 0; i < cont->n_runs; ++i)
{
@ -471,20 +497,18 @@ private:
for (Int32 j = 0; j <= le; ++j)
{
UInt32 val = run_start + j;
writePODBinary( val, dbBuf );
writePODBinary(val, dbBuf);
outpos++;
}
}
return outpos;
}
};
template <typename T>
struct AggregateFunctionGroupBitmapData
{
RoaringBitmapWithSmallSet<T,32> rbs;
RoaringBitmapWithSmallSet<T, 32> rbs;
static const char * name() { return "groupBitmap"; }
};

View File

@ -1,23 +1,21 @@
#pragma once
#include <Common/typeid_cast.h>
#include <Columns/ColumnVector.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionGroupBitmapData.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnFunction.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnAggregateFunction.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <AggregateFunctions/AggregateFunctionGroupBitmapData.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Functions/IFunction.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
@ -80,13 +78,15 @@ public:
auto array_type = typeid_cast<const DataTypeArray *>(arguments[0].get());
if (!array_type)
throw Exception("First argument for function " + getName() + " must be an array but it has type "
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"First argument for function " + getName() + " must be an array but it has type " + arguments[0]->getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
auto nested_type = array_type->getNestedType();
DataTypes argument_types = {nested_type};
Array params_row;
AggregateFunctionPtr bitmap_function = AggregateFunctionFactory::instance().get(AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row);
AggregateFunctionPtr bitmap_function
= AggregateFunctionFactory::instance().get(AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row);
return std::make_shared<DataTypeAggregateFunction>(bitmap_function, argument_types, params_row);
}
@ -102,18 +102,22 @@ public:
DataTypes argument_types = {nested_type};
WhichDataType which(nested_type);
if (which.isUInt8()) executeBitmapData<UInt8>(block, argument_types, arguments, result);
else if (which.isUInt16()) executeBitmapData<UInt16>(block, argument_types, arguments, result);
else if (which.isUInt32()) executeBitmapData<UInt32>(block, argument_types, arguments, result);
else if (which.isUInt64()) executeBitmapData<UInt64>(block, argument_types, arguments, result);
if (which.isUInt8())
executeBitmapData<UInt8>(block, argument_types, arguments, result);
else if (which.isUInt16())
executeBitmapData<UInt16>(block, argument_types, arguments, result);
else if (which.isUInt32())
executeBitmapData<UInt32>(block, argument_types, arguments, result);
else if (which.isUInt64())
executeBitmapData<UInt64>(block, argument_types, arguments, result);
else
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
template <typename T>
void executeBitmapData( Block & block, DataTypes & argument_types, const ColumnNumbers & arguments, size_t result)
void executeBitmapData(Block & block, DataTypes & argument_types, const ColumnNumbers & arguments, size_t result)
{
// input data
const ColumnArray * array = typeid_cast<const ColumnArray *>(block.getByPosition(arguments[0]).column.get());
@ -124,15 +128,17 @@ private:
// output data
Array params_row;
AggregateFunctionPtr bitmap_function = AggregateFunctionFactory::instance().get(AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row);
AggregateFunctionPtr bitmap_function
= AggregateFunctionFactory::instance().get(AggregateFunctionGroupBitmapData<UInt32>::name(), argument_types, params_row);
auto col_to = ColumnAggregateFunction::create(bitmap_function);
col_to->reserve(offsets.size());
size_t pos = 0;
for(size_t i = 0; i < offsets.size(); ++i)
for (size_t i = 0; i < offsets.size(); ++i)
{
col_to->insertDefault();
AggregateFunctionGroupBitmapData<T>& bitmap_data = *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>( col_to->getData()[i] );
AggregateFunctionGroupBitmapData<T> & bitmap_data
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
for (; pos < offsets[i]; ++pos)
{
bitmap_data.rbs.add(input_data[pos]);
@ -158,11 +164,13 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>( arguments[0].get() );
if (!aggr_type)
throw Exception("First argument for function " + getName() + " must be an AggregateFunction but it has type "
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypePtr data_type = aggr_type->getArgumentsDataTypes()[0];
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() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const DataTypePtr data_type = bitmap_type->getArgumentsDataTypes()[0];
return std::make_shared<DataTypeArray>(data_type);
}
@ -182,32 +190,40 @@ public:
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, input_rows_count, res_data, res_offsets);
else if (which.isUInt16()) executeIntType<UInt16>(block, arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt32()) executeIntType<UInt32>(block, arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt64()) executeIntType<UInt64>(block, arguments, input_rows_count, res_data, res_offsets);
if (which.isUInt8())
executeIntType<UInt8>(block, arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt16())
executeIntType<UInt16>(block, arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt32())
executeIntType<UInt32>(block, arguments, input_rows_count, res_data, res_offsets);
else if (which.isUInt64())
executeIntType<UInt64>(block, 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);
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(res_ptr);
}
private:
using ToType = UInt64;
template <typename T>
void executeIntType( Block & block, const ColumnNumbers & arguments, size_t input_rows_count,
IColumn & res_data_col, ColumnArray::Offsets & res_offsets ) const
void executeIntType(
Block & block, const ColumnNumbers & arguments, size_t input_rows_count, IColumn & res_data_col, ColumnArray::Offsets & res_offsets)
const
{
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
const ColumnAggregateFunction * column
= typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
PaddedPODArray<T> & res_data = typeid_cast<ColumnVector<T> &>(res_data_col).getData();
ColumnArray::Offset res_offset = 0;
for(size_t i = 0; i < input_rows_count; ++i)
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T>& bd1 = *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>( column->getData()[i] );
UInt64 count = bd1.rbs.rb_to_array( res_data );
const AggregateFunctionGroupBitmapData<T> & bd1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(column->getData()[i]);
UInt64 count = bd1.rbs.rb_to_array(res_data);
res_offset += count;
res_offsets.emplace_back(res_offset);
}
@ -228,8 +244,15 @@ public:
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments */) const override
{ return std::make_shared<DataTypeNumber<ToType>>(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
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() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<ToType>>();
}
bool useDefaultImplementationForConstants() const override { return true; }
@ -241,26 +264,34 @@ public:
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8()) executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt16()) executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt32()) executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt64()) executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
if (which.isUInt8())
executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt16())
executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt32())
executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt64())
executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
else
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(col_to);
}
private:
using ToType = UInt64;
template <typename T>
void executeIntType( Block & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to)
void executeIntType(
Block & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to)
{
const ColumnAggregateFunction * column = typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
for(size_t i = 0; i < input_rows_count; ++i)
const ColumnAggregateFunction * column
= typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[0]).column.get());
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T>& bd1 = *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>( column->getData()[i] );
const AggregateFunctionGroupBitmapData<T> & bd1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(column->getData()[i]);
vec_to[i] = bd1.rbs.size();
}
}
@ -270,10 +301,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> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
{
// roaring_bitmap_and_cardinality( rb1, rb2 );
return bd1.rbs.rb_and_cardinality( bd2.rbs );
return bd1.rbs.rb_and_cardinality(bd2.rbs);
}
};
@ -282,10 +313,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> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
{
// return roaring_bitmap_or_cardinality( rb1, rb2 );
return bd1.rbs.rb_or_cardinality( bd2.rbs );
return bd1.rbs.rb_or_cardinality(bd2.rbs);
}
};
@ -293,10 +324,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> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
{
// return roaring_bitmap_xor_cardinality( rb1, rb2 );
return bd1.rbs.rb_xor_cardinality( bd2.rbs );
return bd1.rbs.rb_xor_cardinality(bd2.rbs);
}
};
@ -304,10 +335,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> & bd1, const AggregateFunctionGroupBitmapData<T> & bd2)
{
// roaring_bitmap_andnot_cardinality( rb1, rb2 );
return bd1.rbs.rb_andnot_cardinality( bd2.rbs );
return bd1.rbs.rb_andnot_cardinality(bd2.rbs);
}
};
@ -316,7 +347,7 @@ class FunctionBitmapCardinality : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionBitmapCardinality>(); }
String getName() const override { return name; }
@ -325,8 +356,21 @@ public:
size_t getNumberOfArguments() const override { return 2; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments */) const override
{ return std::make_shared<DataTypeNumber<ToType>>(); }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
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() + ".",
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() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNumber<ToType>>();
}
bool useDefaultImplementationForConstants() const override { return true; }
@ -338,31 +382,39 @@ public:
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>(from_type);
WhichDataType which(aggr_type->getArgumentsDataTypes()[0]);
if (which.isUInt8()) executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt16()) executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt32()) executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt64()) executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
if (which.isUInt8())
executeIntType<UInt8>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt16())
executeIntType<UInt16>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt32())
executeIntType<UInt32>(block, arguments, input_rows_count, vec_to);
else if (which.isUInt64())
executeIntType<UInt64>(block, arguments, input_rows_count, vec_to);
else
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
block.getByPosition(result).column = std::move(col_to);
}
private:
using ToType = UInt64;
template <typename T>
void executeIntType( Block & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to)
void executeIntType(
Block & block, const ColumnNumbers & arguments, size_t input_rows_count, typename ColumnVector<ToType>::Container & vec_to)
{
const ColumnAggregateFunction * columns[2];
for (size_t i = 0; i < 2; ++i)
columns[i] = typeid_cast<const ColumnAggregateFunction *>(block.getByPosition(arguments[i]).column.get());
for(size_t i = 0; i < input_rows_count; ++i)
for (size_t i = 0; i < input_rows_count; ++i)
{
const AggregateFunctionGroupBitmapData<T>& bd1 = *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>( columns[0]->getData()[i] );
const AggregateFunctionGroupBitmapData<T>& bd2 = *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>( columns[1]->getData()[i] );
vec_to[i] = Impl<T>::apply( bd1, bd2 );
const AggregateFunctionGroupBitmapData<T> & bd1
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(columns[0]->getData()[i]);
const AggregateFunctionGroupBitmapData<T> & bd2
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(columns[1]->getData()[i]);
vec_to[i] = Impl<T>::apply(bd1, bd2);
}
}
};
@ -370,36 +422,36 @@ private:
template <typename T>
struct BitmapAndImpl
{
static void apply( AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2 )
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
{
toBd.rbs.rb_and( bd2.rbs );
toBd.rbs.rb_and(bd2.rbs);
}
};
template <typename T>
struct BitmapOrImpl
{
static void apply( AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2 )
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
{
toBd.rbs.rb_or( bd2.rbs );
toBd.rbs.rb_or(bd2.rbs);
}
};
template <typename T>
struct BitmapXorImpl
{
static void apply( AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2 )
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
{
toBd.rbs.rb_xor( bd2.rbs );
toBd.rbs.rb_xor(bd2.rbs);
}
};
template <typename T>
struct BitmapAndnotImpl
{
static void apply( AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2 )
static void apply(AggregateFunctionGroupBitmapData<T> & toBd, const AggregateFunctionGroupBitmapData<T> & bd2)
{
toBd.rbs.rb_andnot( bd2.rbs );
toBd.rbs.rb_andnot(bd2.rbs);
}
};
@ -419,10 +471,17 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeAggregateFunction * aggr_type = typeid_cast<const DataTypeAggregateFunction *>( arguments[0].get() );
if (!aggr_type)
throw Exception("First argument for function " + getName() + " must be an AggregateFunction but it has type "
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
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() + ".",
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() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0];
}
@ -433,18 +492,22 @@ public:
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()) executeBitmapData<UInt8>(block, arguments, result, input_rows_count);
else if (which.isUInt16()) executeBitmapData<UInt16>(block, arguments, result, input_rows_count);
else if (which.isUInt32()) executeBitmapData<UInt32>(block, arguments, result, input_rows_count);
else if (which.isUInt64()) executeBitmapData<UInt64>(block, arguments, result, input_rows_count);
if (which.isUInt8())
executeBitmapData<UInt8>(block, arguments, result, input_rows_count);
else if (which.isUInt16())
executeBitmapData<UInt16>(block, arguments, result, input_rows_count);
else if (which.isUInt32())
executeBitmapData<UInt32>(block, arguments, result, input_rows_count);
else if (which.isUInt64())
executeBitmapData<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);
throw Exception(
"Unexpected type " + from_type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
private:
template <typename T>
void executeBitmapData( Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
void executeBitmapData(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
{
const ColumnAggregateFunction * columns[2];
for (size_t i = 0; i < 2; ++i)
@ -454,46 +517,77 @@ private:
col_to->reserve(input_rows_count);
for(size_t i = 0; i < input_rows_count; ++i)
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 = *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>( columns[1]->getData()[i] );
Impl<T>::apply( toBd, bd2 );
AggregateFunctionGroupBitmapData<T> & toBd = *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
const AggregateFunctionGroupBitmapData<T> & bd2
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(columns[1]->getData()[i]);
Impl<T>::apply(toBd, bd2);
}
block.getByPosition(result).column = std::move(col_to);
}
};
struct NameBitmapBuild { static constexpr auto name = "bitmapBuild"; };
struct NameBitmapBuild
{
static constexpr auto name = "bitmapBuild";
};
using FunctionBitmapBuild = FunctionBitmapBuildImpl<NameBitmapBuild>;
struct NameBitmapToArray { static constexpr auto name = "bitmapToArray"; };
struct NameBitmapToArray
{
static constexpr auto name = "bitmapToArray";
};
using FunctionBitmapToArray = FunctionBitmapToArrayImpl<NameBitmapToArray>;
struct NameBitmapCardinality { static constexpr auto name = "bitmapCardinality"; };
struct NameBitmapAndCardinality { static constexpr auto name = "bitmapAndCardinality"; };
struct NameBitmapOrCardinality { static constexpr auto name = "bitmapOrCardinality"; };
struct NameBitmapXorCardinality { static constexpr auto name = "bitmapXorCardinality"; };
struct NameBitmapAndnotCardinality { static constexpr auto name = "bitmapAndnotCardinality"; };
struct NameBitmapCardinality
{
static constexpr auto name = "bitmapCardinality";
};
struct NameBitmapAndCardinality
{
static constexpr auto name = "bitmapAndCardinality";
};
struct NameBitmapOrCardinality
{
static constexpr auto name = "bitmapOrCardinality";
};
struct NameBitmapXorCardinality
{
static constexpr auto name = "bitmapXorCardinality";
};
struct NameBitmapAndnotCardinality
{
static constexpr auto name = "bitmapAndnotCardinality";
};
using FunctionBitmapSelfCardinality = FunctionBitmapSelfCardinalityImpl<NameBitmapCardinality>;
using FunctionBitmapAndCardinality = FunctionBitmapCardinality<BitmapAndCardinalityImpl, NameBitmapAndCardinality>;
using FunctionBitmapOrCardinality = FunctionBitmapCardinality<BitmapOrCardinalityImpl, NameBitmapOrCardinality>;
using FunctionBitmapXorCardinality = FunctionBitmapCardinality<BitmapXorCardinalityImpl, NameBitmapXorCardinality>;
using FunctionBitmapSelfCardinality = FunctionBitmapSelfCardinalityImpl<NameBitmapCardinality>;
using FunctionBitmapAndCardinality = FunctionBitmapCardinality<BitmapAndCardinalityImpl, NameBitmapAndCardinality>;
using FunctionBitmapOrCardinality = FunctionBitmapCardinality<BitmapOrCardinalityImpl, NameBitmapOrCardinality>;
using FunctionBitmapXorCardinality = FunctionBitmapCardinality<BitmapXorCardinalityImpl, NameBitmapXorCardinality>;
using FunctionBitmapAndnotCardinality = FunctionBitmapCardinality<BitmapAndnotCardinalityImpl, NameBitmapAndnotCardinality>;
struct NameBitmapAnd { static constexpr auto name = "bitmapAnd"; };
struct NameBitmapOr { static constexpr auto name = "bitmapOr"; };
struct NameBitmapXor { static constexpr auto name = "bitmapXor"; };
struct NameBitmapAndnot { static constexpr auto name = "bitmapAndnot"; };
using FunctionBitmapAnd = FunctionBitmap<BitmapAndImpl, NameBitmapAnd>;
using FunctionBitmapOr = FunctionBitmap<BitmapOrImpl, NameBitmapOr>;
using FunctionBitmapXor = FunctionBitmap<BitmapXorImpl, NameBitmapXor>;
struct NameBitmapAnd
{
static constexpr auto name = "bitmapAnd";
};
struct NameBitmapOr
{
static constexpr auto name = "bitmapOr";
};
struct NameBitmapXor
{
static constexpr auto name = "bitmapXor";
};
struct NameBitmapAndnot
{
static constexpr auto name = "bitmapAndnot";
};
using FunctionBitmapAnd = FunctionBitmap<BitmapAndImpl, NameBitmapAnd>;
using FunctionBitmapOr = FunctionBitmap<BitmapOrImpl, NameBitmapOr>;
using FunctionBitmapXor = FunctionBitmap<BitmapXorImpl, NameBitmapXor>;
using FunctionBitmapAndnot = FunctionBitmap<BitmapAndnotImpl, NameBitmapAndnot>;
}