Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into url_engine_problem

This commit is contained in:
Mikhail Korotov 2019-11-05 18:47:05 +03:00
commit 1bbf9b2ef9
20 changed files with 662 additions and 205 deletions

View File

@ -1,8 +1,6 @@
I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en
For changelog. Remove if this is non-significant change.
Category (leave one):
Changelog category (leave one):
- New Feature
- Bug Fix
- Improvement
@ -11,11 +9,14 @@ Category (leave one):
- Build/Testing/Packaging Improvement
- Documentation
- Other
- Non-significant (changelog entry is not needed)
Short description (up to few sentences):
Changelog entry (up to few sentences, not needed for non-significant PRs):
...
Detailed description (optional):
...

View File

@ -1,4 +1,4 @@
## ClickHouse release v19.16.2.2, 30-10-2019
## ClickHouse release v19.16.2.2, 2019-10-30
### Backward Incompatible Change
* Add missing arity validation for count/counIf.

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

@ -25,6 +25,7 @@ void registerFunctionsCoding(FunctionFactory & factory)
factory.registerFunction<FunctionUUIDStringToNum>();
factory.registerFunction<FunctionHex>();
factory.registerFunction<FunctionUnhex>();
factory.registerFunction<FunctionChar>(FunctionFactory::CaseInsensitive);
factory.registerFunction<FunctionBitmaskToArray>();
factory.registerFunction<FunctionToIPv4>();
factory.registerFunction<FunctionToIPv6>();

View File

