mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Update FunctionsBitmap.h
This commit is contained in:
parent
f9d14f78c0
commit
0c1510d3e3
@ -249,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);
|
||||
}
|
||||
@ -325,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;
|
||||
|
||||
@ -335,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
|
||||
@ -352,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
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
|
||||
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);
|
||||
}
|
||||
@ -377,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -388,15 +385,16 @@ 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:
|
||||
@ -417,11 +415,11 @@ public:
|
||||
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++)
|
||||
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() + ".";
|
||||
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);
|
||||
|
||||
@ -458,12 +456,11 @@ private:
|
||||
|
||||
template <typename T>
|
||||
void executeIntType(
|
||||
Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
|
||||
const
|
||||
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 * colAggFunc;
|
||||
const ColumnAggregateFunction * col_agg_func;
|
||||
const PaddedPODArray<AggregateDataPtr> * container0;
|
||||
const ColumnArray * array;
|
||||
|
||||
@ -474,13 +471,13 @@ private:
|
||||
}
|
||||
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]);
|
||||
col_agg_func = typeid_cast<const ColumnAggregateFunction*>(columns[0]);
|
||||
}
|
||||
container0 = &colAggFunc->getData();
|
||||
container0 = &col_agg_func->getData();
|
||||
|
||||
if (is_column_const[1])
|
||||
array = typeid_cast<const ColumnArray*>(typeid_cast<const ColumnConst*>(columns[1])->getDataColumnPtr().get());
|
||||
@ -494,20 +491,22 @@ private:
|
||||
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(colAggFunc->getAggregateFunction());
|
||||
auto col_to = ColumnAggregateFunction::create(col_agg_func->getAggregateFunction());
|
||||
col_to->reserve(input_rows_count);
|
||||
|
||||
size_t from_start, from_end, to_start, to_end;
|
||||
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 dataPtr0 = is_column_const[0] ? (*container0)[0] : (*container0)[i];
|
||||
const AggregateFunctionGroupBitmapData<T>& bd0
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data_0
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T> *>(dataPtr0);
|
||||
if (is_column_const[1])
|
||||
{
|
||||
from_start = 0;
|
||||
@ -515,7 +514,7 @@ private:
|
||||
}
|
||||
else
|
||||
{
|
||||
from_start = i==0 ? 0 : from_offsets[i-1];
|
||||
from_start = i == 0 ? 0 : from_offsets[i - 1];
|
||||
from_end = from_offsets[i];
|
||||
}
|
||||
if (is_column_const[2])
|
||||
@ -525,17 +524,17 @@ private:
|
||||
}
|
||||
else
|
||||
{
|
||||
to_start = i==0 ? 0 : to_offsets[i-1];
|
||||
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> & bd2
|
||||
AggregateFunctionGroupBitmapData<T> & bitmap_data_2
|
||||
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
|
||||
bd2.rbs.merge(bd0.rbs);
|
||||
bd2.rbs.rb_replace(&from_container[from_start], &to_container[to_start], from_end - from_start);
|
||||
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);
|
||||
}
|
||||
@ -601,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);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -613,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();
|
||||
}
|
||||
};
|
||||
|
||||
@ -624,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();
|
||||
}
|
||||
};
|
||||
|
||||
@ -635,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();
|
||||
}
|
||||
};
|
||||
|
||||
@ -645,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -657,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -668,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -679,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -690,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -700,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -788,9 +787,9 @@ private:
|
||||
{
|
||||
const AggregateDataPtr dataPtr0 = 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> *>(dataPtr0);
|
||||
vec_to[i] = bitmap_data_0.rbs.rb_contains(data1);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -885,11 +884,11 @@ private:
|
||||
{
|
||||
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
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data_1
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr0);
|
||||
const AggregateFunctionGroupBitmapData<T> & bd2
|
||||
const AggregateFunctionGroupBitmapData<T> & bitmap_data_2
|
||||
= *reinterpret_cast<const AggregateFunctionGroupBitmapData<T>*>(dataPtr1);
|
||||
vec_to[i] = Impl<T>::apply(bd1, bd2);
|
||||
vec_to[i] = Impl<T>::apply(bitmap_data_1, bitmap_data_2);
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -897,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);
|
||||
}
|
||||
};
|
||||
|
||||
@ -1007,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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user