mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 19:12:03 +00:00
Merge pull request #5170 from yandex/cidr-to-range-better-code
Cidr to range better code
This commit is contained in:
commit
9e3c80d2ed
@ -28,8 +28,8 @@ void registerFunctionsCoding(FunctionFactory & factory)
|
|||||||
factory.registerFunction<FunctionBitmaskToArray>();
|
factory.registerFunction<FunctionBitmaskToArray>();
|
||||||
factory.registerFunction<FunctionToIPv4>();
|
factory.registerFunction<FunctionToIPv4>();
|
||||||
factory.registerFunction<FunctionToIPv6>();
|
factory.registerFunction<FunctionToIPv6>();
|
||||||
factory.registerFunction<FunctionIPv6CIDRtoIPv6Range>();
|
factory.registerFunction<FunctionIPv6CIDRToRange>();
|
||||||
factory.registerFunction<FunctionIPv4CIDRtoIPv4Range>();
|
factory.registerFunction<FunctionIPv4CIDRToRange>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1453,50 +1453,37 @@ public:
|
|||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
constexpr size_t ip_range_tuple_size = 2;
|
class FunctionIPv6CIDRToRange : public IFunction
|
||||||
|
|
||||||
class FunctionIPv6CIDRtoIPv6Range : public IFunction
|
|
||||||
{
|
{
|
||||||
static constexpr size_t bits_in_uint8 = 8;
|
private:
|
||||||
|
/// TODO Inefficient.
|
||||||
|
/// NOTE IPv6 is stored in memory in big endian format that makes some difficulties.
|
||||||
|
static void applyCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst_lower, UInt8 * __restrict dst_upper, UInt8 bits_to_keep)
|
||||||
|
{
|
||||||
|
UInt8 mask[16]{};
|
||||||
|
|
||||||
|
UInt8 bytes_to_keep = bits_to_keep / 8;
|
||||||
|
UInt8 bits_to_keep_in_last_byte = bits_to_keep % 8;
|
||||||
|
|
||||||
|
for (size_t i = 0; i < bits_to_keep / 8; ++i)
|
||||||
|
mask[i] = 0xFFU;
|
||||||
|
|
||||||
|
if (bits_to_keep_in_last_byte)
|
||||||
|
mask[bytes_to_keep] = 0xFFU << (8 - bits_to_keep_in_last_byte);
|
||||||
|
|
||||||
|
for (size_t i = 0; i < 16; ++i)
|
||||||
|
{
|
||||||
|
dst_lower[i] = src[i] & mask[i];
|
||||||
|
dst_upper[i] = dst_lower[i] | ~mask[i];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public:
|
public:
|
||||||
template <bool lower_range>
|
static constexpr auto name = "IPv6CIDRToRange";
|
||||||
static void setCIDRMask(const UInt8 * __restrict src, UInt8 * __restrict dst, UInt8 bits_to_keep)
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6CIDRToRange>(); }
|
||||||
{
|
|
||||||
for (size_t offset = 0, byte_offset = bits_in_uint8; offset != IPV6_BINARY_LENGTH; ++offset, byte_offset += bits_in_uint8)
|
|
||||||
{
|
|
||||||
if (bits_to_keep > byte_offset)
|
|
||||||
{
|
|
||||||
dst[offset] = src[offset];
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/** Check how many bits we need to set the masks, if we got more bits who can be contain in one byte
|
|
||||||
* with our current offset, we just clean the whole byte,
|
|
||||||
*/
|
|
||||||
const size_t shifts_bits = byte_offset - bits_to_keep > bits_in_uint8
|
|
||||||
? bits_in_uint8
|
|
||||||
: byte_offset - bits_to_keep;
|
|
||||||
|
|
||||||
constexpr UInt8 byte_reference = lower_range ? 0 : std::numeric_limits<UInt8>::max();
|
|
||||||
|
|
||||||
/// Clean the bits we don't want on byte
|
|
||||||
const UInt16 src_byte_shift = (static_cast<UInt16>(src[offset]) >> shifts_bits) << shifts_bits;
|
|
||||||
/// Set the CIDR mask.
|
|
||||||
const UInt16 cidr_mask_byte_shift = static_cast<UInt16>(byte_reference) >> (bits_in_uint8 - shifts_bits);
|
|
||||||
|
|
||||||
dst[offset] = static_cast<UInt8>(src_byte_shift | cidr_mask_byte_shift);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
static constexpr auto name = "IPv6CIDRtoIPv6Range";
|
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv6CIDRtoIPv6Range>(); }
|
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
bool isInjective(const Block &) override { return true; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
@ -1508,36 +1495,30 @@ public:
|
|||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
const DataTypePtr & second_argument = arguments[1];
|
const DataTypePtr & second_argument = arguments[1];
|
||||||
if (!isNumber(second_argument))
|
if (!isUInt8(second_argument))
|
||||||
throw Exception{"Illegal type " + second_argument->getName()
|
throw Exception{"Illegal type " + second_argument->getName()
|
||||||
+ " of second argument of function " + getName()
|
+ " of second argument of function " + getName()
|
||||||
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
decltype(arguments) return_type = {
|
DataTypePtr element = DataTypeFactory::instance().get("IPv6");
|
||||||
DataTypeFactory::instance().get("IPv6"),
|
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||||
DataTypeFactory::instance().get("IPv6")
|
|
||||||
};
|
|
||||||
return std::make_shared<DataTypeTuple>(return_type);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
|
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
|
||||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||||
|
|
||||||
const auto col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
|
const auto col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
|
||||||
|
|
||||||
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
|
if (!col_ip_in)
|
||||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
|
||||||
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
|
||||||
|
|
||||||
if (col_ip_in && (col_const_cidr_in || col_cidr_in))
|
|
||||||
{
|
|
||||||
if (col_ip_in->getN() != IPV6_BINARY_LENGTH)
|
if (col_ip_in->getN() != IPV6_BINARY_LENGTH)
|
||||||
throw Exception("Illegal type " + col_type_name_ip.type->getName() +
|
throw Exception("Illegal type " + col_type_name_ip.type->getName() +
|
||||||
" of column " + col_ip_in->getName() +
|
" of column " + col_ip_in->getName() +
|
||||||
@ -1545,80 +1526,66 @@ public:
|
|||||||
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||||
|
|
||||||
const auto & vec_in = col_ip_in->getChars();
|
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
|
||||||
const auto size = vec_in.size();
|
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||||
|
|
||||||
Columns tuple_columns(ip_range_tuple_size);
|
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
||||||
|
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||||
|
|
||||||
|
if (!col_const_cidr_in && !col_cidr_in)
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
|
+ " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
|
const auto & vec_in = col_ip_in->getChars();
|
||||||
|
|
||||||
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||||
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
|
||||||
|
|
||||||
ColumnString::Chars & vec_res_lower_range = col_res_lower_range->getChars();
|
ColumnString::Chars & vec_res_lower_range = col_res_lower_range->getChars();
|
||||||
vec_res_lower_range.resize(size);
|
vec_res_lower_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
|
||||||
|
|
||||||
ColumnString::Chars & vec_res_upper_range = col_res_upper_range->getChars();
|
ColumnString::Chars & vec_res_upper_range = col_res_upper_range->getChars();
|
||||||
vec_res_upper_range.resize(size);
|
vec_res_upper_range.resize(input_rows_count * IPV6_BINARY_LENGTH);
|
||||||
|
|
||||||
for (size_t offset = 0; offset < col_ip_in->size(); ++offset)
|
for (size_t offset = 0; offset < input_rows_count; ++offset)
|
||||||
{
|
{
|
||||||
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
|
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
|
||||||
UInt8 cidr = col_const_cidr_in
|
UInt8 cidr = col_const_cidr_in
|
||||||
? col_const_cidr_in->getValue<UInt8>()
|
? col_const_cidr_in->getValue<UInt8>()
|
||||||
: col_cidr_in->getData()[offset];
|
: col_cidr_in->getData()[offset];
|
||||||
|
|
||||||
setCIDRMask<true>(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], cidr);
|
applyCIDRMask(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
|
||||||
setCIDRMask<false>(&vec_in[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
tuple_columns[0] = std::move(col_res_lower_range);
|
block.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||||
tuple_columns[1] = std::move(col_res_upper_range);
|
|
||||||
|
|
||||||
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
|
|
||||||
}
|
|
||||||
else if (!col_ip_in)
|
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
||||||
+ " of argument of function " + getName(),
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
else
|
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
||||||
+ " of argument of function " + getName(),
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
class FunctionIPv4CIDRtoIPv4Range : public IFunction
|
|
||||||
|
class FunctionIPv4CIDRToRange : public IFunction
|
||||||
{
|
{
|
||||||
static constexpr size_t bits_in_uint32 = 32;
|
private:
|
||||||
|
static inline std::pair<UInt32, UInt32> applyCIDRMask(UInt32 src, UInt8 bits_to_keep)
|
||||||
public:
|
|
||||||
template <bool lower_range>
|
|
||||||
static UInt32 setCIDRMask(UInt32 src, UInt8 bits_to_keep)
|
|
||||||
{
|
{
|
||||||
UInt32 byte_reference = lower_range ? 0 : std::numeric_limits<UInt32>::max();
|
if (bits_to_keep >= 8 * sizeof(UInt32))
|
||||||
|
return { src, src };
|
||||||
|
if (bits_to_keep == 0)
|
||||||
|
return { UInt32(0), UInt32(-1) };
|
||||||
|
|
||||||
if (bits_to_keep >= bits_in_uint32)
|
UInt32 mask = UInt32(-1) << (8 * sizeof(UInt32) - bits_to_keep);
|
||||||
return src;
|
UInt32 lower = src & mask;
|
||||||
|
UInt32 upper = lower | ~mask;
|
||||||
|
|
||||||
const UInt8 shifts_bits = bits_in_uint32 - bits_to_keep;
|
return { lower, upper };
|
||||||
|
|
||||||
/** Using a 32 bits variable with a 32 shits or more is considered as UB
|
|
||||||
* with a 64 bits type casting solve the problem if the cidr mask = 0
|
|
||||||
* Reference : ISO/IEC 9899:1999 6.5.7 Bitwise shift operators P.3
|
|
||||||
*/
|
|
||||||
const UInt64 src_byte_shift = (static_cast<UInt64>(src) >> shifts_bits) << shifts_bits;
|
|
||||||
const UInt64 cidr_mask_byte_shift = static_cast<UInt64>(byte_reference) >> (bits_to_keep);
|
|
||||||
|
|
||||||
return static_cast<UInt32>(src_byte_shift | cidr_mask_byte_shift);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
static constexpr auto name = "IPv4CIDRtoIPv4Range";
|
public:
|
||||||
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4CIDRtoIPv4Range>(); }
|
static constexpr auto name = "IPv4CIDRToRange";
|
||||||
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionIPv4CIDRToRange>(); }
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
size_t getNumberOfArguments() const override { return 2; }
|
size_t getNumberOfArguments() const override { return 2; }
|
||||||
bool isInjective(const Block &) override { return true; }
|
|
||||||
|
|
||||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||||
{
|
{
|
||||||
@ -1630,24 +1597,28 @@ public:
|
|||||||
|
|
||||||
|
|
||||||
const DataTypePtr & second_argument = arguments[1];
|
const DataTypePtr & second_argument = arguments[1];
|
||||||
if (!isNumber(second_argument))
|
if (!isUInt8(second_argument))
|
||||||
throw Exception{"Illegal type " + second_argument->getName()
|
throw Exception{"Illegal type " + second_argument->getName()
|
||||||
+ " of second argument of function " + getName()
|
+ " of second argument of function " + getName()
|
||||||
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
decltype(arguments) return_type = { DataTypeFactory::instance().get("IPv4"), DataTypeFactory::instance().get("IPv4") };
|
DataTypePtr element = DataTypeFactory::instance().get("IPv4");
|
||||||
return std::make_shared<DataTypeTuple>(return_type);
|
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
|
||||||
}
|
}
|
||||||
|
|
||||||
bool useDefaultImplementationForConstants() const override { return true; }
|
bool useDefaultImplementationForConstants() const override { return true; }
|
||||||
|
|
||||||
|
|
||||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||||
{
|
{
|
||||||
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
|
const auto & col_type_name_ip = block.getByPosition(arguments[0]);
|
||||||
const ColumnPtr & column_ip = col_type_name_ip.column;
|
const ColumnPtr & column_ip = col_type_name_ip.column;
|
||||||
|
|
||||||
const auto col_ip_in = checkAndGetColumn<ColumnUInt32>(column_ip.get());
|
const auto col_ip_in = checkAndGetColumn<ColumnUInt32>(column_ip.get());
|
||||||
|
if (!col_ip_in)
|
||||||
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||||
|
+ " of argument of function " + getName(),
|
||||||
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
|
const auto & col_type_name_cidr = block.getByPosition(arguments[1]);
|
||||||
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
const ColumnPtr & column_cidr = col_type_name_cidr.column;
|
||||||
@ -1655,49 +1626,34 @@ public:
|
|||||||
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
const auto col_const_cidr_in = checkAndGetColumnConst<ColumnUInt8>(column_cidr.get());
|
||||||
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
const auto col_cidr_in = checkAndGetColumn<ColumnUInt8>(column_cidr.get());
|
||||||
|
|
||||||
if (col_ip_in && (col_const_cidr_in || col_cidr_in))
|
if (!col_const_cidr_in && !col_cidr_in)
|
||||||
{
|
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
||||||
const auto size = col_ip_in->size();
|
+ " of argument of function " + getName(),
|
||||||
const auto & vec_in = col_ip_in->getData();
|
ErrorCodes::ILLEGAL_COLUMN);
|
||||||
|
|
||||||
Columns tuple_columns(ip_range_tuple_size);
|
const auto & vec_in = col_ip_in->getData();
|
||||||
|
|
||||||
auto col_res_lower_range = ColumnUInt32::create();
|
auto col_res_lower_range = ColumnUInt32::create();
|
||||||
auto col_res_upper_range = ColumnUInt32::create();
|
auto col_res_upper_range = ColumnUInt32::create();
|
||||||
|
|
||||||
auto & vec_res_lower_range = col_res_lower_range->getData();
|
auto & vec_res_lower_range = col_res_lower_range->getData();
|
||||||
vec_res_lower_range.resize(size);
|
vec_res_lower_range.resize(input_rows_count);
|
||||||
|
|
||||||
auto & vec_res_upper_range = col_res_upper_range->getData();
|
auto & vec_res_upper_range = col_res_upper_range->getData();
|
||||||
vec_res_upper_range.resize(size);
|
vec_res_upper_range.resize(input_rows_count);
|
||||||
|
|
||||||
for (size_t i = 0; i < vec_in.size(); ++i)
|
for (size_t i = 0; i < input_rows_count; ++i)
|
||||||
{
|
{
|
||||||
UInt8 cidr = col_const_cidr_in
|
UInt8 cidr = col_const_cidr_in
|
||||||
? col_const_cidr_in->getValue<UInt8>()
|
? col_const_cidr_in->getValue<UInt8>()
|
||||||
: col_cidr_in->getData()[i];
|
: col_cidr_in->getData()[i];
|
||||||
|
|
||||||
vec_res_lower_range[i] = setCIDRMask<true>(vec_in[i], cidr);
|
std::tie(vec_res_lower_range[i], vec_res_upper_range[i]) = applyCIDRMask(vec_in[i], cidr);
|
||||||
vec_res_upper_range[i] = setCIDRMask<false>(vec_in[i], cidr);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
tuple_columns[0] = std::move(col_res_lower_range);
|
block.getByPosition(result).column = ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
|
||||||
tuple_columns[1] = std::move(col_res_upper_range);
|
|
||||||
|
|
||||||
block.getByPosition(result).column = ColumnTuple::create(tuple_columns);
|
|
||||||
}
|
|
||||||
else if (!col_ip_in)
|
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
||||||
+ " of argument of function " + getName(),
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
else
|
|
||||||
throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName()
|
|
||||||
+ " of argument of function " + getName(),
|
|
||||||
ErrorCodes::ILLEGAL_COLUMN);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
27
dbms/tests/performance/basename.xml
Normal file
27
dbms/tests/performance/basename.xml
Normal file
@ -0,0 +1,27 @@
|
|||||||
|
<test>
|
||||||
|
<name>basename</name>
|
||||||
|
<type>loop</type>
|
||||||
|
|
||||||
|
<stop_conditions>
|
||||||
|
<all_of>
|
||||||
|
<iterations>3</iterations>
|
||||||
|
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
|
||||||
|
</all_of>
|
||||||
|
<any_of>
|
||||||
|
<iterations>5</iterations>
|
||||||
|
<total_time_ms>60000</total_time_ms>
|
||||||
|
</any_of>
|
||||||
|
</stop_conditions>
|
||||||
|
|
||||||
|
<main_metric>
|
||||||
|
<min_time/>
|
||||||
|
</main_metric>
|
||||||
|
|
||||||
|
<preconditions>
|
||||||
|
<table_exists>test.hits</table_exists>
|
||||||
|
</preconditions>
|
||||||
|
|
||||||
|
<query>SELECT count() FROM test.hits WHERE NOT ignore(basename(URL))</query>
|
||||||
|
<query>SELECT count() FROM test.hits WHERE NOT ignore(basename(Referer))</query>
|
||||||
|
|
||||||
|
</test>
|
27
dbms/tests/performance/cidr.xml
Normal file
27
dbms/tests/performance/cidr.xml
Normal file
@ -0,0 +1,27 @@
|
|||||||
|
<test>
|
||||||
|
<name>cidr</name>
|
||||||
|
<type>loop</type>
|
||||||
|
|
||||||
|
<stop_conditions>
|
||||||
|
<all_of>
|
||||||
|
<iterations>3</iterations>
|
||||||
|
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
|
||||||
|
</all_of>
|
||||||
|
<any_of>
|
||||||
|
<iterations>5</iterations>
|
||||||
|
<total_time_ms>60000</total_time_ms>
|
||||||
|
</any_of>
|
||||||
|
</stop_conditions>
|
||||||
|
|
||||||
|
<main_metric>
|
||||||
|
<min_time/>
|
||||||
|
</main_metric>
|
||||||
|
|
||||||
|
<preconditions>
|
||||||
|
<table_exists>test.hits</table_exists>
|
||||||
|
</preconditions>
|
||||||
|
|
||||||
|
<query>SELECT count() FROM test.hits WHERE NOT ignore(IPv4CIDRToRange(ClientIP, rand() % 33))</query>
|
||||||
|
<query>SELECT count() FROM test.hits WHERE NOT ignore(IPv6CIDRToRange(ClientIP6, rand() % 33))</query>
|
||||||
|
|
||||||
|
</test>
|
@ -5,21 +5,21 @@ CREATE TABLE ipv4_range(ip IPv4, cidr UInt8) ENGINE = Memory;
|
|||||||
|
|
||||||
INSERT INTO ipv4_range (ip, cidr) VALUES (toIPv4('192.168.5.2'), 0), (toIPv4('192.168.5.20'), 32), (toIPv4('255.255.255.255'), 16), (toIPv4('192.142.32.2'), 32), (toIPv4('192.172.5.2'), 16), (toIPv4('0.0.0.0'), 8), (toIPv4('255.0.0.0'), 4);
|
INSERT INTO ipv4_range (ip, cidr) VALUES (toIPv4('192.168.5.2'), 0), (toIPv4('192.168.5.20'), 32), (toIPv4('255.255.255.255'), 16), (toIPv4('192.142.32.2'), 32), (toIPv4('192.172.5.2'), 16), (toIPv4('0.0.0.0'), 8), (toIPv4('255.0.0.0'), 4);
|
||||||
|
|
||||||
WITH IPv4CIDRtoIPv4Range(toIPv4('192.168.0.0'), 8) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv4CIDRToRange(toIPv4('192.168.0.0'), 8) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv4CIDRtoIPv4Range(toIPv4('192.168.0.0'), 13) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv4CIDRToRange(toIPv4('192.168.0.0'), 13) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv4CIDRtoIPv4Range(toIPv4('192.168.0.0'), 16) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv4CIDRToRange(toIPv4('192.168.0.0'), 16) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv4CIDRtoIPv4Range(toIPv4('192.168.0.0'), 0) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv4CIDRToRange(toIPv4('192.168.0.0'), 0) as ip_range SELECT COUNT(*) FROM ipv4_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv4CIDRtoIPv4Range(ip, cidr) as ip_range SELECT ip, cidr, IPv4NumToString(tupleElement(ip_range, 1)), ip_range FROM ipv4_range;
|
WITH IPv4CIDRToRange(ip, cidr) as ip_range SELECT ip, cidr, IPv4NumToString(tupleElement(ip_range, 1)), ip_range FROM ipv4_range;
|
||||||
|
|
||||||
DROP TABLE ipv4_range;
|
DROP TABLE ipv4_range;
|
||||||
|
|
||||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 0);
|
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 0);
|
||||||
SELEcT IPv4CIDRtoIPv4Range(toIPv4('255.255.255.255'), 8);
|
SELEcT IPv4CIDRToRange(toIPv4('255.255.255.255'), 8);
|
||||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('192.168.5.2'), 32);
|
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 32);
|
||||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('0.0.0.0'), 8);
|
SELECT IPv4CIDRToRange(toIPv4('0.0.0.0'), 8);
|
||||||
SELECT IPv4CIDRtoIPv4Range(toIPv4('255.0.0.0'), 4);
|
SELECT IPv4CIDRToRange(toIPv4('255.0.0.0'), 4);
|
||||||
|
|
||||||
|
@ -5,23 +5,22 @@ CREATE TABLE ipv6_range(ip IPv6, cidr UInt8) ENGINE = Memory;
|
|||||||
|
|
||||||
INSERT INTO ipv6_range (ip, cidr) VALUES (IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0), (IPv6StringToNum('2001:0db8:0000:85a3:ffff:ffff:ffff:ffff'), 32), (IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 16), (IPv6StringToNum('2001:df8:0:85a3::ac1f:8001'), 32), (IPv6StringToNum('2001:0db8:85a3:85a3:0000:0000:ac1f:8001'), 16), (IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8), (IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
|
INSERT INTO ipv6_range (ip, cidr) VALUES (IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0), (IPv6StringToNum('2001:0db8:0000:85a3:ffff:ffff:ffff:ffff'), 32), (IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 16), (IPv6StringToNum('2001:df8:0:85a3::ac1f:8001'), 32), (IPv6StringToNum('2001:0db8:85a3:85a3:0000:0000:ac1f:8001'), 16), (IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8), (IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
|
||||||
|
|
||||||
WITH IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 32) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 25) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 25) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 26) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 26) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 64) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 64) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
WITH IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
WITH IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0) as ip_range SELECT COUNT(*) FROM ipv6_range WHERE ip BETWEEN tupleElement(ip_range, 1) AND tupleElement(ip_range, 2);
|
||||||
|
|
||||||
SELECT IPv6NumToString(ip), cidr, IPv6CIDRtoIPv6Range(ip, cidr) FROM ipv6_range;
|
SELECT IPv6NumToString(ip), cidr, IPv6CIDRToRange(ip, cidr) FROM ipv6_range;
|
||||||
|
|
||||||
DROP TABLE ipv6_range;
|
DROP TABLE ipv6_range;
|
||||||
|
|
||||||
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0);
|
SELECT IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 0);
|
||||||
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128);
|
SELECT IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128);
|
||||||
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 64);
|
SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'), 64);
|
||||||
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8);
|
SELECT IPv6CIDRToRange(IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8);
|
||||||
SELECT IPv6CIDRtoIPv6Range(IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
|
SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user