@ -1276,6 +1276,100 @@ public:
}
};
class FunctionChar : public IFunction
{
public:
static constexpr auto name = "char";
static FunctionPtr create(const Context &) { return std::make_shared<FunctionChar>(); }
String getName() const override
{
return name;
}
bool isVariadic() const override { return true; }
bool isInjective(const Block &) override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("Number of arguments for function " + getName() + " can't be " + toString(arguments.size())
+ ", should be at least 1", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto & arg : arguments)
{
WhichDataType which(arg);
if (!(which.isInt() || which.isUInt() || which.isFloat()))
throw Exception("Illegal type " + arg->getName() + " of argument of function " + getName()
+ ", must be Int, UInt or Float number",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
auto col_str = ColumnString::create();
ColumnString::Chars & out_vec = col_str->getChars();
ColumnString::Offsets & out_offsets = col_str->getOffsets();
const auto size_per_row = arguments.size() + 1;
out_vec.resize(size_per_row * input_rows_count);
out_offsets.resize(input_rows_count);
for (size_t row = 0; row < input_rows_count; ++row)
{
out_offsets[row] = size_per_row + out_offsets[row - 1];
out_vec[row * size_per_row + size_per_row - 1] = '\0';
}
Columns columns_holder(arguments.size());
for (size_t idx = 0; idx < arguments.size(); ++idx)
{
//partial const column
columns_holder[idx] = block.getByPosition(arguments[idx]).column->convertToFullColumnIfConst();
const IColumn * column = columns_holder[idx].get();
if (!(executeNumber<UInt8>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<UInt16>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<UInt32>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<UInt64>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Int8>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Int16>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Int32>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Int64>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Float32>(*column, out_vec, idx, input_rows_count, size_per_row)
|| executeNumber<Float64>(*column, out_vec, idx, input_rows_count, size_per_row)))
{
throw Exception{"Illegal column " + block.getByPosition(arguments[idx]).column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
}
block.getByPosition(result).column = std::move(col_str);
}
private:
template <typename T>
bool executeNumber(const IColumn & src_data, ColumnString::Chars & out_vec, const size_t & column_idx, const size_t & rows, const size_t & size_per_row)
{
const ColumnVector<T> * src_data_concrete = checkAndGetColumn<ColumnVector<T>>(&src_data);
if (!src_data_concrete)
{
return false;
}
for (size_t row = 0; row < rows; ++row)
{
out_vec[row * size_per_row + column_idx] = static_cast<char>(src_data_concrete->getInt(row));
}
return true;
}
};
class FunctionBitmaskToArray : public IFunction
{

View File

@ -678,7 +678,7 @@ public:
MergeTreeDataFormatVersion format_version;
Context global_context;
Context & global_context;
/// Merging params - what additional actions to perform during merge.
const MergingParams merging_params;

View File

@ -0,0 +1,6 @@
<yandex>
<part_log>
<database>database_name</database>
<table>table_name</table>
</part_log>
</yandex>

View File

@ -0,0 +1,4 @@
<yandex>
<part_log>
</part_log>
</yandex>

View File

@ -0,0 +1,43 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance("node1")
node2 = cluster.add_instance("node2", main_configs=["configs/config_with_standard_part_log.xml"])
node3 = cluster.add_instance("node3", main_configs=["configs/config_with_non_standard_part_log.xml"])
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_config_without_part_log(start_cluster):
assert "Table system.part_log doesn't exist" in node1.query_and_get_error("SELECT * FROM system.part_log")
node1.query("CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() ORDER BY value")
assert "Table system.part_log doesn't exist" in node1.query_and_get_error("SELECT * FROM system.part_log")
node1.query("INSERT INTO test_table VALUES ('name', 1)")
time.sleep(10)
assert "Table system.part_log doesn't exist" in node1.query_and_get_error("SELECT * FROM system.part_log")
def test_config_with_standard_part_log(start_cluster):
assert "Table system.part_log doesn't exist" in node2.query_and_get_error("SELECT * FROM system.part_log")
node2.query("CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() Order by value")
assert "Table system.part_log doesn't exist" in node2.query_and_get_error("SELECT * FROM system.part_log")
node2.query("INSERT INTO test_table VALUES ('name', 1)")
time.sleep(10)
assert node2.query("SELECT * FROM system.part_log") != ""
def test_config_with_non_standard_part_log(start_cluster):
node3.query("CREATE DATABASE database_name")
assert "table_name" not in node3.query("SHOW TABLES FROM database_name")
node3.query("CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() ORDER BY value")
assert "table_name" not in node3.query("SHOW TABLES FROM database_name")
node3.query("INSERT INTO test_table VALUES ('name', 1)")
time.sleep(10)
assert "table_name" in node3.query("SHOW TABLES FROM database_name")

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

@ -0,0 +1,5 @@
ABCDabcdefg
ABC
ABC
ACD
ADE

View File

@ -0,0 +1,4 @@
/* char function */
SELECT char(65, 66.1, 67.2, 68.3, 97.4, 98.5, 99.6, 100.7, 101.0, 102.0, 103.0);
SELECT char(65 + 256, 66 + 1024, 66 + 1024 + 1);
SELECT char(65, 66 + number, 67 + number) from numbers(3);

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.

View File

@ -1,5 +1,8 @@
# Encoding functions
## char
Accepts multiple arguments of `Number` types. Returns a string, each char of the results corresponds to the ascii character of the input numbers. It'll cast the first byte from the number, if the byte overflows the range of ascii(which is 127), it returns an unrecognized character(<28>).
## hex
Accepts arguments of types: `String`, `unsigned integer`, `Date`, or `DateTime`. Returns a string containing the argument's hexadecimal representation. Uses uppercase letters `A-F`. Does not use `0x` prefixes or `h` suffixes. For strings, all bytes are simply encoded as two hexadecimal numbers. Numbers are converted to big endian ("human readable") format. For numbers, older zeros are trimmed, but only by entire bytes. For example, `hex (1) = '01'`. `Date` is encoded as the number of days since the beginning of the Unix epoch. `DateTime` is encoded as the number of seconds since the beginning of the Unix epoch.

View File

@ -1,3 +1,64 @@
# AggregateFunction(name, types_of_arguments...)
# AggregateFunction(name, types_of_arguments...) {#data_type-aggregatefunction}
表示聚合函数中的中间状态。可以在聚合函数中通过 '-State' 后缀来访问它。更多信息,参考 "AggregatingMergeTree"。
聚合函数的中间状态,可以通过聚合函数名称加`-State`后缀的形式得到它。与此同时,当您需要访问该类型的最终状态数据时,您需要以相同的聚合函数名加`-Merge`后缀的形式来得到最终状态数据。
`AggregateFunction` — 参数化的数据类型。
**参数**
- 聚合函数名
如果函数具备多个参数列表,请在此处指定其他参数列表中的值。
- 聚合函数参数的类型
**示例**
```sql
CREATE TABLE t
(
column1 AggregateFunction(uniq, UInt64),
column2 AggregateFunction(anyIf, String, UInt8),
column3 AggregateFunction(quantiles(0.5, 0.9), UInt64)
) ENGINE = ...
```
上述中的[uniq](../../query_language/agg_functions/reference.md#agg_function-uniq) anyIf ([any](../../query_language/agg_functions/reference.md#agg_function-any)+[If](../../query_language/agg_functions/combinators.md#agg-functions-combinator-if)) 以及 [quantiles](../../query_language/agg_functions/reference.md) 都为ClickHouse中支持的聚合函数。
## 使用指南
### 数据写入
当需要写入数据时,您需要将数据包含在`INSERT SELECT`语句中,同时对于`AggregateFunction`类型的数据,您需要使用对应的以`-State`为后缀的函数进行处理。
**函数使用示例**
```sql
uniqState(UserID)
quantilesState(0.5, 0.9)(SendTiming)
```
不同于`uniq`和`quantiles`函数返回聚合结果的最终值,以`-State`后缀的函数总是返回`AggregateFunction`类型的数据的中间状态。
对于`SELECT`而言,`AggregateFunction`类型总是以特定的二进制形式展现在所有的输出格式中。例如,您可以使用`SELECT`语句将函数的状态数据转储为`TabSeparated`格式的同时使用`INSERT`语句将数据转储回去。
### 数据查询
当从`AggregatingMergeTree`表中查询数据时,对于`AggregateFunction`类型的字段,您需要使用以`-Merge`为后缀的相同聚合函数来聚合数据。对于非`AggregateFunction`类型的字段,请将它们包含在`GROUP BY`子句中。
以`-Merge`为后缀的聚合函数,可以将多个`AggregateFunction`类型的中间状态组合计算为最终的聚合结果。
例如,如下的两个查询返回的结果总是一致:
```sql
SELECT uniq(UserID) FROM table
SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP BY RegionID)
```
## 使用示例
请参阅 [AggregatingMergeTree](../../operations/table_engines/aggregatingmergetree.md) 的说明
[来源文章](https://clickhouse.yandex/docs/en/data_types/nested_data_structures/aggregatefunction/) <!--hide-->

View File

@ -217,49 +217,61 @@ def process_unknown_commits(commits, commits_info, users):
text = 'Commits which are not from any pull request:\n\n'
return text + '\n\n'.join(texts)
# This function mirrors the PR description checks in ClickhousePullRequestTrigger.
# Returns False if the PR should not be mentioned changelog.
def parse_one_pull_request(item):
description = item['description']
lines = [line for line in map(lambda x: x.strip(), description.split('\n')) if line]
lines = [re.sub(r'\s+', ' ', l) for l in lines]
cat_pos = None
short_descr_pos = None
long_descr_pos = None
if lines:
for i in range(len(lines) - 1):
if re.match(r'(?i)category.*:$', lines[i]):
cat_pos = i
if re.match(r'(?i)^\**\s*(Short description|Change\s*log entry)', lines[i]):
short_descr_pos = i
if re.match(r'(?i)^\**\s*Detailed description', lines[i]):
long_descr_pos = i
if cat_pos is None:
return False
cat = lines[cat_pos + 1]
cat = re.sub(r'^[-*\s]*', '', cat)
# Filter out the PR categories that are not for changelog.
if re.match(r'(?i)doc|((non|in|not|un)[-\s]*significant)', cat):
return False
short_descr = ''
if short_descr_pos:
short_descr_end = long_descr_pos or len(lines)
short_descr = lines[short_descr_pos + 1]
if short_descr_pos + 2 != short_descr_end:
short_descr += ' ...'
# If we have nothing meaningful
if not re.match('\w', short_descr):
short_descr = item['title']
# TODO: Add detailed description somewhere
item['entry'] = short_descr
item['category'] = cat
return True
# List of pull requests -> text description.
def process_pull_requests(pull_requests, users, repo):
groups = {}
for id, item in pull_requests.items():
lines = list(filter(len, map(lambda x: x.strip(), item['description'].split('\n'))))
cat_pos = None
short_descr_pos = None
long_descr_pos = None
if lines:
for i in range(len(lines) - 1):
if re.match('^\**Category', lines[i]):
cat_pos = i
if re.match('^\**\s*Short description', lines[i]):
short_descr_pos = i
if re.match('^\**\s*Detailed description', lines[i]):
long_descr_pos = i
cat = ''
if cat_pos is not None:
# TODO: Sometimes have more than one
cat = lines[cat_pos + 1]
cat = cat.strip().lstrip('-').strip()
# We are not interested in documentation PRs in changelog.
if re.match('^\**\s*(?:Documentation|Doc\s)', cat):
continue;
short_descr = ''
if short_descr_pos:
short_descr_end = long_descr_pos or len(lines)
short_descr = lines[short_descr_pos + 1]
if short_descr_pos + 2 != short_descr_end:
short_descr += ' ...'
# If we have nothing meaningful
if not re.match('\w', short_descr):
short_descr = item['title']
# TODO: Add detailed description somewhere
if not parse_one_pull_request(item):
continue
pattern = u"{} [#{}]({}) ({})"
link = 'https://github.com/{}/pull/{}'.format(repo, id)
@ -269,20 +281,21 @@ def process_pull_requests(pull_requests, users, repo):
user = users[item['user']]
author = u'[{}]({})'.format(user['name'] or user['login'], user['html_url'])
cat = item['category']
if cat not in groups:
groups[cat] = []
groups[cat].append(pattern.format(short_descr, id, link, author))
groups[cat].append(pattern.format(item['entry'], id, link, author))
categories_preferred_order = ['New Feature', 'Bug Fix', 'Improvement', 'Performance Improvement', 'Build/Testing/Packaging Improvement', 'Backward Incompatible Change', 'Other']
categories_preferred_order = ['Backward Incompatible Change', 'New Feature', 'Bug Fix', 'Improvement', 'Performance Improvement', 'Build/Testing/Packaging Improvement', 'Other']
def categories_sort_key(name):
if name in categories_preferred_order:
return categories_preferred_order.index(name)
return str(categories_preferred_order.index(name)).zfill(3)
else:
return name.lower()
texts = []
for group, text in sorted(groups.items(), key = lambda (k, v): categories_sort_key(k)):
for group, text in sorted(groups.items(), key = lambda kv: categories_sort_key(kv[0])):
items = [u'* {}'.format(pr) for pr in text]
texts.append(u'### {}\n{}'.format(group if group else u'[No category]', '\n'.join(items)))